From 43f5c9a41b50bfea17abea4c015473fa2e668aeb Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 20 Jun 2023 15:49:38 +0300 Subject: [PATCH 01/37] Initial commit --- datafusion/common/src/dfschema.rs | 8 +++ datafusion/core/src/execution/context.rs | 34 +++++++++--- .../physical_optimizer/sort_enforcement.rs | 52 +++++++++++++++++++ datafusion/sql/src/select.rs | 39 +++++++++++++- 4 files changed, 124 insertions(+), 9 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 292c19886b723..f9d7647b946db 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -650,6 +650,14 @@ impl DFField { } } + /// Create a qualified field from an existing Arrow field + pub fn from_field(field: impl Into) -> Self { + Self { + qualifier: None, + field: field.into(), + } + } + /// Returns an immutable reference to the `DFField`'s unqualified name pub fn name(&self) -> &String { self.field.name() diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 8d9fdc2e537cd..f5d1bb08cac06 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -86,7 +86,7 @@ use crate::physical_planner::PhysicalPlanner; use crate::variable::{VarProvider, VarType}; use async_trait::async_trait; use chrono::{DateTime, Utc}; -use datafusion_common::{OwnedTableReference, SchemaReference}; +use datafusion_common::{DFField, DFSchema, OwnedTableReference, SchemaReference}; use datafusion_sql::{ parser::DFParser, planner::{ContextProvider, SqlToRel}, @@ -473,12 +473,6 @@ impl SessionContext { primary_key, } = cmd; - if !primary_key.is_empty() { - Err(DataFusionError::Execution( - "Primary keys on MemoryTables are not currently supported!".to_string(), - ))?; - } - let input = Arc::try_unwrap(input).unwrap_or_else(|e| e.as_ref().clone()); let input = self.state().optimize(&input)?; let table = self.table(&name).await; @@ -500,7 +494,31 @@ impl SessionContext { "'IF NOT EXISTS' cannot coexist with 'REPLACE'".to_string(), )), (_, _, Err(_)) => { - let schema = Arc::new(input.schema().as_ref().into()); + let df_schema = input.schema(); + let mut new_fields = df_schema.fields().to_vec(); + // Update metadata of PRIMARY_KEY fields. + for dffield in new_fields.iter_mut() { + if primary_key + .iter() + .any(|item| item.flat_name() == dffield.qualified_name()) + { + let field = dffield.field(); + let mut metadata = field.metadata().clone(); + metadata.insert("primary_key".to_string(), "true".to_string()); + let updated_field = + field.as_ref().clone().with_metadata(metadata); + let qualifier = dffield.qualifier().cloned(); + *dffield = if let Some(qualifier) = qualifier { + DFField::from_qualified(qualifier, updated_field) + } else { + DFField::from_field(updated_field) + }; + } + } + let metadata = df_schema.metadata().clone(); + let updated_schema = DFSchema::new_with_metadata(new_fields, metadata)?; + + let schema = Arc::new(updated_schema.into()); let physical = DataFrame::new(self.state(), input); let batches: Vec<_> = physical.collect_partitioned().await?; diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index a79552de49de2..bd537928aadff 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2893,3 +2893,55 @@ mod tests { ) } } + +#[cfg(test)] +mod tmp_tests { + use crate::physical_plan::{collect, displayable, ExecutionPlan}; + use crate::prelude::SessionContext; + use arrow::util::pretty::print_batches; + use datafusion_common::Result; + use datafusion_execution::config::SessionConfig; + use std::sync::Arc; + + fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) + } + + #[tokio::test] + #[ignore] + async fn test_primary_key_aggregation() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + ctx.sql( + "CREATE TABLE sales_global ( + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount INT, + primary key(sn) + ) as VALUES + (1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), + (3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00)", + ) + .await?; + + let sql = "SELECT s.sn, s.amount, 2*s.sn + FROM sales_global AS s + GROUP BY sn"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + println!("----------------------PLAN----------------------"); + print_plan(&physical_plan)?; + let batches = collect(physical_plan, ctx.task_ctx()).await?; + print_batches(&batches)?; + assert_eq!(0, 1); + Ok(()) + } +} diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index f12830df4254c..6f7bbe642edc6 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -428,9 +428,46 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: LogicalPlan, select_exprs: &[Expr], having_expr_opt: Option<&Expr>, - group_by_exprs: Vec, + mut group_by_exprs: Vec, aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { + println!("group_by_exprs:{:?}", group_by_exprs); + println!("select_exprs:{:?}", select_exprs); + println!("input_schema:{:?}", input.schema()); + let field_names = input + .schema() + .fields() + .iter() + .map(|elem| elem.qualified_name()) + .collect::>(); + println!("field_names:{:?}", field_names); + // TODO: Add primary key check + let group_by_contains_primary = group_by_exprs.iter().any(|expr| { + let res = input.schema().fields(); + res.iter().any(|item| { + let expr_name = format!("{:?}", expr); + println!("expr name:{:?}", expr_name); + println!("item.qualified_name():{:?}", item.qualified_name()); + println!("item.field().metadata(){:?}", item.field().metadata()); + let is_primary_key = item + .field() + .metadata() + .get("primary_key") + .map(|val| val == "true") + .unwrap_or(false); + item.qualified_name() == expr_name && is_primary_key + }) + }); + if group_by_contains_primary { + for expr in select_exprs { + if !group_by_exprs.contains(expr) + && field_names.contains(&format!("{:?}", expr)) + { + group_by_exprs.push(expr.clone()); + } + } + } + println!("group_by_exprs:{:?}", group_by_exprs); // create the aggregate plan let plan = LogicalPlanBuilder::from(input.clone()) .aggregate(group_by_exprs.clone(), aggr_exprs.clone())? From 769307619bde27202e2e83b34809363e72af10b4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 21 Jun 2023 17:47:25 +0300 Subject: [PATCH 02/37] Add primary key to DFSchema --- datafusion/common/src/dfschema.rs | 23 ++++++-- datafusion/core/src/dataframe.rs | 1 + .../core/src/datasource/listing/helpers.rs | 1 + datafusion/core/src/execution/context.rs | 54 ++++++++++++------- datafusion/core/src/physical_planner.rs | 21 +++++++- datafusion/expr/src/logical_plan/builder.rs | 46 ++++++++++++++-- datafusion/expr/src/logical_plan/plan.rs | 5 +- datafusion/expr/src/utils.rs | 3 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 4 +- .../optimizer/src/push_down_projection.rs | 2 + .../src/single_distinct_to_groupby.rs | 32 ++++++++--- datafusion/sql/src/select.rs | 15 +++--- 13 files changed, 160 insertions(+), 49 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index f9d7647b946db..8c8e68cccec74 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -40,6 +40,7 @@ pub struct DFSchema { fields: Vec, /// Additional metadata in form of key value pairs metadata: HashMap, + primary_keys: Vec, } impl DFSchema { @@ -48,20 +49,23 @@ impl DFSchema { Self { fields: vec![], metadata: HashMap::new(), + primary_keys: vec![], } } #[deprecated(since = "7.0.0", note = "please use `new_with_metadata` instead")] /// Create a new `DFSchema` pub fn new(fields: Vec) -> Result { - Self::new_with_metadata(fields, HashMap::new()) + Self::new_with_metadata(fields, HashMap::new(), vec![]) } /// Create a new `DFSchema` pub fn new_with_metadata( fields: Vec, metadata: HashMap, + primary_keys: Vec, ) -> Result { + println!("new_with_metadata is called, primary_keys: {:?}", primary_keys); let mut qualified_names = HashSet::new(); let mut unqualified_names = HashSet::new(); @@ -97,13 +101,14 @@ impl DFSchema { )); } } - Ok(Self { fields, metadata }) + Ok(Self { fields, metadata, primary_keys }) } /// Create a `DFSchema` from an Arrow schema and a given qualifier pub fn try_from_qualified_schema<'a>( qualifier: impl Into>, schema: &Schema, + primary_keys: Vec, ) -> Result { let qualifier = qualifier.into(); Self::new_with_metadata( @@ -113,6 +118,7 @@ impl DFSchema { .map(|f| DFField::from_qualified(qualifier.clone(), f.clone())) .collect(), schema.metadata().clone(), + primary_keys, ) } @@ -123,7 +129,10 @@ impl DFSchema { let mut metadata = self.metadata.clone(); fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); - Self::new_with_metadata(fields, metadata) + let mut primary_keys = self.primary_keys.clone(); + let other_primary_keys = schema.primary_keys.iter().map(|idx| idx + self.fields.len()).collect::>(); + primary_keys.extend(other_primary_keys); + Self::new_with_metadata(fields, metadata, primary_keys) } /// Modify this schema by appending the fields from the supplied schema, ignoring any @@ -475,6 +484,11 @@ impl DFSchema { pub fn metadata(&self) -> &HashMap { &self.metadata } + + /// Get primary keys + pub fn primary_keys(&self) -> &[usize] { + &self.primary_keys + } } impl From for Schema { @@ -504,6 +518,7 @@ impl TryFrom for DFSchema { .map(|f| DFField::from(f.clone())) .collect(), schema.metadata().clone(), + vec![], ) } } @@ -555,7 +570,7 @@ impl ToDFSchema for SchemaRef { impl ToDFSchema for Vec { fn to_dfschema(self) -> Result { - DFSchema::new_with_metadata(self, HashMap::new()) + DFSchema::new_with_metadata(self, HashMap::new(), vec![]) } } diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 2834fb571fd79..d9b201579eae1 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -95,6 +95,7 @@ impl DataFrame { /// Create a physical plan pub async fn create_physical_plan(self) -> Result> { + println!("logical plan schema:{}", self.plan.schema()); self.session_state.create_physical_plan(&self.plan).await } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index efe1f7b59afbc..1fa2b71d5d3c1 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -266,6 +266,7 @@ async fn prune_partitions( .map(|(n, d)| DFField::new_unqualified(n, d.clone(), true)) .collect(), Default::default(), + Default::default(), )?; let batch = RecordBatch::try_new(schema.clone(), arrays)?; diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index f5d1bb08cac06..43318f4096d41 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -496,28 +496,42 @@ impl SessionContext { (_, _, Err(_)) => { let df_schema = input.schema(); let mut new_fields = df_schema.fields().to_vec(); - // Update metadata of PRIMARY_KEY fields. - for dffield in new_fields.iter_mut() { - if primary_key - .iter() - .any(|item| item.flat_name() == dffield.qualified_name()) - { - let field = dffield.field(); - let mut metadata = field.metadata().clone(); - metadata.insert("primary_key".to_string(), "true".to_string()); - let updated_field = - field.as_ref().clone().with_metadata(metadata); - let qualifier = dffield.qualifier().cloned(); - *dffield = if let Some(qualifier) = qualifier { - DFField::from_qualified(qualifier, updated_field) - } else { - DFField::from_field(updated_field) - }; + println!("ctx, new_fields:{:?}", new_fields); + println!("ctx, primary_key:{:?}", primary_key); + let primary_keys = primary_key.iter().map(|pk|{ + if let Some(idx) = new_fields.iter().position(|item| { + println!("item.qualified_name():{:?}, pk.flat_name():{:?}", item.qualified_name(), pk.flat_name()); + item.qualified_name() == pk.flat_name() + }){ + println!("match found"); + Ok(idx) + } else { + Err(DataFusionError::Execution("Primary Key doesn't exist".to_string())) } - } + }).collect::>>()?; + println!("ctx, primary_keys: {:?}", primary_keys); + // // Update metadata of PRIMARY_KEY fields. + // for dffield in new_fields.iter_mut() { + // if primary_key + // .iter() + // .any(|item| item.flat_name() == dffield.qualified_name()) + // { + // let field = dffield.field(); + // let mut metadata = field.metadata().clone(); + // metadata.insert("primary_key".to_string(), "true".to_string()); + // let updated_field = + // field.as_ref().clone().with_metadata(metadata); + // let qualifier = dffield.qualifier().cloned(); + // *dffield = if let Some(qualifier) = qualifier { + // DFField::from_qualified(qualifier, updated_field) + // } else { + // DFField::from_field(updated_field) + // }; + // } + // } let metadata = df_schema.metadata().clone(); - let updated_schema = DFSchema::new_with_metadata(new_fields, metadata)?; - + let updated_schema = DFSchema::new_with_metadata(new_fields, metadata, primary_keys)?; + println!("updated_schema:{:?}", updated_schema); let schema = Arc::new(updated_schema.into()); let physical = DataFrame::new(self.state(), input); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 3cb6efc6541aa..86c878d56e1c0 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1025,10 +1025,17 @@ impl DefaultPhysicalPlanner { )) ) .unzip(); + let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices); + let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices); + // offset indices of right + let right_primary_keys = right_primary_keys.iter().map(|item| item + left_field_indices.len()); + let mut primary_keys = vec![]; + primary_keys.extend(left_primary_keys); + primary_keys.extend(right_primary_keys); // Construct intermediate schemas used for filtering data and // convert logical expression to physical according to filter schema - let filter_df_schema = DFSchema::new_with_metadata(filter_df_fields, HashMap::new())?; + let filter_df_schema = DFSchema::new_with_metadata(filter_df_fields, HashMap::new(), primary_keys)?; let filter_schema = Schema::new_with_metadata(filter_fields, HashMap::new()); let filter_expr = create_physical_expr( expr, @@ -1299,6 +1306,17 @@ impl DefaultPhysicalPlanner { } } +fn get_updated_primary_keys(df_schema: &DFSchema, field_indices: &[usize]) -> Vec{ + let mut primary_keys = vec![]; + let existing_primary_keys = df_schema.primary_keys(); + for (idx, field_idx) in field_indices.iter().enumerate(){ + if existing_primary_keys.contains(field_idx){ + primary_keys.push(idx); + } + } + primary_keys +} + /// Expand and align a GROUPING SET expression. /// (see ) /// @@ -2453,6 +2471,7 @@ mod tests { DFSchema::new_with_metadata( vec![DFField::new_unqualified("a", DataType::Int32, false)], HashMap::new(), + vec![], ) .unwrap(), ), diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 08a167ff0de1e..79abc81b26c50 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -187,7 +187,7 @@ impl LogicalPlanBuilder { values[i][j] = Expr::Literal(ScalarValue::try_from(fields[j].data_type())?); } let schema = - DFSchemaRef::new(DFSchema::new_with_metadata(fields, HashMap::new())?); + DFSchemaRef::new(DFSchema::new_with_metadata(fields, HashMap::new(), vec![])?); Ok(Self::from(LogicalPlan::Values(Values { schema, values }))) } @@ -276,10 +276,11 @@ impl LogicalPlanBuilder { }) .collect(), schema.metadata().clone(), + vec![], ) }) .unwrap_or_else(|| { - DFSchema::try_from_qualified_schema(table_name.clone(), &schema) + DFSchema::try_from_qualified_schema(table_name.clone(), &schema, vec![]) })?; let table_scan = LogicalPlan::TableScan(TableScan { @@ -829,11 +830,12 @@ impl LogicalPlanBuilder { let mut window_fields: Vec = self.plan.schema().fields().clone(); window_fields.extend_from_slice(&exprlist_to_fields(all_expr, &self.plan)?); let metadata = self.plan.schema().metadata().clone(); + let primary_keys = self.plan.schema().primary_keys().to_vec(); Ok(Self::from(LogicalPlan::Window(Window { input: Arc::new(self.plan), window_expr, - schema: Arc::new(DFSchema::new_with_metadata(window_fields, metadata)?), + schema: Arc::new(DFSchema::new_with_metadata(window_fields, metadata, primary_keys)?), }))) } @@ -1048,7 +1050,6 @@ pub fn build_join_schema( let right_fields = right.fields(); let left_fields = left.fields(); - let fields: Vec = match join_type { JoinType::Inner => { // left then right @@ -1092,9 +1093,42 @@ pub fn build_join_schema( } }; + let right_primary_keys = right.primary_keys().iter().map(|idx| idx+left_fields.len()).collect::>(); + let left_primary_keys = left.primary_keys(); + let primary_keys: Vec = match join_type { + JoinType::Inner => { + // left then right + left_primary_keys + .iter() + .chain(right_primary_keys.iter()) + .cloned() + .collect() + } + JoinType::Left => { + // left then right, right set to nullable in case of not matched scenario + left_primary_keys.to_vec() + } + JoinType::Right => { + // left then right, left set to nullable in case of not matched scenario + right_primary_keys.to_vec() + } + JoinType::Full => { + // left then right, all set to nullable in case of not matched scenario + vec![] + } + JoinType::LeftSemi | JoinType::LeftAnti => { + // Only use the left side for the schema + left_primary_keys.to_vec() + } + JoinType::RightSemi | JoinType::RightAnti => { + // Only use the right side for the schema + right_primary_keys.to_vec() + } + }; + let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); - DFSchema::new_with_metadata(fields, metadata) + DFSchema::new_with_metadata(fields, metadata, primary_keys) } /// Errors if one or more expressions have equal names. @@ -1246,6 +1280,7 @@ pub fn project( let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), + plan.schema().primary_keys().to_vec(), )?; Ok(LogicalPlan::Projection(Projection::try_new_with_schema( @@ -1412,6 +1447,7 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new(DFSchema::new_with_metadata( fields, input_schema.metadata().clone(), + input_schema.primary_keys().to_vec() )?); Ok(LogicalPlan::Unnest(Unnest { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 4f21ad38c4d2f..dcff478678bfc 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1246,6 +1246,7 @@ impl Projection { let schema = Arc::new(DFSchema::new_with_metadata( exprlist_to_fields(&expr, &input)?, input.schema().metadata().clone(), + input.schema().primary_keys().to_vec(), )?); Self::try_new_with_schema(expr, input, schema) } @@ -1309,8 +1310,9 @@ impl SubqueryAlias { ) -> Result { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); + let primary_keys = plan.schema().primary_keys().to_vec(); let schema = - DFSchemaRef::new(DFSchema::try_from_qualified_schema(&alias, &schema)?); + DFSchemaRef::new(DFSchema::try_from_qualified_schema(&alias, &schema, primary_keys)?); Ok(SubqueryAlias { input: Arc::new(plan), alias, @@ -1570,6 +1572,7 @@ impl Aggregate { let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), + input.schema().primary_keys().to_vec(), )?; Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 05ebab3d9998c..79de3329261f1 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -447,7 +447,7 @@ pub fn expand_qualified_wildcard( ))); } let qualified_schema = - DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())?; + DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone(), schema.primary_keys().to_vec())?; let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, @@ -1014,6 +1014,7 @@ pub fn from_plan( let schema = Arc::new(DFSchema::new_with_metadata( fields, input.schema().metadata().clone(), + input.schema().primary_keys().to_vec(), )?); Ok(LogicalPlan::Unnest(Unnest { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 51354cb666612..1e0391a6a7fa0 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -261,7 +261,7 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { }) .collect::>(); DFSchemaRef::new( - DFSchema::new_with_metadata(new_fields, schema.metadata().clone()).unwrap(), + DFSchema::new_with_metadata(new_fields, schema.metadata().clone(), schema.primary_keys().to_vec()).unwrap(), ) } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 31f3c119be46d..3278fc7042c56 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -74,6 +74,7 @@ fn analyze_internal( external_schema: &DFSchema, plan: &LogicalPlan, ) -> Result { + println!("external_schema: {:?}", external_schema); // optimize child plans first let new_inputs = plan .inputs() @@ -85,8 +86,9 @@ fn analyze_internal( let mut schema = merge_schema(new_inputs.iter().collect()); if let LogicalPlan::TableScan(ts) = plan { + // arrow schema doesn't support primary_key. We do not know primary_key of ts.source if any. let source_schema = - DFSchema::try_from_qualified_schema(&ts.table_name, &ts.source.schema())?; + DFSchema::try_from_qualified_schema(&ts.table_name, &ts.source.schema(), vec![])?; schema.merge(&source_schema); } diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 16ddb9b41f41a..c113a55201b49 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -227,6 +227,8 @@ impl OptimizerRule for PushDownProjection { let schema = DFSchema::new_with_metadata( exprlist_to_fields(&projection_column_exprs, child_plan)?, union.schema.metadata().clone(), + // Union doesn't preserve primary key + vec![], )?; let new_union = LogicalPlan::Union(Union { inputs, diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index ba7e89094b0f3..f142d7040dbf0 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -19,7 +19,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{DFSchema, Result}; +use datafusion_common::{DFField, DFSchema, Result}; use datafusion_expr::{ col, expr::AggregateFunction, @@ -157,9 +157,13 @@ impl OptimizerRule for SingleDistinctToGroupBy { .iter() .map(|expr| expr.to_field(input.schema())) .collect::>>()?; + + let primary_keys = get_updated_primary_keys(input.schema(), &inner_fields); + let inner_schema = DFSchema::new_with_metadata( inner_fields, input.schema().metadata().clone(), + primary_keys )?; let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), @@ -167,13 +171,17 @@ impl OptimizerRule for SingleDistinctToGroupBy { Vec::new(), )?); + let outer_fields = outer_group_exprs + .iter() + .chain(new_aggr_exprs.iter()) + .map(|expr| expr.to_field(&inner_schema)) + .collect::>>()?; + + let primary_keys = get_updated_primary_keys(&inner_schema, &outer_fields); let outer_aggr_schema = Arc::new(DFSchema::new_with_metadata( - outer_group_exprs - .iter() - .chain(new_aggr_exprs.iter()) - .map(|expr| expr.to_field(&inner_schema)) - .collect::>>()?, + outer_fields, input.schema().metadata().clone(), + primary_keys, )?); // so the aggregates are displayed in the same way even after the rewrite @@ -225,6 +233,18 @@ impl OptimizerRule for SingleDistinctToGroupBy { } } +fn get_updated_primary_keys(old_schema: &DFSchema, new_fields: &[DFField]) -> Vec{ + let old_primary_keys = old_schema.primary_keys(); + let fields = old_schema.fields(); + let mut primary_keys = vec![]; + for pk in old_primary_keys{ + if let Some(idx) = new_fields.iter().position(|item | item == &fields[*pk]){ + primary_keys.push(idx); + } + } + primary_keys +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 6f7bbe642edc6..6c4af064b77a9 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -434,6 +434,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { println!("group_by_exprs:{:?}", group_by_exprs); println!("select_exprs:{:?}", select_exprs); println!("input_schema:{:?}", input.schema()); + println!("aggregate primary keys:{:?}", input.schema().primary_keys()); + let primary_keys = input.schema().primary_keys(); + let input_fields = input.schema().fields(); + let input_primary_key_fields = primary_keys.iter().map(|idx| &input_fields[*idx]).collect::>(); let field_names = input .schema() .fields() @@ -443,19 +447,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { println!("field_names:{:?}", field_names); // TODO: Add primary key check let group_by_contains_primary = group_by_exprs.iter().any(|expr| { - let res = input.schema().fields(); - res.iter().any(|item| { + input_primary_key_fields.iter().any(|item| { let expr_name = format!("{:?}", expr); println!("expr name:{:?}", expr_name); println!("item.qualified_name():{:?}", item.qualified_name()); println!("item.field().metadata(){:?}", item.field().metadata()); - let is_primary_key = item - .field() - .metadata() - .get("primary_key") - .map(|val| val == "true") - .unwrap_or(false); - item.qualified_name() == expr_name && is_primary_key + item.qualified_name() == expr_name }) }); if group_by_contains_primary { From ddde0c47bd5db79a1e6384a40683586b98957acd Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 14:12:51 +0300 Subject: [PATCH 03/37] store primary key in metadata during schema --- datafusion/common/src/dfschema.rs | 47 +++++++++++++++++-- datafusion/core/src/execution/context.rs | 35 +++++++++----- datafusion/core/src/physical_planner.rs | 6 +-- datafusion/expr/src/logical_plan/builder.rs | 40 +++++++++++++--- datafusion/expr/src/logical_plan/plan.rs | 7 ++- datafusion/expr/src/utils.rs | 7 ++- .../src/analyzer/count_wildcard_rule.rs | 7 ++- .../optimizer/src/analyzer/type_coercion.rs | 7 ++- .../src/single_distinct_to_groupby.rs | 14 +++--- datafusion/sql/src/select.rs | 12 ++++- 10 files changed, 141 insertions(+), 41 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 8c8e68cccec74..c2d30af502fe1 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -65,7 +65,10 @@ impl DFSchema { metadata: HashMap, primary_keys: Vec, ) -> Result { - println!("new_with_metadata is called, primary_keys: {:?}", primary_keys); + println!( + "new_with_metadata is called, primary_keys: {:?}", + primary_keys + ); let mut qualified_names = HashSet::new(); let mut unqualified_names = HashSet::new(); @@ -101,7 +104,11 @@ impl DFSchema { )); } } - Ok(Self { fields, metadata, primary_keys }) + Ok(Self { + fields, + metadata, + primary_keys, + }) } /// Create a `DFSchema` from an Arrow schema and a given qualifier @@ -130,7 +137,11 @@ impl DFSchema { fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); let mut primary_keys = self.primary_keys.clone(); - let other_primary_keys = schema.primary_keys.iter().map(|idx| idx + self.fields.len()).collect::>(); + let other_primary_keys = schema + .primary_keys + .iter() + .map(|idx| idx + self.fields.len()) + .collect::>(); primary_keys.extend(other_primary_keys); Self::new_with_metadata(fields, metadata, primary_keys) } @@ -491,19 +502,45 @@ impl DFSchema { } } +fn encode_primary_key_to_metadata( + metadata: &mut HashMap, + primary_keys: &[usize], +) { + let mut pks = String::new(); + // Store primary key information in the metadata during conversion + for (idx, pk) in primary_keys.iter().enumerate() { + pks = format!("{pks}{pk}"); + if idx < primary_keys.len() - 1 { + pks = format!("{pks}, "); + } + } + if !pks.is_empty() { + metadata.insert("primary_keys".to_string(), pks); + } + println!("DFSchema to Schema conversion"); +} + impl From for Schema { /// Convert DFSchema into a Schema fn from(df_schema: DFSchema) -> Self { + let mut metadata = df_schema.metadata; + if !df_schema.primary_keys.is_empty() { + encode_primary_key_to_metadata(&mut metadata, &df_schema.primary_keys); + } let fields: Fields = df_schema.fields.into_iter().map(|f| f.field).collect(); - Schema::new_with_metadata(fields, df_schema.metadata) + Schema::new_with_metadata(fields, metadata) } } impl From<&DFSchema> for Schema { /// Convert DFSchema reference into a Schema fn from(df_schema: &DFSchema) -> Self { + let mut metadata = df_schema.metadata.clone(); + if !df_schema.primary_keys.is_empty() { + encode_primary_key_to_metadata(&mut metadata, &df_schema.primary_keys); + } let fields: Fields = df_schema.fields.iter().map(|f| f.field.clone()).collect(); - Schema::new_with_metadata(fields, df_schema.metadata.clone()) + Schema::new_with_metadata(fields, metadata) } } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 43318f4096d41..3fc991c4924de 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -498,17 +498,26 @@ impl SessionContext { let mut new_fields = df_schema.fields().to_vec(); println!("ctx, new_fields:{:?}", new_fields); println!("ctx, primary_key:{:?}", primary_key); - let primary_keys = primary_key.iter().map(|pk|{ - if let Some(idx) = new_fields.iter().position(|item| { - println!("item.qualified_name():{:?}, pk.flat_name():{:?}", item.qualified_name(), pk.flat_name()); - item.qualified_name() == pk.flat_name() - }){ - println!("match found"); - Ok(idx) - } else { - Err(DataFusionError::Execution("Primary Key doesn't exist".to_string())) - } - }).collect::>>()?; + let primary_keys = primary_key + .iter() + .map(|pk| { + if let Some(idx) = new_fields.iter().position(|item| { + println!( + "item.qualified_name():{:?}, pk.flat_name():{:?}", + item.qualified_name(), + pk.flat_name() + ); + item.qualified_name() == pk.flat_name() + }) { + println!("match found"); + Ok(idx) + } else { + Err(DataFusionError::Execution( + "Primary Key doesn't exist".to_string(), + )) + } + }) + .collect::>>()?; println!("ctx, primary_keys: {:?}", primary_keys); // // Update metadata of PRIMARY_KEY fields. // for dffield in new_fields.iter_mut() { @@ -530,9 +539,11 @@ impl SessionContext { // } // } let metadata = df_schema.metadata().clone(); - let updated_schema = DFSchema::new_with_metadata(new_fields, metadata, primary_keys)?; + let updated_schema = + DFSchema::new_with_metadata(new_fields, metadata, primary_keys)?; println!("updated_schema:{:?}", updated_schema); let schema = Arc::new(updated_schema.into()); + println!("schema:{:?}", schema); let physical = DataFrame::new(self.state(), input); let batches: Vec<_> = physical.collect_partitioned().await?; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 86c878d56e1c0..f31cc33f1dad9 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1306,11 +1306,11 @@ impl DefaultPhysicalPlanner { } } -fn get_updated_primary_keys(df_schema: &DFSchema, field_indices: &[usize]) -> Vec{ +fn get_updated_primary_keys(df_schema: &DFSchema, field_indices: &[usize]) -> Vec { let mut primary_keys = vec![]; let existing_primary_keys = df_schema.primary_keys(); - for (idx, field_idx) in field_indices.iter().enumerate(){ - if existing_primary_keys.contains(field_idx){ + for (idx, field_idx) in field_indices.iter().enumerate() { + if existing_primary_keys.contains(field_idx) { primary_keys.push(idx); } } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 79abc81b26c50..dc33eff6e4fff 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -186,8 +186,11 @@ impl LogicalPlanBuilder { for (i, j) in nulls { values[i][j] = Expr::Literal(ScalarValue::try_from(fields[j].data_type())?); } - let schema = - DFSchemaRef::new(DFSchema::new_with_metadata(fields, HashMap::new(), vec![])?); + let schema = DFSchemaRef::new(DFSchema::new_with_metadata( + fields, + HashMap::new(), + vec![], + )?); Ok(Self::from(LogicalPlan::Values(Values { schema, values }))) } @@ -263,6 +266,17 @@ impl LogicalPlanBuilder { let schema = table_source.schema(); + let primary_keys = if let Some(pks) = schema.metadata().get("primary_keys") { + println!("pks: {:?}", pks); + let pks: Vec = + pks.split(',').map(|s| s.trim().parse().unwrap()).collect(); + + println!("pks:{:?}", pks); + pks + } else { + vec![] + }; + let projected_schema = projection .as_ref() .map(|p| { @@ -276,11 +290,15 @@ impl LogicalPlanBuilder { }) .collect(), schema.metadata().clone(), - vec![], + primary_keys.clone(), ) }) .unwrap_or_else(|| { - DFSchema::try_from_qualified_schema(table_name.clone(), &schema, vec![]) + DFSchema::try_from_qualified_schema( + table_name.clone(), + &schema, + primary_keys, + ) })?; let table_scan = LogicalPlan::TableScan(TableScan { @@ -835,7 +853,11 @@ impl LogicalPlanBuilder { Ok(Self::from(LogicalPlan::Window(Window { input: Arc::new(self.plan), window_expr, - schema: Arc::new(DFSchema::new_with_metadata(window_fields, metadata, primary_keys)?), + schema: Arc::new(DFSchema::new_with_metadata( + window_fields, + metadata, + primary_keys, + )?), }))) } @@ -1093,7 +1115,11 @@ pub fn build_join_schema( } }; - let right_primary_keys = right.primary_keys().iter().map(|idx| idx+left_fields.len()).collect::>(); + let right_primary_keys = right + .primary_keys() + .iter() + .map(|idx| idx + left_fields.len()) + .collect::>(); let left_primary_keys = left.primary_keys(); let primary_keys: Vec = match join_type { JoinType::Inner => { @@ -1447,7 +1473,7 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new(DFSchema::new_with_metadata( fields, input_schema.metadata().clone(), - input_schema.primary_keys().to_vec() + input_schema.primary_keys().to_vec(), )?); Ok(LogicalPlan::Unnest(Unnest { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index dcff478678bfc..0c21d9d7543f2 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1311,8 +1311,11 @@ impl SubqueryAlias { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); let primary_keys = plan.schema().primary_keys().to_vec(); - let schema = - DFSchemaRef::new(DFSchema::try_from_qualified_schema(&alias, &schema, primary_keys)?); + let schema = DFSchemaRef::new(DFSchema::try_from_qualified_schema( + &alias, + &schema, + primary_keys, + )?); Ok(SubqueryAlias { input: Arc::new(plan), alias, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 79de3329261f1..573c36c9e6294 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -446,8 +446,11 @@ pub fn expand_qualified_wildcard( "Invalid qualifier {qualifier}" ))); } - let qualified_schema = - DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone(), schema.primary_keys().to_vec())?; + let qualified_schema = DFSchema::new_with_metadata( + qualified_fields, + schema.metadata().clone(), + schema.primary_keys().to_vec(), + )?; let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 1e0391a6a7fa0..cf7f5684ed6fd 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -261,7 +261,12 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { }) .collect::>(); DFSchemaRef::new( - DFSchema::new_with_metadata(new_fields, schema.metadata().clone(), schema.primary_keys().to_vec()).unwrap(), + DFSchema::new_with_metadata( + new_fields, + schema.metadata().clone(), + schema.primary_keys().to_vec(), + ) + .unwrap(), ) } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 3278fc7042c56..9025d9bc254fe 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -87,8 +87,11 @@ fn analyze_internal( if let LogicalPlan::TableScan(ts) = plan { // arrow schema doesn't support primary_key. We do not know primary_key of ts.source if any. - let source_schema = - DFSchema::try_from_qualified_schema(&ts.table_name, &ts.source.schema(), vec![])?; + let source_schema = DFSchema::try_from_qualified_schema( + &ts.table_name, + &ts.source.schema(), + vec![], + )?; schema.merge(&source_schema); } diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index f142d7040dbf0..22010fbdb9bb9 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -158,12 +158,13 @@ impl OptimizerRule for SingleDistinctToGroupBy { .map(|expr| expr.to_field(input.schema())) .collect::>>()?; - let primary_keys = get_updated_primary_keys(input.schema(), &inner_fields); + let primary_keys = + get_updated_primary_keys(input.schema(), &inner_fields); let inner_schema = DFSchema::new_with_metadata( inner_fields, input.schema().metadata().clone(), - primary_keys + primary_keys, )?; let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), @@ -177,7 +178,8 @@ impl OptimizerRule for SingleDistinctToGroupBy { .map(|expr| expr.to_field(&inner_schema)) .collect::>>()?; - let primary_keys = get_updated_primary_keys(&inner_schema, &outer_fields); + let primary_keys = + get_updated_primary_keys(&inner_schema, &outer_fields); let outer_aggr_schema = Arc::new(DFSchema::new_with_metadata( outer_fields, input.schema().metadata().clone(), @@ -233,12 +235,12 @@ impl OptimizerRule for SingleDistinctToGroupBy { } } -fn get_updated_primary_keys(old_schema: &DFSchema, new_fields: &[DFField]) -> Vec{ +fn get_updated_primary_keys(old_schema: &DFSchema, new_fields: &[DFField]) -> Vec { let old_primary_keys = old_schema.primary_keys(); let fields = old_schema.fields(); let mut primary_keys = vec![]; - for pk in old_primary_keys{ - if let Some(idx) = new_fields.iter().position(|item | item == &fields[*pk]){ + for pk in old_primary_keys { + if let Some(idx) = new_fields.iter().position(|item| item == &fields[*pk]) { primary_keys.push(idx); } } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 6c4af064b77a9..19976b54666a3 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -435,9 +435,19 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { println!("select_exprs:{:?}", select_exprs); println!("input_schema:{:?}", input.schema()); println!("aggregate primary keys:{:?}", input.schema().primary_keys()); + if let Some(pks) = input.schema().metadata().get("primary_keys") { + println!("pks: {:?}", pks); + let pks: Vec = + pks.split(',').map(|s| s.trim().parse().unwrap()).collect(); + + println!("pks:{:?}", pks); + } let primary_keys = input.schema().primary_keys(); let input_fields = input.schema().fields(); - let input_primary_key_fields = primary_keys.iter().map(|idx| &input_fields[*idx]).collect::>(); + let input_primary_key_fields = primary_keys + .iter() + .map(|idx| &input_fields[*idx]) + .collect::>(); let field_names = input .schema() .fields() From 16db70dac969598905d8a267aca696d6ae8f9303 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 14:44:06 +0300 Subject: [PATCH 04/37] all tests pass --- datafusion/common/src/dfschema.rs | 21 +++---- .../core/src/datasource/listing/helpers.rs | 1 - datafusion/core/src/execution/context.rs | 47 +++----------- datafusion/core/src/physical_planner.rs | 5 +- datafusion/core/tests/sql/timestamp.rs | 6 +- datafusion/expr/src/logical_plan/builder.rs | 62 +++++++++---------- datafusion/expr/src/logical_plan/plan.rs | 25 ++++---- datafusion/expr/src/utils.rs | 17 +++-- .../src/analyzer/count_wildcard_rule.rs | 9 +-- .../optimizer/src/analyzer/type_coercion.rs | 7 +-- datafusion/optimizer/src/optimizer.rs | 4 +- .../optimizer/src/push_down_projection.rs | 2 - .../src/single_distinct_to_groupby.rs | 16 ++--- 13 files changed, 89 insertions(+), 133 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index c2d30af502fe1..a5f2f5da8b561 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -56,19 +56,14 @@ impl DFSchema { #[deprecated(since = "7.0.0", note = "please use `new_with_metadata` instead")] /// Create a new `DFSchema` pub fn new(fields: Vec) -> Result { - Self::new_with_metadata(fields, HashMap::new(), vec![]) + Self::new_with_metadata(fields, HashMap::new()) } /// Create a new `DFSchema` pub fn new_with_metadata( fields: Vec, metadata: HashMap, - primary_keys: Vec, ) -> Result { - println!( - "new_with_metadata is called, primary_keys: {:?}", - primary_keys - ); let mut qualified_names = HashSet::new(); let mut unqualified_names = HashSet::new(); @@ -107,7 +102,7 @@ impl DFSchema { Ok(Self { fields, metadata, - primary_keys, + primary_keys: vec![], }) } @@ -115,7 +110,6 @@ impl DFSchema { pub fn try_from_qualified_schema<'a>( qualifier: impl Into>, schema: &Schema, - primary_keys: Vec, ) -> Result { let qualifier = qualifier.into(); Self::new_with_metadata( @@ -125,10 +119,14 @@ impl DFSchema { .map(|f| DFField::from_qualified(qualifier.clone(), f.clone())) .collect(), schema.metadata().clone(), - primary_keys, ) } + pub fn with_primary_keys(mut self, primary_keys: Vec) -> Self { + self.primary_keys = primary_keys; + self + } + /// Create a new schema that contains the fields from this schema followed by the fields /// from the supplied schema. An error will be returned if there are duplicate field names. pub fn join(&self, schema: &DFSchema) -> Result { @@ -143,7 +141,7 @@ impl DFSchema { .map(|idx| idx + self.fields.len()) .collect::>(); primary_keys.extend(other_primary_keys); - Self::new_with_metadata(fields, metadata, primary_keys) + Ok(Self::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) } /// Modify this schema by appending the fields from the supplied schema, ignoring any @@ -555,7 +553,6 @@ impl TryFrom for DFSchema { .map(|f| DFField::from(f.clone())) .collect(), schema.metadata().clone(), - vec![], ) } } @@ -607,7 +604,7 @@ impl ToDFSchema for SchemaRef { impl ToDFSchema for Vec { fn to_dfschema(self) -> Result { - DFSchema::new_with_metadata(self, HashMap::new(), vec![]) + DFSchema::new_with_metadata(self, HashMap::new()) } } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 1fa2b71d5d3c1..efe1f7b59afbc 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -266,7 +266,6 @@ async fn prune_partitions( .map(|(n, d)| DFField::new_unqualified(n, d.clone(), true)) .collect(), Default::default(), - Default::default(), )?; let batch = RecordBatch::try_new(schema.clone(), arrays)?; diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 3fc991c4924de..79bdb5c9ab3ba 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -86,7 +86,7 @@ use crate::physical_planner::PhysicalPlanner; use crate::variable::{VarProvider, VarType}; use async_trait::async_trait; use chrono::{DateTime, Utc}; -use datafusion_common::{DFField, DFSchema, OwnedTableReference, SchemaReference}; +use datafusion_common::{DFSchema, OwnedTableReference, SchemaReference}; use datafusion_sql::{ parser::DFParser, planner::{ContextProvider, SqlToRel}, @@ -495,21 +495,15 @@ impl SessionContext { )), (_, _, Err(_)) => { let df_schema = input.schema(); - let mut new_fields = df_schema.fields().to_vec(); - println!("ctx, new_fields:{:?}", new_fields); - println!("ctx, primary_key:{:?}", primary_key); + let new_fields = df_schema.fields().to_vec(); + let primary_keys = primary_key .iter() .map(|pk| { - if let Some(idx) = new_fields.iter().position(|item| { - println!( - "item.qualified_name():{:?}, pk.flat_name():{:?}", - item.qualified_name(), - pk.flat_name() - ); - item.qualified_name() == pk.flat_name() - }) { - println!("match found"); + if let Some(idx) = new_fields + .iter() + .position(|item| item.qualified_name() == pk.flat_name()) + { Ok(idx) } else { Err(DataFusionError::Execution( @@ -518,32 +512,11 @@ impl SessionContext { } }) .collect::>>()?; - println!("ctx, primary_keys: {:?}", primary_keys); - // // Update metadata of PRIMARY_KEY fields. - // for dffield in new_fields.iter_mut() { - // if primary_key - // .iter() - // .any(|item| item.flat_name() == dffield.qualified_name()) - // { - // let field = dffield.field(); - // let mut metadata = field.metadata().clone(); - // metadata.insert("primary_key".to_string(), "true".to_string()); - // let updated_field = - // field.as_ref().clone().with_metadata(metadata); - // let qualifier = dffield.qualifier().cloned(); - // *dffield = if let Some(qualifier) = qualifier { - // DFField::from_qualified(qualifier, updated_field) - // } else { - // DFField::from_field(updated_field) - // }; - // } - // } + let metadata = df_schema.metadata().clone(); - let updated_schema = - DFSchema::new_with_metadata(new_fields, metadata, primary_keys)?; - println!("updated_schema:{:?}", updated_schema); + let updated_schema = DFSchema::new_with_metadata(new_fields, metadata)? + .with_primary_keys(primary_keys); let schema = Arc::new(updated_schema.into()); - println!("schema:{:?}", schema); let physical = DataFrame::new(self.state(), input); let batches: Vec<_> = physical.collect_partitioned().await?; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index f31cc33f1dad9..4db1e9c8c3ff2 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1035,7 +1035,7 @@ impl DefaultPhysicalPlanner { // Construct intermediate schemas used for filtering data and // convert logical expression to physical according to filter schema - let filter_df_schema = DFSchema::new_with_metadata(filter_df_fields, HashMap::new(), primary_keys)?; + let filter_df_schema = DFSchema::new_with_metadata(filter_df_fields, HashMap::new())?.with_primary_keys(primary_keys); let filter_schema = Schema::new_with_metadata(filter_fields, HashMap::new()); let filter_expr = create_physical_expr( expr, @@ -2218,7 +2218,7 @@ mod tests { dict_id: 0, \ dict_is_ordered: false, \ metadata: {} } }\ - ], metadata: {} }, \ + ], metadata: {}, primary_keys: [] }, \ ExecutionPlan schema: Schema { fields: [\ Field { \ name: \"b\", \ @@ -2471,7 +2471,6 @@ mod tests { DFSchema::new_with_metadata( vec![DFField::new_unqualified("a", DataType::Int32, false)], HashMap::new(), - vec![], ) .unwrap(), ), diff --git a/datafusion/core/tests/sql/timestamp.rs b/datafusion/core/tests/sql/timestamp.rs index bb4d54a61c2a2..b2b6c57351a59 100644 --- a/datafusion/core/tests/sql/timestamp.rs +++ b/datafusion/core/tests/sql/timestamp.rs @@ -568,7 +568,7 @@ async fn timestamp_sub_interval_days() -> Result<()> { async fn timestamp_add_interval_months() -> Result<()> { let ctx = SessionContext::new(); - let sql = "SELECT NOW(), NOW() + INTERVAL '17' MONTH;"; + let sql = "SELECT NOW(), NOW() + INTERVAL '18' MONTH;"; let results = execute_to_batches(&ctx, sql).await; let actual = result_vec(&results); @@ -579,8 +579,8 @@ async fn timestamp_add_interval_months() -> Result<()> { let t1_naive = chrono::NaiveDateTime::parse_from_str(res1, format).unwrap(); let t2_naive = chrono::NaiveDateTime::parse_from_str(res2, format).unwrap(); - let year = t1_naive.year() + (t1_naive.month() as i32 + 17) / 12; - let month = (t1_naive.month() + 17) % 12; + let year = t1_naive.year() + (t1_naive.month() as i32 + 18) / 12; + let month = (t1_naive.month() + 18) % 12; assert_eq!( t1_naive.with_year(year).unwrap().with_month(month).unwrap(), diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index dc33eff6e4fff..b03a114a14c2a 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -186,11 +186,8 @@ impl LogicalPlanBuilder { for (i, j) in nulls { values[i][j] = Expr::Literal(ScalarValue::try_from(fields[j].data_type())?); } - let schema = DFSchemaRef::new(DFSchema::new_with_metadata( - fields, - HashMap::new(), - vec![], - )?); + let schema = + DFSchemaRef::new(DFSchema::new_with_metadata(fields, HashMap::new())?); Ok(Self::from(LogicalPlan::Values(Values { schema, values }))) } @@ -280,24 +277,25 @@ impl LogicalPlanBuilder { let projected_schema = projection .as_ref() .map(|p| { - DFSchema::new_with_metadata( - p.iter() - .map(|i| { - DFField::from_qualified( - table_name.clone(), - schema.field(*i).clone(), - ) - }) - .collect(), - schema.metadata().clone(), - primary_keys.clone(), + Ok::( + DFSchema::new_with_metadata( + p.iter() + .map(|i| { + DFField::from_qualified( + table_name.clone(), + schema.field(*i).clone(), + ) + }) + .collect(), + schema.metadata().clone(), + )? + .with_primary_keys(primary_keys.clone()), ) }) .unwrap_or_else(|| { - DFSchema::try_from_qualified_schema( - table_name.clone(), - &schema, - primary_keys, + Ok( + DFSchema::try_from_qualified_schema(table_name.clone(), &schema)? + .with_primary_keys(primary_keys), ) })?; @@ -853,11 +851,10 @@ impl LogicalPlanBuilder { Ok(Self::from(LogicalPlan::Window(Window { input: Arc::new(self.plan), window_expr, - schema: Arc::new(DFSchema::new_with_metadata( - window_fields, - metadata, - primary_keys, - )?), + schema: Arc::new( + DFSchema::new_with_metadata(window_fields, metadata)? + .with_primary_keys(primary_keys), + ), }))) } @@ -1154,7 +1151,7 @@ pub fn build_join_schema( let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); - DFSchema::new_with_metadata(fields, metadata, primary_keys) + Ok(DFSchema::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) } /// Errors if one or more expressions have equal names. @@ -1306,8 +1303,8 @@ pub fn project( let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), - plan.schema().primary_keys().to_vec(), - )?; + )? + .with_primary_keys(plan.schema().primary_keys().to_vec()); Ok(LogicalPlan::Projection(Projection::try_new_with_schema( projected_expr, @@ -1470,11 +1467,10 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { }) .collect::>(); - let schema = Arc::new(DFSchema::new_with_metadata( - fields, - input_schema.metadata().clone(), - input_schema.primary_keys().to_vec(), - )?); + let schema = Arc::new( + DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? + .with_primary_keys(input_schema.primary_keys().to_vec()), + ); 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 0c21d9d7543f2..5c29d5d25ceae 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1243,11 +1243,13 @@ 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(), - input.schema().primary_keys().to_vec(), - )?); + let schema = Arc::new( + DFSchema::new_with_metadata( + exprlist_to_fields(&expr, &input)?, + input.schema().metadata().clone(), + )? + .with_primary_keys(input.schema().primary_keys().to_vec()), + ); Self::try_new_with_schema(expr, input, schema) } @@ -1311,11 +1313,10 @@ impl SubqueryAlias { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); let primary_keys = plan.schema().primary_keys().to_vec(); - let schema = DFSchemaRef::new(DFSchema::try_from_qualified_schema( - &alias, - &schema, - primary_keys, - )?); + let schema = DFSchemaRef::new( + DFSchema::try_from_qualified_schema(&alias, &schema)? + .with_primary_keys(primary_keys), + ); Ok(SubqueryAlias { input: Arc::new(plan), alias, @@ -1575,8 +1576,8 @@ impl Aggregate { let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), - input.schema().primary_keys().to_vec(), - )?; + )? + .with_primary_keys(input.schema().primary_keys().to_vec()); Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 573c36c9e6294..5a30f51cdddff 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -446,11 +446,9 @@ pub fn expand_qualified_wildcard( "Invalid qualifier {qualifier}" ))); } - let qualified_schema = DFSchema::new_with_metadata( - qualified_fields, - schema.metadata().clone(), - schema.primary_keys().to_vec(), - )?; + let qualified_schema = + DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? + .with_primary_keys(schema.primary_keys().to_vec()); let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, @@ -1014,11 +1012,10 @@ pub fn from_plan( }) .collect::>(); - let schema = Arc::new(DFSchema::new_with_metadata( - fields, - input.schema().metadata().clone(), - input.schema().primary_keys().to_vec(), - )?); + let schema = Arc::new( + DFSchema::new_with_metadata(fields, input.schema().metadata().clone())? + .with_primary_keys(input.schema().primary_keys().to_vec()), + ); Ok(LogicalPlan::Unnest(Unnest { input, diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index cf7f5684ed6fd..e90ef315eb36a 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -261,12 +261,9 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { }) .collect::>(); DFSchemaRef::new( - DFSchema::new_with_metadata( - new_fields, - schema.metadata().clone(), - schema.primary_keys().to_vec(), - ) - .unwrap(), + DFSchema::new_with_metadata(new_fields, schema.metadata().clone()) + .unwrap() + .with_primary_keys(schema.primary_keys().to_vec()), ) } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 9025d9bc254fe..096c9858e433d 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -87,11 +87,8 @@ fn analyze_internal( if let LogicalPlan::TableScan(ts) = plan { // arrow schema doesn't support primary_key. We do not know primary_key of ts.source if any. - let source_schema = DFSchema::try_from_qualified_schema( - &ts.table_name, - &ts.source.schema(), - vec![], - )?; + let source_schema = + DFSchema::try_from_qualified_schema(&ts.table_name, &ts.source.schema())?; schema.merge(&source_schema); } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 2306593d424b3..fe159bf54b56f 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -493,12 +493,12 @@ mod tests { assert_eq!( "get table_scan rule\ncaused by\n\ Internal error: Optimizer rule 'get table_scan rule' failed, due to generate a different schema, \ - original schema: DFSchema { fields: [], metadata: {} }, \ + original schema: DFSchema { fields: [], metadata: {}, primary_keys: [] }, \ 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: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \ - metadata: {} }. \ + metadata: {}, primary_keys: [] }. \ This was likely caused by a bug in DataFusion's code \ and we would welcome that you file an bug report in our issue tracker", err.to_string() diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index c113a55201b49..16ddb9b41f41a 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -227,8 +227,6 @@ impl OptimizerRule for PushDownProjection { let schema = DFSchema::new_with_metadata( exprlist_to_fields(&projection_column_exprs, child_plan)?, union.schema.metadata().clone(), - // Union doesn't preserve primary key - vec![], )?; let new_union = LogicalPlan::Union(Union { inputs, diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 22010fbdb9bb9..d741be4c9f9ba 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -164,8 +164,8 @@ impl OptimizerRule for SingleDistinctToGroupBy { let inner_schema = DFSchema::new_with_metadata( inner_fields, input.schema().metadata().clone(), - primary_keys, - )?; + )? + .with_primary_keys(primary_keys); let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), inner_group_exprs, @@ -180,11 +180,13 @@ impl OptimizerRule for SingleDistinctToGroupBy { let primary_keys = get_updated_primary_keys(&inner_schema, &outer_fields); - let outer_aggr_schema = Arc::new(DFSchema::new_with_metadata( - outer_fields, - input.schema().metadata().clone(), - primary_keys, - )?); + let outer_aggr_schema = Arc::new( + DFSchema::new_with_metadata( + outer_fields, + input.schema().metadata().clone(), + )? + .with_primary_keys(primary_keys), + ); // so the aggregates are displayed in the same way even after the rewrite // this optimizer has two kinds of alias: From b401c67f8d672ba5eee162699adbf4e52619a8d4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 14:56:25 +0300 Subject: [PATCH 05/37] simplifications --- datafusion/core/src/dataframe.rs | 1 - datafusion/expr/src/logical_plan/builder.rs | 7 +---- .../optimizer/src/analyzer/type_coercion.rs | 1 - datafusion/sql/src/select.rs | 26 ++++--------------- 4 files changed, 6 insertions(+), 29 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index d9b201579eae1..2834fb571fd79 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -95,7 +95,6 @@ impl DataFrame { /// Create a physical plan pub async fn create_physical_plan(self) -> Result> { - println!("logical plan schema:{}", self.plan.schema()); self.session_state.create_physical_plan(&self.plan).await } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index b03a114a14c2a..f7a17f32800da 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -264,12 +264,7 @@ impl LogicalPlanBuilder { let schema = table_source.schema(); let primary_keys = if let Some(pks) = schema.metadata().get("primary_keys") { - println!("pks: {:?}", pks); - let pks: Vec = - pks.split(',').map(|s| s.trim().parse().unwrap()).collect(); - - println!("pks:{:?}", pks); - pks + pks.split(',').map(|s| s.trim().parse().unwrap()).collect() } else { vec![] }; diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 096c9858e433d..5abe332a3d777 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -74,7 +74,6 @@ fn analyze_internal( external_schema: &DFSchema, plan: &LogicalPlan, ) -> Result { - println!("external_schema: {:?}", external_schema); // optimize child plans first let new_inputs = plan .inputs() diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 19976b54666a3..a702317ebbd72 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -431,37 +431,22 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { mut group_by_exprs: Vec, aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { - println!("group_by_exprs:{:?}", group_by_exprs); - println!("select_exprs:{:?}", select_exprs); - println!("input_schema:{:?}", input.schema()); - println!("aggregate primary keys:{:?}", input.schema().primary_keys()); - if let Some(pks) = input.schema().metadata().get("primary_keys") { - println!("pks: {:?}", pks); - let pks: Vec = - pks.split(',').map(|s| s.trim().parse().unwrap()).collect(); - - println!("pks:{:?}", pks); - } - let primary_keys = input.schema().primary_keys(); - let input_fields = input.schema().fields(); + let schema = input.schema(); + let primary_keys = schema.primary_keys(); + let input_fields = schema.fields(); let input_primary_key_fields = primary_keys .iter() .map(|idx| &input_fields[*idx]) .collect::>(); - let field_names = input - .schema() + let field_names = schema .fields() .iter() .map(|elem| elem.qualified_name()) .collect::>(); - println!("field_names:{:?}", field_names); - // TODO: Add primary key check + let group_by_contains_primary = group_by_exprs.iter().any(|expr| { input_primary_key_fields.iter().any(|item| { let expr_name = format!("{:?}", expr); - println!("expr name:{:?}", expr_name); - println!("item.qualified_name():{:?}", item.qualified_name()); - println!("item.field().metadata(){:?}", item.field().metadata()); item.qualified_name() == expr_name }) }); @@ -474,7 +459,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } } - println!("group_by_exprs:{:?}", group_by_exprs); // create the aggregate plan let plan = LogicalPlanBuilder::from(input.clone()) .aggregate(group_by_exprs.clone(), aggr_exprs.clone())? From dd9715f19e6596175b67c207c20e451e0e7bf4f2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 15:21:39 +0300 Subject: [PATCH 06/37] Move test to the .slt file --- datafusion/common/src/dfschema.rs | 29 +--------- datafusion/core/src/execution/context.rs | 28 +++++++++- .../physical_optimizer/sort_enforcement.rs | 52 ------------------ .../sqllogictests/test_files/groupby.slt | 53 +++++++++++++++++++ 4 files changed, 82 insertions(+), 80 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index a5f2f5da8b561..6c1457945552c 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -500,43 +500,18 @@ impl DFSchema { } } -fn encode_primary_key_to_metadata( - metadata: &mut HashMap, - primary_keys: &[usize], -) { - let mut pks = String::new(); - // Store primary key information in the metadata during conversion - for (idx, pk) in primary_keys.iter().enumerate() { - pks = format!("{pks}{pk}"); - if idx < primary_keys.len() - 1 { - pks = format!("{pks}, "); - } - } - if !pks.is_empty() { - metadata.insert("primary_keys".to_string(), pks); - } - println!("DFSchema to Schema conversion"); -} - impl From for Schema { /// Convert DFSchema into a Schema fn from(df_schema: DFSchema) -> Self { - let mut metadata = df_schema.metadata; - if !df_schema.primary_keys.is_empty() { - encode_primary_key_to_metadata(&mut metadata, &df_schema.primary_keys); - } let fields: Fields = df_schema.fields.into_iter().map(|f| f.field).collect(); - Schema::new_with_metadata(fields, metadata) + Schema::new_with_metadata(fields, df_schema.metadata) } } impl From<&DFSchema> for Schema { /// Convert DFSchema reference into a Schema fn from(df_schema: &DFSchema) -> Self { - let mut metadata = df_schema.metadata.clone(); - if !df_schema.primary_keys.is_empty() { - encode_primary_key_to_metadata(&mut metadata, &df_schema.primary_keys); - } + let metadata = df_schema.metadata.clone(); let fields: Fields = df_schema.fields.iter().map(|f| f.field.clone()).collect(); Schema::new_with_metadata(fields, metadata) } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 79bdb5c9ab3ba..7f519f884ee98 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -513,9 +513,15 @@ impl SessionContext { }) .collect::>>()?; - let metadata = df_schema.metadata().clone(); + let mut metadata = df_schema.metadata().clone(); + if !primary_keys.is_empty() { + // Store primary key information in schema also. By this way we can keep track of primary key + // after DFSchema is converted to arrow::Schema + encode_primary_key_to_metadata(&mut metadata, &primary_keys) + } let updated_schema = DFSchema::new_with_metadata(new_fields, metadata)? .with_primary_keys(primary_keys); + let schema = Arc::new(updated_schema.into()); let physical = DataFrame::new(self.state(), input); @@ -2059,6 +2065,26 @@ impl SerializerRegistry for EmptySerializerRegistry { } } +/// writes primary key indices as comma seperated integers to the +/// metadata. If columns a 0 and 2 are primary keys, +/// "primary_keys" -> "0, 2" will be added to metadata. +fn encode_primary_key_to_metadata( + metadata: &mut HashMap, + primary_keys: &[usize], +) { + let mut pks = String::new(); + // Store primary key information in the metadata during conversion + for (idx, pk) in primary_keys.iter().enumerate() { + pks = format!("{pks}{pk}"); + if idx < primary_keys.len() - 1 { + pks = format!("{pks}, "); + } + } + if !pks.is_empty() { + metadata.insert("primary_keys".to_string(), pks); + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index bd537928aadff..a79552de49de2 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2893,55 +2893,3 @@ mod tests { ) } } - -#[cfg(test)] -mod tmp_tests { - use crate::physical_plan::{collect, displayable, ExecutionPlan}; - use crate::prelude::SessionContext; - use arrow::util::pretty::print_batches; - use datafusion_common::Result; - use datafusion_execution::config::SessionConfig; - use std::sync::Arc; - - fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent().to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) - } - - #[tokio::test] - #[ignore] - async fn test_primary_key_aggregation() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - ctx.sql( - "CREATE TABLE sales_global ( - sn INT, - ts TIMESTAMP, - currency VARCHAR(3), - amount INT, - primary key(sn) - ) as VALUES - (1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), - (3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00)", - ) - .await?; - - let sql = "SELECT s.sn, s.amount, 2*s.sn - FROM sales_global AS s - GROUP BY sn"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - println!("----------------------PLAN----------------------"); - print_plan(&physical_plan)?; - let batches = collect(physical_plan, ctx.task_ctx()).await?; - print_batches(&batches)?; - assert_eq!(0, 1); - Ok(()) - } -} diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index ca400e0ef4994..e2bcbc056ee50 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2589,3 +2589,56 @@ SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, LAST_VALUE(amount ORDER BY amount DESC) AS fv2 FROM sales_global GROUP BY country + +# create a table for testing +statement ok +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) + +# when group by contains primary key expression +# we can use all the expressions in the table during selection +# (not just group by expressions + aggregation result) +query TT +EXPLAIN SELECT s.sn, s.amount, 2*s.sn + FROM sales_global_with_pk AS s + GROUP BY sn + ORDER BY sn +---- +logical_plan +Sort: s.sn ASC NULLS LAST +--Projection: s.sn, s.amount, Int64(2) * CAST(s.sn AS Int64) +----Aggregate: groupBy=[[s.sn, s.amount]], aggr=[[]] +------SubqueryAlias: s +--------TableScan: sales_global_with_pk projection=[sn, amount] +physical_plan +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 +----------RepartitionExec: partitioning=Hash([Column { name: "sn", index: 0 }, Column { name: "amount", index: 1 }], 2), input_partitions=2 +------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +--------------MemoryExec: partitions=2, partition_sizes=[1, 0] + +query IRI +SELECT s.sn, s.amount, 2*s.sn + FROM sales_global_with_pk AS s + GROUP BY sn + ORDER BY sn +---- +0 30 0 +1 50 2 +2 75 4 +3 200 6 +4 100 8 From 945029f511f94c56d2fa9615a9c0e8eb788d0375 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 16:05:31 +0300 Subject: [PATCH 07/37] simplifications --- datafusion/common/src/dfschema.rs | 11 +---------- datafusion/core/src/execution/context.rs | 7 ++++--- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 56eeef79ffb05..416da39299380 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -511,9 +511,8 @@ impl From for Schema { impl From<&DFSchema> for Schema { /// Convert DFSchema reference into a Schema fn from(df_schema: &DFSchema) -> Self { - let metadata = df_schema.metadata.clone(); let fields: Fields = df_schema.fields.iter().map(|f| f.field.clone()).collect(); - Schema::new_with_metadata(fields, metadata) + Schema::new_with_metadata(fields, df_schema.metadata.clone()) } } @@ -674,14 +673,6 @@ impl DFField { } } - /// Create a qualified field from an existing Arrow field - pub fn from_field(field: impl Into) -> Self { - Self { - qualifier: None, - field: field.into(), - } - } - /// Returns an immutable reference to the `DFField`'s unqualified name pub fn name(&self) -> &String { self.field.name() diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index cb9b4c8fa91dd..8455439e05519 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -495,12 +495,12 @@ impl SessionContext { )), (_, _, Err(_)) => { let df_schema = input.schema(); - let new_fields = df_schema.fields().to_vec(); let primary_keys = primary_key .iter() .map(|pk| { - if let Some(idx) = new_fields + if let Some(idx) = df_schema + .fields() .iter() .position(|item| item.qualified_name() == pk.flat_name()) { @@ -519,7 +519,8 @@ impl SessionContext { // after DFSchema is converted to arrow::Schema encode_primary_key_to_metadata(&mut metadata, &primary_keys) } - let updated_schema = DFSchema::new_with_metadata(new_fields, metadata)? + let fields = df_schema.fields().to_vec(); + let updated_schema = DFSchema::new_with_metadata(fields, metadata)? .with_primary_keys(primary_keys); let schema = Arc::new(updated_schema.into()); From 72eba733e79ccdd8ae4ad9661794874a92f73f35 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 16:28:30 +0300 Subject: [PATCH 08/37] Simplifications --- datafusion/core/src/physical_planner.rs | 6 ++++ datafusion/expr/src/logical_plan/builder.rs | 40 +++++++++++---------- 2 files changed, 28 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index df1efb5b6725c..d92fa8fefd53c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1025,6 +1025,7 @@ impl DefaultPhysicalPlanner { )) ) .unzip(); + // Update indices of the fields according to filter schema (the index seen in the filter schema). let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices); let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices); // offset indices of right @@ -1306,6 +1307,11 @@ impl DefaultPhysicalPlanner { } } +/// Update primary key indices, with index of the number in `field_indices` +/// If `field_indices` is [2, 5, 8], primary keys are [4, 5, 7] in the `df_schema`. +/// return value will be `1`. This means that 1st index of the `field_indices`(5) is primary key +/// e.g inside primary_keys vector ([4, 5, 7]). In the updated schema, fields at the indices [2, 5, 8] will +/// be at [0, 1, 2]. fn get_updated_primary_keys(df_schema: &DFSchema, field_indices: &[usize]) -> Vec { let mut primary_keys = vec![]; let existing_primary_keys = df_schema.primary_keys(); diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index d5e37a1f9d95e..54d2a932b3c13 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -265,7 +265,14 @@ impl LogicalPlanBuilder { let schema = table_source.schema(); let primary_keys = if let Some(pks) = schema.metadata().get("primary_keys") { - pks.split(',').map(|s| s.trim().parse().unwrap()).collect() + pks.split(',') + .map(|s| { + s.trim() + .parse() + // Convert parsing error to Datafusion Error + .map_err(|e| DataFusionError::External(Box::new(e))) + }) + .collect::>>()? } else { vec![] }; @@ -273,26 +280,22 @@ impl LogicalPlanBuilder { let projected_schema = projection .as_ref() .map(|p| { - Ok::( - DFSchema::new_with_metadata( - p.iter() - .map(|i| { - DFField::from_qualified( - table_name.clone(), - schema.field(*i).clone(), - ) - }) - .collect(), - schema.metadata().clone(), - )? - .with_primary_keys(primary_keys.clone()), + 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_primary_keys(primary_keys.clone())) }) .unwrap_or_else(|| { - Ok( - DFSchema::try_from_qualified_schema(table_name.clone(), &schema)? - .with_primary_keys(primary_keys), - ) + DFSchema::try_from_qualified_schema(table_name.clone(), &schema) + .map(|df_schema| df_schema.with_primary_keys(primary_keys)) })?; let table_scan = LogicalPlan::TableScan(TableScan { @@ -1055,6 +1058,7 @@ pub fn build_join_schema( let right_fields = right.fields(); let left_fields = left.fields(); + let fields: Vec = match join_type { JoinType::Inner => { // left then right From 511353e855bf474472f7a757c9322f095ebba95d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jul 2023 18:04:52 +0300 Subject: [PATCH 09/37] primary key with associated expression indices --- datafusion/common/src/dfschema.rs | 23 ++++--- .../src/datasource/default_table_source.rs | 4 ++ datafusion/core/src/datasource/memory.rs | 12 ++++ datafusion/core/src/datasource/provider.rs | 6 ++ datafusion/core/src/execution/context.rs | 50 ++++++--------- datafusion/core/src/physical_plan/memory.rs | 1 - datafusion/core/src/physical_planner.rs | 29 ++++++--- .../sqllogictests/test_files/groupby.slt | 56 +++++++++++++++++ datafusion/expr/src/logical_plan/builder.rs | 61 +++++++++---------- datafusion/expr/src/logical_plan/plan.rs | 8 +-- datafusion/expr/src/table_source.rs | 6 ++ datafusion/expr/src/utils.rs | 4 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- datafusion/optimizer/src/optimizer.rs | 4 +- .../src/single_distinct_to_groupby.rs | 23 +++++-- datafusion/sql/src/select.rs | 40 ++++++------ 16 files changed, 217 insertions(+), 112 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 416da39299380..509fb4397b567 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -40,7 +40,8 @@ pub struct DFSchema { fields: Vec, /// Additional metadata in form of key value pairs metadata: HashMap, - primary_keys: Vec, + /// primary key index and its associated fields indices + primary_keys: HashMap>, } impl DFSchema { @@ -49,7 +50,7 @@ impl DFSchema { Self { fields: vec![], metadata: HashMap::new(), - primary_keys: vec![], + primary_keys: HashMap::new(), } } @@ -102,7 +103,7 @@ impl DFSchema { Ok(Self { fields, metadata, - primary_keys: vec![], + primary_keys: HashMap::new(), }) } @@ -122,7 +123,7 @@ impl DFSchema { ) } - pub fn with_primary_keys(mut self, primary_keys: Vec) -> Self { + pub fn with_primary_keys(mut self, primary_keys: HashMap>) -> Self { self.primary_keys = primary_keys; self } @@ -138,8 +139,16 @@ impl DFSchema { let other_primary_keys = schema .primary_keys .iter() - .map(|idx| idx + self.fields.len()) - .collect::>(); + .map(|(idx, associations)| { + ( + idx + self.fields.len(), + associations + .iter() + .map(|item| item + self.fields.len()) + .collect::>(), + ) + }) + .collect::>>(); primary_keys.extend(other_primary_keys); Ok(Self::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) } @@ -495,7 +504,7 @@ impl DFSchema { } /// Get primary keys - pub fn primary_keys(&self) -> &[usize] { + pub fn primary_keys(&self) -> &HashMap> { &self.primary_keys } } diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index c6fd87e7f18b3..6e40f4aa78453 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -52,6 +52,10 @@ impl TableSource for DefaultTableSource { self.table_provider.schema() } + fn primary_keys(&self) -> &[usize] { + self.table_provider.primary_keys() + } + /// Tests whether the table provider can make use of any or all filter expressions /// to optimise data retrieval. fn supports_filters_pushdown( diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 784aa2aff2324..60bfe599366f4 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -52,6 +52,7 @@ pub type PartitionData = Arc>>; pub struct MemTable { schema: SchemaRef, pub(crate) batches: Vec, + primary_keys: Vec, } impl MemTable { @@ -76,9 +77,16 @@ impl MemTable { .into_iter() .map(|e| Arc::new(RwLock::new(e))) .collect::>(), + primary_keys: vec![], }) } + /// Assign primary keys + pub fn with_primary_keys(mut self, primary_keys: Vec) -> Self { + self.primary_keys = primary_keys; + self + } + /// Create a mem table by reading from another data source pub async fn load( t: Arc, @@ -148,6 +156,10 @@ impl TableProvider for MemTable { self.schema.clone() } + fn primary_keys(&self) -> &[usize] { + &self.primary_keys + } + fn table_type(&self) -> TableType { TableType::Base } diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 11f30f33d1399..3483b6f9ba343 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -41,6 +41,12 @@ pub trait TableProvider: Sync + Send { /// Get a reference to the schema for this table fn schema(&self) -> SchemaRef; + /// Get a reference to the primary_key indices + fn primary_keys(&self) -> &[usize] { + // By default return None + &[] + } + /// Get the type of this table for metadata/catalog purposes. fn table_type(&self) -> TableType; diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 8455439e05519..fc7392005913d 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -513,21 +513,31 @@ impl SessionContext { }) .collect::>>()?; - let mut metadata = df_schema.metadata().clone(); - if !primary_keys.is_empty() { - // Store primary key information in schema also. By this way we can keep track of primary key - // after DFSchema is converted to arrow::Schema - encode_primary_key_to_metadata(&mut metadata, &primary_keys) - } let fields = df_schema.fields().to_vec(); - let updated_schema = DFSchema::new_with_metadata(fields, metadata)? - .with_primary_keys(primary_keys); - + let primary_keys_with_associations = primary_keys + .iter() + .map(|idx| (*idx, (0..fields.len()).collect::>())) + .collect(); + let updated_schema = + DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? + .with_primary_keys(primary_keys_with_associations); + + // let mut schema: Schema = updated_schema.into(); + // if !primary_keys.is_empty() { + // // Store primary key information in schema also. By this way we can keep track of primary key + // // after DFSchema is converted to arrow::Schema + // let mut metadata = schema.metadata().clone(); + // encode_primary_key_to_metadata(&mut metadata, &primary_keys); + // schema = Schema::new_with_metadata(schema.fields, metadata); + // } let schema = Arc::new(updated_schema.into()); + 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( + MemTable::try_new(schema, batches)?.with_primary_keys(primary_keys), + ); self.register_table(&name, table)?; self.return_empty_dataframe() @@ -2107,26 +2117,6 @@ impl SerializerRegistry for EmptySerializerRegistry { } } -/// writes primary key indices as comma seperated integers to the -/// metadata. If columns a 0 and 2 are primary keys, -/// "primary_keys" -> "0, 2" will be added to metadata. -fn encode_primary_key_to_metadata( - metadata: &mut HashMap, - primary_keys: &[usize], -) { - let mut pks = String::new(); - // Store primary key information in the metadata during conversion - for (idx, pk) in primary_keys.iter().enumerate() { - pks = format!("{pks}{pk}"); - if idx < primary_keys.len() - 1 { - pks = format!("{pks}, "); - } - } - if !pks.is_empty() { - metadata.insert("primary_keys".to_string(), pks); - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/core/src/physical_plan/memory.rs b/datafusion/core/src/physical_plan/memory.rs index fa456d15eebfe..1c95cd6fa3133 100644 --- a/datafusion/core/src/physical_plan/memory.rs +++ b/datafusion/core/src/physical_plan/memory.rs @@ -220,7 +220,6 @@ impl Stream for MemoryStream { Some(columns) => batch.project(columns)?, None => batch.clone(), }; - Some(Ok(batch)) } else { None diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index d92fa8fefd53c..952a946d198b4 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1026,11 +1026,10 @@ impl DefaultPhysicalPlanner { ) .unzip(); // Update indices of the fields according to filter schema (the index seen in the filter schema). - let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices); - let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices); - // offset indices of right - let right_primary_keys = right_primary_keys.iter().map(|item| item + left_field_indices.len()); - let mut primary_keys = vec![]; + let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices, 0); + let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices, left_field_indices.len()); + + let mut primary_keys = HashMap::new(); primary_keys.extend(left_primary_keys); primary_keys.extend(right_primary_keys); @@ -1312,12 +1311,22 @@ impl DefaultPhysicalPlanner { /// return value will be `1`. This means that 1st index of the `field_indices`(5) is primary key /// e.g inside primary_keys vector ([4, 5, 7]). In the updated schema, fields at the indices [2, 5, 8] will /// be at [0, 1, 2]. -fn get_updated_primary_keys(df_schema: &DFSchema, field_indices: &[usize]) -> Vec { - let mut primary_keys = vec![]; +fn get_updated_primary_keys( + df_schema: &DFSchema, + field_indices: &[usize], + offset: usize, +) -> HashMap> { + let mut primary_keys = HashMap::new(); let existing_primary_keys = df_schema.primary_keys(); for (idx, field_idx) in field_indices.iter().enumerate() { - if existing_primary_keys.contains(field_idx) { - primary_keys.push(idx); + if let Some(associated_indices) = existing_primary_keys.get(field_idx) { + let mut new_associated_indices = vec![]; + for (idx, field_idx) in field_indices.iter().enumerate() { + if associated_indices.contains(field_idx) { + new_associated_indices.push(idx + offset); + } + } + primary_keys.insert(idx + offset, new_associated_indices); } } primary_keys @@ -2227,7 +2236,7 @@ mod tests { dict_id: 0, \ dict_is_ordered: false, \ metadata: {} } }\ - ], metadata: {}, primary_keys: [] }, \ + ], metadata: {}, primary_keys: {} }, \ ExecutionPlan schema: Schema { fields: [\ Field { \ name: \"b\", \ diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index de56b35849b78..3720a5613bc7f 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2910,3 +2910,59 @@ SELECT s.sn, s.amount, 2*s.sn 2 75 4 3 200 6 4 100 8 + +# Join should propagate primary key successfully +query TT +EXPLAIN SELECT r.sn, SUM(l.amount), r.amount + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r + ON l.sn >= r.sn + GROUP BY r.sn + ORDER BY r.sn +---- +logical_plan +Sort: r.sn ASC NULLS LAST +--Projection: r.sn, SUM(l.amount), r.amount +----Aggregate: groupBy=[[r.sn, r.amount]], aggr=[[SUM(l.amount)]] +------Projection: l.amount, r.sn, r.amount +--------Inner Join: Filter: l.sn >= r.sn +----------SubqueryAlias: l +------------TableScan: sales_global_with_pk projection=[sn, amount] +----------SubqueryAlias: r +------------TableScan: sales_global_with_pk projection=[sn, amount] +physical_plan +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 +----------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] +----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 +------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +------------------CoalescePartitionsExec +--------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] + +query IRR +SELECT r.sn, SUM(l.amount), r.amount + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r + ON l.sn >= r.sn + GROUP BY r.sn + ORDER BY r.sn +---- +0 455 30 +1 425 50 +2 375 75 +3 300 200 +4 100 100 + +# 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\) +SELECT l.sn, r.sn, SUM(l.amount), r.amount + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r + ON l.sn >= r.sn + GROUP BY l.sn + ORDER BY l.sn diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 54d2a932b3c13..c1ff3c6bcc4ad 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -263,19 +263,13 @@ impl LogicalPlanBuilder { } let schema = table_source.schema(); - - let primary_keys = if let Some(pks) = schema.metadata().get("primary_keys") { - pks.split(',') - .map(|s| { - s.trim() - .parse() - // Convert parsing error to Datafusion Error - .map_err(|e| DataFusionError::External(Box::new(e))) - }) - .collect::>>()? - } else { - vec![] - }; + let mut primary_keys = HashMap::new(); + let n_field = schema.fields.len(); + for pk in table_source.primary_keys() { + // At source primary key is associated with all the fields + let associations = (0..n_field).collect(); + primary_keys.insert(*pk, associations); + } let projected_schema = projection .as_ref() @@ -1105,39 +1099,44 @@ pub fn build_join_schema( let right_primary_keys = right .primary_keys() .iter() - .map(|idx| idx + left_fields.len()) - .collect::>(); - let left_primary_keys = left.primary_keys(); - let primary_keys: Vec = match join_type { + .map(|(idx, associated_indices)| { + ( + idx + left_fields.len(), + associated_indices + .iter() + .map(|item| item + left_fields.len()) + .collect::>(), + ) + }) + .collect::>>(); + let left_primary_keys = left.primary_keys().clone(); + let mut primary_keys = HashMap::new(); + match join_type { JoinType::Inner => { // left then right - left_primary_keys - .iter() - .chain(right_primary_keys.iter()) - .cloned() - .collect() + primary_keys.extend(left_primary_keys); + primary_keys.extend(right_primary_keys); } JoinType::Left => { // left then right, right set to nullable in case of not matched scenario - left_primary_keys.to_vec() + primary_keys.extend(left_primary_keys); } JoinType::Right => { // left then right, left set to nullable in case of not matched scenario - right_primary_keys.to_vec() + primary_keys.extend(right_primary_keys); } JoinType::Full => { - // left then right, all set to nullable in case of not matched scenario - vec![] + // do nothing } JoinType::LeftSemi | JoinType::LeftAnti => { // Only use the left side for the schema - left_primary_keys.to_vec() + primary_keys.extend(left_primary_keys); } JoinType::RightSemi | JoinType::RightAnti => { // Only use the right side for the schema - right_primary_keys.to_vec() + primary_keys.extend(right_primary_keys); } - }; + } let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); @@ -1295,7 +1294,7 @@ pub fn project( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), )? - .with_primary_keys(plan.schema().primary_keys().to_vec()); + .with_primary_keys(plan.schema().primary_keys().clone()); Ok(LogicalPlan::Projection(Projection::try_new_with_schema( projected_expr, @@ -1460,7 +1459,7 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new( DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? - .with_primary_keys(input_schema.primary_keys().to_vec()), + .with_primary_keys(input_schema.primary_keys().clone()), ); Ok(LogicalPlan::Unnest(Unnest { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 972e68ff4e11b..aebef0852defd 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1263,7 +1263,7 @@ impl Projection { exprlist_to_fields(&expr, &input)?, input.schema().metadata().clone(), )? - .with_primary_keys(input.schema().primary_keys().to_vec()), + .with_primary_keys(input.schema().primary_keys().clone()), ); Self::try_new_with_schema(expr, input, schema) } @@ -1327,7 +1327,7 @@ impl SubqueryAlias { ) -> Result { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); - let primary_keys = plan.schema().primary_keys().to_vec(); + let primary_keys = plan.schema().primary_keys().clone(); let schema = DFSchemaRef::new( DFSchema::try_from_qualified_schema(&alias, &schema)? .with_primary_keys(primary_keys), @@ -1413,7 +1413,7 @@ impl Window { window_fields .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); let metadata = input.schema().metadata().clone(); - let primary_keys = input.schema().primary_keys().to_vec(); + let primary_keys = input.schema().primary_keys().clone(); Ok(Window { input, window_expr, @@ -1611,7 +1611,7 @@ impl Aggregate { exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), )? - .with_primary_keys(input.schema().primary_keys().to_vec()); + .with_primary_keys(input.schema().primary_keys().clone()); Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 2f5a8923e8bb6..77fe161f5e05f 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -64,6 +64,12 @@ pub trait TableSource: Sync + Send { /// Get a reference to the schema for this table fn schema(&self) -> SchemaRef; + /// Get primary key index if any + fn primary_keys(&self) -> &[usize] { + // By default, return empty reference + &[] + } + /// Get the type of this table for metadata/catalog purposes. fn table_type(&self) -> TableType { TableType::Base diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index ac6b73364e5e4..71b32abaf719e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -448,7 +448,7 @@ pub fn expand_qualified_wildcard( } let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? - .with_primary_keys(schema.primary_keys().to_vec()); + .with_primary_keys(schema.primary_keys().clone()); let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, @@ -1010,7 +1010,7 @@ pub fn from_plan( let schema = Arc::new( DFSchema::new_with_metadata(fields, input.schema().metadata().clone())? - .with_primary_keys(input.schema().primary_keys().to_vec()), + .with_primary_keys(input.schema().primary_keys().clone()), ); Ok(LogicalPlan::Unnest(Unnest { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 4997bc47c9fae..4999b63f5f781 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -265,7 +265,7 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { DFSchemaRef::new( DFSchema::new_with_metadata(new_fields, schema.metadata().clone()) .unwrap() - .with_primary_keys(schema.primary_keys().to_vec()), + .with_primary_keys(schema.primary_keys().clone()), ) } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index fe159bf54b56f..771a6e64aa0e3 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -493,12 +493,12 @@ mod tests { assert_eq!( "get table_scan rule\ncaused by\n\ Internal error: Optimizer rule 'get table_scan rule' failed, due to generate a different schema, \ - original schema: DFSchema { fields: [], metadata: {}, primary_keys: [] }, \ + original schema: DFSchema { fields: [], metadata: {}, primary_keys: {} }, \ 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: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \ - metadata: {}, primary_keys: [] }. \ + metadata: {}, primary_keys: {} }. \ This was likely caused by a bug in DataFusion's code \ and we would welcome that you file an bug report in our issue tracker", err.to_string() diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index d741be4c9f9ba..96e800b9c3b43 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -237,13 +237,26 @@ impl OptimizerRule for SingleDistinctToGroupBy { } } -fn get_updated_primary_keys(old_schema: &DFSchema, new_fields: &[DFField]) -> Vec { +fn get_updated_primary_keys( + old_schema: &DFSchema, + new_fields: &[DFField], +) -> std::collections::HashMap> { let old_primary_keys = old_schema.primary_keys(); let fields = old_schema.fields(); - let mut primary_keys = vec![]; - for pk in old_primary_keys { - if let Some(idx) = new_fields.iter().position(|item| item == &fields[*pk]) { - primary_keys.push(idx); + let mut primary_keys = std::collections::HashMap::new(); + for (pk, associated_indices) in old_primary_keys { + if let Some(new_pk_idx) = new_fields.iter().position(|item| item == &fields[*pk]) + { + let mut new_associated_indices = vec![]; + for associated_idx in associated_indices { + if let Some(match_idx) = new_fields + .iter() + .position(|item| item == &fields[*associated_idx]) + { + new_associated_indices.push(match_idx); + } + } + primary_keys.insert(new_pk_idx, new_associated_indices); } } primary_keys diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 4b343c9b021ca..8b2d3bffe246f 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -428,37 +428,39 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: LogicalPlan, select_exprs: &[Expr], having_expr_opt: Option<&Expr>, - mut group_by_exprs: Vec, + group_by_exprs: Vec, aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { let schema = input.schema(); let primary_keys = schema.primary_keys(); - let input_fields = schema.fields(); - let input_primary_key_fields = primary_keys - .iter() - .map(|idx| &input_fields[*idx]) - .collect::>(); + let field_names = schema .fields() .iter() .map(|elem| elem.qualified_name()) .collect::>(); - - let group_by_contains_primary = group_by_exprs.iter().any(|expr| { - input_primary_key_fields.iter().any(|item| { - let expr_name = format!("{}", expr); - item.qualified_name() == expr_name - }) - }); - if group_by_contains_primary { - for expr in select_exprs { - if !group_by_exprs.contains(expr) - && field_names.contains(&format!("{}", expr)) - { - group_by_exprs.push(expr.clone()); + let mut new_group_by_exprs = group_by_exprs.clone(); + for (pk, associations) in primary_keys { + for group_by_expr in &group_by_exprs { + let expr_name = format!("{}", group_by_expr); + if field_names[*pk] == expr_name { + let associated_field_names = associations + .iter() + .map(|idx| field_names[*idx].clone()) + .collect::>(); + 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()); + } + } } } } + let group_by_exprs = new_group_by_exprs; + // create the aggregate plan let plan = LogicalPlanBuilder::from(input.clone()) .aggregate(group_by_exprs.clone(), aggr_exprs.clone())? From 4986bdad6e9e39f60b087a133545d9866a215021 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 4 Jul 2023 15:29:23 +0300 Subject: [PATCH 10/37] boilerplate for primary key propagation between executors --- datafusion/common/src/dfschema.rs | 39 +++++++++----- datafusion/common/src/lib.rs | 5 +- .../src/datasource/default_table_source.rs | 1 + datafusion/core/src/datasource/provider.rs | 2 +- datafusion/core/src/execution/context.rs | 13 ++--- datafusion/core/src/physical_plan/memory.rs | 1 + datafusion/core/src/physical_planner.rs | 45 ++++++++++++---- datafusion/expr/src/logical_plan/builder.rs | 52 +++++++------------ datafusion/expr/src/logical_plan/plan.rs | 13 +++-- datafusion/expr/src/utils.rs | 12 ++++- .../optimizer/src/analyzer/type_coercion.rs | 1 - .../src/single_distinct_to_groupby.rs | 48 +++-------------- 12 files changed, 115 insertions(+), 117 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 509fb4397b567..da0e6ed29cdf0 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,6 +33,8 @@ use std::fmt::{Display, Formatter}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; +type PrimaryKeyToAssociations = HashMap>; + /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] pub struct DFSchema { @@ -41,7 +43,7 @@ pub struct DFSchema { /// Additional metadata in form of key value pairs metadata: HashMap, /// primary key index and its associated fields indices - primary_keys: HashMap>, + primary_keys: PrimaryKeyToAssociations, } impl DFSchema { @@ -123,6 +125,7 @@ impl DFSchema { ) } + /// Assing primary key pub fn with_primary_keys(mut self, primary_keys: HashMap>) -> Self { self.primary_keys = primary_keys; self @@ -136,19 +139,8 @@ impl DFSchema { fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); let mut primary_keys = self.primary_keys.clone(); - let other_primary_keys = schema - .primary_keys - .iter() - .map(|(idx, associations)| { - ( - idx + self.fields.len(), - associations - .iter() - .map(|item| item + self.fields.len()) - .collect::>(), - ) - }) - .collect::>>(); + let other_primary_keys = + add_offset_to_primary_key(schema.primary_keys(), self.fields.len()); primary_keys.extend(other_primary_keys); Ok(Self::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) } @@ -789,6 +781,25 @@ impl SchemaExt for Schema { } } +/// Add offset value to primary_keys and its associated indices +pub fn add_offset_to_primary_key( + primary_keys: &PrimaryKeyToAssociations, + offset: usize, +) -> PrimaryKeyToAssociations { + primary_keys + .iter() + .map(|(idx, associated_indices)| { + ( + idx + offset, + associated_indices + .iter() + .map(|item| item + offset) + .collect::>(), + ) + }) + .collect::>>() +} + #[cfg(test)] mod tests { use crate::assert_contains; diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 63b4024579f1e..1b0918ca44620 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -36,7 +36,10 @@ pub mod tree_node; pub mod utils; pub use column::Column; -pub use dfschema::{DFField, DFSchema, DFSchemaRef, ExprSchema, SchemaExt, ToDFSchema}; +pub use dfschema::{ + add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, SchemaExt, + ToDFSchema, +}; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, SharedResult, diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 6e40f4aa78453..36b49dee94de6 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -52,6 +52,7 @@ impl TableSource for DefaultTableSource { self.table_provider.schema() } + /// Get a reference to the primary key indices fn primary_keys(&self) -> &[usize] { self.table_provider.primary_keys() } diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 3483b6f9ba343..f727b55870cd8 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -43,7 +43,7 @@ pub trait TableProvider: Sync + Send { /// Get a reference to the primary_key indices fn primary_keys(&self) -> &[usize] { - // By default return None + // By default return empty reference &[] } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index fc7392005913d..d2f4a9c28e3c7 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -514,22 +514,17 @@ impl SessionContext { .collect::>>()?; let fields = df_schema.fields().to_vec(); + + // all of the primary keys are associated with all of the fields (since it is source). + let association_indices = (0..fields.len()).collect::>(); let primary_keys_with_associations = primary_keys .iter() - .map(|idx| (*idx, (0..fields.len()).collect::>())) + .map(|idx| (*idx, association_indices.clone())) .collect(); let updated_schema = DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? .with_primary_keys(primary_keys_with_associations); - // let mut schema: Schema = updated_schema.into(); - // if !primary_keys.is_empty() { - // // Store primary key information in schema also. By this way we can keep track of primary key - // // after DFSchema is converted to arrow::Schema - // let mut metadata = schema.metadata().clone(); - // encode_primary_key_to_metadata(&mut metadata, &primary_keys); - // schema = Schema::new_with_metadata(schema.fields, metadata); - // } let schema = Arc::new(updated_schema.into()); let physical = DataFrame::new(self.state(), input); diff --git a/datafusion/core/src/physical_plan/memory.rs b/datafusion/core/src/physical_plan/memory.rs index 1c95cd6fa3133..fa456d15eebfe 100644 --- a/datafusion/core/src/physical_plan/memory.rs +++ b/datafusion/core/src/physical_plan/memory.rs @@ -220,6 +220,7 @@ impl Stream for MemoryStream { Some(columns) => batch.project(columns)?, None => batch.clone(), }; + Some(Ok(batch)) } else { None diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 952a946d198b4..5331b43bf4581 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -62,7 +62,7 @@ use crate::{ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; -use datafusion_common::{DFSchema, ScalarValue}; +use datafusion_common::{add_offset_to_primary_key, DFSchema, ScalarValue}; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, WindowFunction, @@ -1026,8 +1026,11 @@ impl DefaultPhysicalPlanner { ) .unzip(); // Update indices of the fields according to filter schema (the index seen in the filter schema). - let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices, 0); - let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices, left_field_indices.len()); + let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices); + let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices); + + // after join, indices from the right table will increase with the size of the left table. hence add offset + let right_primary_keys = add_offset_to_primary_key(&right_primary_keys, left_field_indices.len()); let mut primary_keys = HashMap::new(); primary_keys.extend(left_primary_keys); @@ -1307,14 +1310,13 @@ impl DefaultPhysicalPlanner { } /// Update primary key indices, with index of the number in `field_indices` -/// If `field_indices` is [2, 5, 8], primary keys are [4, 5, 7] in the `df_schema`. -/// return value will be `1`. This means that 1st index of the `field_indices`(5) is primary key -/// e.g inside primary_keys vector ([4, 5, 7]). In the updated schema, fields at the indices [2, 5, 8] will -/// be at [0, 1, 2]. +/// If `field_indices` is [2, 5, 8], primary keys are 5 -> [5, 8] in the `df_schema`. +/// return value will be 1 -> [1, 2]. This means that 1st index of the `field_indices`(5) is primary key, +/// and this primary key is associated with columns at indices 1 and 2 (in the updated schema). +/// In the updated schema, fields at the indices [2, 5, 8] will be at [0, 1, 2]. fn get_updated_primary_keys( df_schema: &DFSchema, field_indices: &[usize], - offset: usize, ) -> HashMap> { let mut primary_keys = HashMap::new(); let existing_primary_keys = df_schema.primary_keys(); @@ -1323,10 +1325,10 @@ fn get_updated_primary_keys( let mut new_associated_indices = vec![]; for (idx, field_idx) in field_indices.iter().enumerate() { if associated_indices.contains(field_idx) { - new_associated_indices.push(idx + offset); + new_associated_indices.push(idx); } } - primary_keys.insert(idx + offset, new_associated_indices); + primary_keys.insert(idx, new_associated_indices); } } primary_keys @@ -2460,6 +2462,29 @@ mod tests { ); } } + + #[tokio::test] + async fn test_get_updated_primary_keys() { + let mut schema = get_test_dfschema().unwrap(); + + let mut primary_keys = HashMap::new(); + primary_keys.insert(1, vec![0, 1, 2]); + schema = schema.with_primary_keys(primary_keys); + let res = get_updated_primary_keys(&schema, &[1, 2]); + let mut expected = HashMap::new(); + expected.insert(0, vec![0, 1]); + assert_eq!(res, expected); + } + + fn get_test_dfschema() -> Result { + let fields = vec![ + DFField::new_unqualified("a", DataType::Int64, false), + DFField::new_unqualified("b", DataType::Int64, false), + DFField::new_unqualified("c", DataType::Int64, false), + ]; + DFSchema::new_with_metadata(fields, HashMap::new()) + } + struct ErrorExtensionPlanner {} #[async_trait] diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index c1ff3c6bcc4ad..fefb95d3fbc23 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -24,7 +24,9 @@ use crate::expr_rewriter::{ rewrite_sort_cols_by_aggs, }; use crate::type_coercion::binary::comparison_coercion; -use crate::utils::{columnize_expr, compare_sort_expr, exprlist_to_fields}; +use crate::utils::{ + columnize_expr, compare_sort_expr, exprlist_to_fields, exprlist_to_primary_keys, +}; use crate::{and, binary_expr, DmlStatement, Operator, WriteOp}; use crate::{ logical_plan::{ @@ -41,8 +43,9 @@ use crate::{ }; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ - display::ToStringifiedPlan, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - OwnedTableReference, Result, ScalarValue, TableReference, ToDFSchema, + add_offset_to_primary_key, display::ToStringifiedPlan, Column, DFField, DFSchema, + DFSchemaRef, DataFusionError, OwnedTableReference, Result, ScalarValue, + TableReference, ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -265,10 +268,10 @@ impl LogicalPlanBuilder { let schema = table_source.schema(); let mut primary_keys = HashMap::new(); let n_field = schema.fields.len(); + // All the field indices are associated, since it is source, + let associated_indices = (0..n_field).collect::>(); for pk in table_source.primary_keys() { - // At source primary key is associated with all the fields - let associations = (0..n_field).collect(); - primary_keys.insert(*pk, associations); + primary_keys.insert(*pk, associated_indices.clone()); } let projected_schema = projection @@ -1096,19 +1099,9 @@ pub fn build_join_schema( } }; - let right_primary_keys = right - .primary_keys() - .iter() - .map(|(idx, associated_indices)| { - ( - idx + left_fields.len(), - associated_indices - .iter() - .map(|item| item + left_fields.len()) - .collect::>(), - ) - }) - .collect::>>(); + let right_primary_keys = + add_offset_to_primary_key(right.primary_keys(), left_fields.len()); + let left_primary_keys = left.primary_keys().clone(); let mut primary_keys = HashMap::new(); match join_type { @@ -1117,25 +1110,17 @@ pub fn build_join_schema( primary_keys.extend(left_primary_keys); primary_keys.extend(right_primary_keys); } - JoinType::Left => { - // left then right, right set to nullable in case of not matched scenario - primary_keys.extend(left_primary_keys); - } - JoinType::Right => { - // left then right, left set to nullable in case of not matched scenario - primary_keys.extend(right_primary_keys); - } - JoinType::Full => { - // do nothing - } - JoinType::LeftSemi | JoinType::LeftAnti => { + JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { // Only use the left side for the schema primary_keys.extend(left_primary_keys); } - JoinType::RightSemi | JoinType::RightAnti => { + JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { // Only use the right side for the schema primary_keys.extend(right_primary_keys); } + JoinType::Full => { + // primary key is not preserved + } } let mut metadata = left.metadata().clone(); @@ -1290,11 +1275,12 @@ pub fn project( } } validate_unique_names("Projections", projected_expr.iter())?; + let primary_keys = exprlist_to_primary_keys(&projected_expr, &plan)?; let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), )? - .with_primary_keys(plan.schema().primary_keys().clone()); + .with_primary_keys(primary_keys); Ok(LogicalPlan::Projection(Projection::try_new_with_schema( projected_expr, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index aebef0852defd..cac9a43ada192 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -24,8 +24,9 @@ use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; use crate::utils::{ - enumerate_grouping_sets, exprlist_to_fields, find_out_reference_exprs, from_plan, - grouping_set_expr_count, grouping_set_to_exprlist, inspect_expr_pre, + enumerate_grouping_sets, exprlist_to_fields, exprlist_to_primary_keys, + find_out_reference_exprs, from_plan, grouping_set_expr_count, + grouping_set_to_exprlist, inspect_expr_pre, }; use crate::{ build_join_schema, Expr, ExprSchemable, TableProviderFilterPushDown, TableSource, @@ -1258,12 +1259,13 @@ pub struct Projection { impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { + let primary_keys = exprlist_to_primary_keys(&expr, &input)?; let schema = Arc::new( DFSchema::new_with_metadata( exprlist_to_fields(&expr, &input)?, input.schema().metadata().clone(), )? - .with_primary_keys(input.schema().primary_keys().clone()), + .with_primary_keys(primary_keys), ); Self::try_new_with_schema(expr, input, schema) } @@ -1413,7 +1415,7 @@ impl Window { window_fields .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); let metadata = input.schema().metadata().clone(); - let primary_keys = input.schema().primary_keys().clone(); + let primary_keys = exprlist_to_primary_keys(window_expr.iter(), &input)?; Ok(Window { input, window_expr, @@ -1607,11 +1609,12 @@ impl Aggregate { let group_expr = enumerate_grouping_sets(group_expr)?; let grouping_expr: Vec = grouping_set_to_exprlist(group_expr.as_slice())?; let all_expr = grouping_expr.iter().chain(aggr_expr.iter()); + let primary_keys = exprlist_to_primary_keys(all_expr.clone(), &input)?; let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), )? - .with_primary_keys(input.schema().primary_keys().clone()); + .with_primary_keys(primary_keys); Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 71b32abaf719e..4649f10dcd3a6 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -39,7 +39,7 @@ use datafusion_common::{ }; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; use std::cmp::Ordering; -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; /// The value to which `COUNT(*)` is expanded to in @@ -1081,6 +1081,16 @@ pub fn exprlist_to_fields<'a>( } } +/// Calculate updated primary key for the plan, and expression +pub fn exprlist_to_primary_keys<'a>( + _expr: impl IntoIterator, + _plan: &LogicalPlan, +) -> Result>> { + // TODO: Add handling for primary key propagation here + // if necessary similar to `exprlist_to_fields` implementation + Ok(HashMap::new()) +} + /// Convert an expression into Column expression if it's already provided as input plan. /// /// For example, it rewrites: diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index ba974701aaf7f..8edf734b474f9 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -85,7 +85,6 @@ fn analyze_internal( let mut schema = merge_schema(new_inputs.iter().collect()); if let LogicalPlan::TableScan(ts) = plan { - // arrow schema doesn't support primary_key. We do not know primary_key of ts.source if any. let source_schema = DFSchema::try_from_qualified_schema(&ts.table_name, &ts.source.schema())?; schema.merge(&source_schema); diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 96e800b9c3b43..a9e65b3e7c778 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -19,7 +19,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{DFField, DFSchema, Result}; +use datafusion_common::{DFSchema, Result}; use datafusion_expr::{ col, expr::AggregateFunction, @@ -157,15 +157,10 @@ impl OptimizerRule for SingleDistinctToGroupBy { .iter() .map(|expr| expr.to_field(input.schema())) .collect::>>()?; - - let primary_keys = - get_updated_primary_keys(input.schema(), &inner_fields); - let inner_schema = DFSchema::new_with_metadata( inner_fields, input.schema().metadata().clone(), - )? - .with_primary_keys(primary_keys); + )?; let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), inner_group_exprs, @@ -177,16 +172,10 @@ impl OptimizerRule for SingleDistinctToGroupBy { .chain(new_aggr_exprs.iter()) .map(|expr| expr.to_field(&inner_schema)) .collect::>>()?; - - let primary_keys = - get_updated_primary_keys(&inner_schema, &outer_fields); - let outer_aggr_schema = Arc::new( - DFSchema::new_with_metadata( - outer_fields, - input.schema().metadata().clone(), - )? - .with_primary_keys(primary_keys), - ); + 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: @@ -237,31 +226,6 @@ impl OptimizerRule for SingleDistinctToGroupBy { } } -fn get_updated_primary_keys( - old_schema: &DFSchema, - new_fields: &[DFField], -) -> std::collections::HashMap> { - let old_primary_keys = old_schema.primary_keys(); - let fields = old_schema.fields(); - let mut primary_keys = std::collections::HashMap::new(); - for (pk, associated_indices) in old_primary_keys { - if let Some(new_pk_idx) = new_fields.iter().position(|item| item == &fields[*pk]) - { - let mut new_associated_indices = vec![]; - for associated_idx in associated_indices { - if let Some(match_idx) = new_fields - .iter() - .position(|item| item == &fields[*associated_idx]) - { - new_associated_indices.push(match_idx); - } - } - primary_keys.insert(new_pk_idx, new_associated_indices); - } - } - primary_keys -} - #[cfg(test)] mod tests { use super::*; From 8fe9852c5bddcc98a12abaa1ca175a788478d8da Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 5 Jul 2023 17:30:23 +0300 Subject: [PATCH 11/37] Add new tests --- .../sqllogictests/test_files/groupby.slt | 45 +++++++++++++++++++ datafusion/expr/src/utils.rs | 21 +++++++-- 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 3720a5613bc7f..1728b7f8f7bb1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2966,3 +2966,48 @@ SELECT l.sn, r.sn, SUM(l.amount), r.amount ON l.sn >= r.sn GROUP BY l.sn ORDER BY l.sn + +# window should propagate primary key successfully +query TT +EXPLAIN SELECT * + FROM(SELECT *, SUM(l.amount) OVER(ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum_amount + FROM sales_global_with_pk AS l + ) as l + GROUP BY l.sn + ORDER BY l.sn +---- +logical_plan +Sort: l.sn ASC NULLS LAST +--Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, l.sum_amount +----Aggregate: groupBy=[[l.sn, l.zip_code, l.country, l.ts, l.currency, l.amount, l.sum_amount]], aggr=[[]] +------SubqueryAlias: l +--------Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum_amount +----------WindowAggr: windowExpr=[[SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +------------SubqueryAlias: l +--------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] +physical_plan +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 +----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=1 +------------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=[] +--------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] +----------------BoundedWindowAggExec: wdw=[SUM(l.amount): Ok(Field { name: "SUM(l.amount)", data_type: Float64, 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] +------------------CoalescePartitionsExec +--------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] + +query ITIPTRR +SELECT * + FROM(SELECT *, SUM(l.amount) OVER(ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum_amount + FROM sales_global_with_pk AS l + ) as l + GROUP BY l.sn + ORDER BY l.sn +---- +0 GRC 0 2022-01-01T06:00:00 EUR 30 80 +1 FRA 1 2022-01-01T08:00:00 EUR 50 155 +1 TUR 2 2022-01-01T11:30:00 TRY 75 325 +1 FRA 3 2022-01-02T12:00:00 EUR 200 375 +1 TUR 4 2022-01-03T10:00:00 TRY 100 300 diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 4649f10dcd3a6..1a0f79c76b1bb 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1083,12 +1083,27 @@ pub fn exprlist_to_fields<'a>( /// Calculate updated primary key for the plan, and expression pub fn exprlist_to_primary_keys<'a>( - _expr: impl IntoIterator, - _plan: &LogicalPlan, + expr: impl IntoIterator, + plan: &LogicalPlan, ) -> Result>> { // TODO: Add handling for primary key propagation here // if necessary similar to `exprlist_to_fields` implementation - Ok(HashMap::new()) + let exprs: Vec = expr.into_iter().cloned().collect(); + match plan { + // Below operators associate existing primary key with all the fields + LogicalPlan::Window(_) + | LogicalPlan::SubqueryAlias(_) + | LogicalPlan::Filter(_) + | LogicalPlan::Limit(_) + | LogicalPlan::Repartition(_) => { + let mut primary_keys = plan.schema().primary_keys().clone(); + for (_pk, associations) in primary_keys.iter_mut() { + *associations = (0..exprs.len()).collect(); + } + Ok(primary_keys) + } + _ => Ok(HashMap::new()), + } } /// Convert an expression into Column expression if it's already provided as input plan. From da1b947843112be0450fbbdc3b268908f55be884 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 5 Jul 2023 18:26:30 +0300 Subject: [PATCH 12/37] Add Projection primary key handling --- datafusion/common/src/dfschema.rs | 21 +++-- datafusion/common/src/lib.rs | 4 +- datafusion/core/src/execution/context.rs | 2 +- datafusion/core/src/physical_planner.rs | 16 ++-- .../sqllogictests/test_files/groupby.slt | 48 ++++++++++ datafusion/expr/src/logical_plan/builder.rs | 88 +++++++++++++++++-- datafusion/expr/src/logical_plan/plan.rs | 28 ++++-- datafusion/expr/src/utils.rs | 27 +----- datafusion/sql/src/select.rs | 3 +- 9 files changed, 176 insertions(+), 61 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index da0e6ed29cdf0..7cefeeb21bcc3 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,7 +33,7 @@ use std::fmt::{Display, Formatter}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -type PrimaryKeyToAssociations = HashMap>; +pub type PrimaryKeyToAssociations = HashMap)>; /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] @@ -126,7 +126,7 @@ impl DFSchema { } /// Assing primary key - pub fn with_primary_keys(mut self, primary_keys: HashMap>) -> Self { + pub fn with_primary_keys(mut self, primary_keys: PrimaryKeyToAssociations) -> Self { self.primary_keys = primary_keys; self } @@ -496,7 +496,7 @@ impl DFSchema { } /// Get primary keys - pub fn primary_keys(&self) -> &HashMap> { + pub fn primary_keys(&self) -> &PrimaryKeyToAssociations { &self.primary_keys } } @@ -788,16 +788,19 @@ pub fn add_offset_to_primary_key( ) -> PrimaryKeyToAssociations { primary_keys .iter() - .map(|(idx, associated_indices)| { + .map(|(idx, (is_unique, associated_indices))| { ( idx + offset, - associated_indices - .iter() - .map(|item| item + offset) - .collect::>(), + ( + *is_unique, + associated_indices + .iter() + .map(|item| item + offset) + .collect::>(), + ), ) }) - .collect::>>() + .collect::() } #[cfg(test)] diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 1b0918ca44620..e52bba059b958 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -37,8 +37,8 @@ pub mod utils; pub use column::Column; pub use dfschema::{ - add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, SchemaExt, - ToDFSchema, + add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, + PrimaryKeyToAssociations, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index d2f4a9c28e3c7..f490d9848fa1b 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -519,7 +519,7 @@ impl SessionContext { let association_indices = (0..fields.len()).collect::>(); let primary_keys_with_associations = primary_keys .iter() - .map(|idx| (*idx, association_indices.clone())) + .map(|idx| (*idx, (true, association_indices.clone()))) .collect(); let updated_schema = DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5331b43bf4581..818866d53853c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -62,7 +62,9 @@ use crate::{ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; -use datafusion_common::{add_offset_to_primary_key, DFSchema, ScalarValue}; +use datafusion_common::{ + add_offset_to_primary_key, DFSchema, PrimaryKeyToAssociations, ScalarValue, +}; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, WindowFunction, @@ -1317,18 +1319,20 @@ impl DefaultPhysicalPlanner { fn get_updated_primary_keys( df_schema: &DFSchema, field_indices: &[usize], -) -> HashMap> { +) -> PrimaryKeyToAssociations { let mut primary_keys = HashMap::new(); let existing_primary_keys = df_schema.primary_keys(); for (idx, field_idx) in field_indices.iter().enumerate() { - if let Some(associated_indices) = existing_primary_keys.get(field_idx) { + if let Some((is_unique, associated_indices)) = + existing_primary_keys.get(field_idx) + { let mut new_associated_indices = vec![]; for (idx, field_idx) in field_indices.iter().enumerate() { if associated_indices.contains(field_idx) { new_associated_indices.push(idx); } } - primary_keys.insert(idx, new_associated_indices); + primary_keys.insert(idx, (*is_unique, new_associated_indices)); } } primary_keys @@ -2468,11 +2472,11 @@ mod tests { let mut schema = get_test_dfschema().unwrap(); let mut primary_keys = HashMap::new(); - primary_keys.insert(1, vec![0, 1, 2]); + primary_keys.insert(1, (true, vec![0, 1, 2])); schema = schema.with_primary_keys(primary_keys); let res = get_updated_primary_keys(&schema, &[1, 2]); let mut expected = HashMap::new(); - expected.insert(0, vec![0, 1]); + expected.insert(0, (true, vec![0, 1])); assert_eq!(res, expected); } diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 1728b7f8f7bb1..eea8a92aada28 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -3011,3 +3011,51 @@ SELECT * 1 TUR 2 2022-01-01T11:30:00 TRY 75 325 1 FRA 3 2022-01-02T12:00:00 EUR 200 375 1 TUR 4 2022-01-03T10:00:00 TRY 100 300 + +# join should propagate primary key correctly +query IRP +SELECT l.sn, SUM(l.amount), l.ts +FROM + (SELECT * + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r ON l.sn >= r.sn) +GROUP BY l.sn +ORDER BY l.sn +---- +0 30 2022-01-01T06:00:00 +1 100 2022-01-01T08:00:00 +2 225 2022-01-01T11:30:00 +3 800 2022-01-02T12:00:00 +4 500 2022-01-03T10:00:00 + +# Projection propagates primary keys correctly +# (we can use r.ts at the final projection, because it +# is associated with primary key r.sn) +query IRP +SELECT r.sn, SUM(r.amount), r.ts +FROM + (SELECT r.ts, r.sn, r.amount + FROM + (SELECT * + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r ON l.sn >= r.sn)) +GROUP BY r.sn +ORDER BY r.sn +---- +0 150 2022-01-01T06:00:00 +1 200 2022-01-01T08:00:00 +2 225 2022-01-01T11:30:00 +3 400 2022-01-02T12:00:00 +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\) +SELECT r.sn, SUM(r.amount), rn1 +FROM + (SELECT r.ts, r.sn, r.amount, + ROW_NUMBER() OVER() AS rn1 + FROM + (SELECT * + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r ON l.sn >= r.sn)) +GROUP BY r.sn diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index fefb95d3fbc23..df171aae3e6ea 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -24,9 +24,7 @@ use crate::expr_rewriter::{ rewrite_sort_cols_by_aggs, }; use crate::type_coercion::binary::comparison_coercion; -use crate::utils::{ - columnize_expr, compare_sort_expr, exprlist_to_fields, exprlist_to_primary_keys, -}; +use crate::utils::{columnize_expr, compare_sort_expr, exprlist_to_fields}; use crate::{and, binary_expr, DmlStatement, Operator, WriteOp}; use crate::{ logical_plan::{ @@ -44,8 +42,8 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ add_offset_to_primary_key, display::ToStringifiedPlan, Column, DFField, DFSchema, - DFSchemaRef, DataFusionError, OwnedTableReference, Result, ScalarValue, - TableReference, ToDFSchema, + DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeyToAssociations, Result, + ScalarValue, TableReference, ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -271,7 +269,7 @@ impl LogicalPlanBuilder { // All the field indices are associated, since it is source, let associated_indices = (0..n_field).collect::>(); for pk in table_source.primary_keys() { - primary_keys.insert(*pk, associated_indices.clone()); + primary_keys.insert(*pk, (true, associated_indices.clone())); } let projected_schema = projection @@ -1099,10 +1097,19 @@ pub fn build_join_schema( } }; - let right_primary_keys = + // TODO: Update is_unique flag + let mut right_primary_keys = add_offset_to_primary_key(right.primary_keys(), left_fields.len()); - let left_primary_keys = left.primary_keys().clone(); + let mut left_primary_keys = left.primary_keys().clone(); + + // After join, primary key may be cloned. reset is_unique flag. + left_primary_keys + .iter_mut() + .for_each(|(_pk, (is_unique, _))| *is_unique = false); + right_primary_keys + .iter_mut() + .for_each(|(_pk, (is_unique, _))| *is_unique = false); let mut primary_keys = HashMap::new(); match join_type { JoinType::Inner => { @@ -1250,6 +1257,56 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result { + let mut updated_primary_key = HashMap::new(); + let mut input_to_proj_mapping: HashMap = HashMap::new(); + + let input_schema = &input.schema(); + // println!("exprs: {:?}", exprs); + // println!("plan schema: {:?}", input_schema); + // println!("existing primary keys: {:?}", input_schema.primary_keys()); + // look for exact match in plan's output schema + let input_fields = input_schema.fields(); + for (idx, expr) in exprs.iter().enumerate() { + // println!("{}", format!("{:?}", expr)); + // println!("{}", format!("{}", expr)); + let expr_name = format!("{}", expr); + if let Some(match_idx) = input_fields + .iter() + .position(|item| item.qualified_name() == expr_name) + { + input_to_proj_mapping.insert(match_idx, idx); + } + // expr.to_field(input_schema) + } + for (idx, (is_unique, associations)) in input_schema.primary_keys() { + if let Some(target) = input_to_proj_mapping.get(idx) { + // Do mapping for associations + let updated_associations = if *is_unique { + (0..exprs.len()).collect() + } else { + // println!("not unique, associations: {:?}", associations); + let mut new_associations = vec![]; + for assoc_idx in associations { + if let Some(target_assoc_idx) = input_to_proj_mapping.get(assoc_idx) { + new_associations.push(*target_assoc_idx); + } + } + new_associations + }; + if !updated_associations.is_empty() { + updated_primary_key.insert(*target, (*is_unique, updated_associations)); + } + } + } + // println!("input_to_proj_mapping:{:?}", input_to_proj_mapping); + // println!("updated_primary_key:{:?}", updated_primary_key); + Ok(updated_primary_key) +} + /// Create Projection /// # Errors /// This function errors under any of the following conditions: @@ -1275,7 +1332,20 @@ pub fn project( } } validate_unique_names("Projections", projected_expr.iter())?; - let primary_keys = exprlist_to_primary_keys(&projected_expr, &plan)?; + + // println!("projected exprs"); + // for item in &projected_expr { + // println!("item:{:?}", item); + // } + // println!("projected exprs"); + + let primary_keys = project_primary_keys(&projected_expr, &plan)?; + // let primary_keys = exprlist_to_primary_keys(&projected_expr, &plan)?; + + // println!("---------------PROJECTION--------------"); + // print_primary_key(&plan); + // println!("projection primary_keys: {:?}", primary_keys); + // println!("---------------PROJECTION--------------"); let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index cac9a43ada192..fcdb9d60e9c72 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -24,9 +24,8 @@ use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; use crate::utils::{ - enumerate_grouping_sets, exprlist_to_fields, exprlist_to_primary_keys, - find_out_reference_exprs, from_plan, grouping_set_expr_count, - grouping_set_to_exprlist, inspect_expr_pre, + enumerate_grouping_sets, exprlist_to_fields, find_out_reference_exprs, from_plan, + grouping_set_expr_count, grouping_set_to_exprlist, inspect_expr_pre, }; use crate::{ build_join_schema, Expr, ExprSchemable, TableProviderFilterPushDown, TableSource, @@ -45,6 +44,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; // backwards compatibility +use crate::builder::project_primary_keys; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -1259,7 +1259,8 @@ pub struct Projection { impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { - let primary_keys = exprlist_to_primary_keys(&expr, &input)?; + // let primary_keys = exprlist_to_primary_keys(&expr, &input)?; + let primary_keys = project_primary_keys(&expr, &input)?; let schema = Arc::new( DFSchema::new_with_metadata( exprlist_to_fields(&expr, &input)?, @@ -1415,7 +1416,20 @@ impl Window { window_fields .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); let metadata = input.schema().metadata().clone(); - let primary_keys = exprlist_to_primary_keys(window_expr.iter(), &input)?; + + // Update primary key for window + let mut primary_keys = input.schema().primary_keys().clone(); + let n_input_fields = input.schema().fields().len(); + let n_window_expr = window_expr.len(); + let new_associated_fields: Vec = + (n_input_fields..n_input_fields + n_window_expr).collect(); + for (_pk, (is_unique, associations)) in primary_keys.iter_mut() { + // if unique extend associations with new fields + if *is_unique { + associations.extend(&new_associated_fields); + } + } + // let primary_keys = exprlist_to_primary_keys(window_expr.iter(), &input)?; Ok(Window { input, window_expr, @@ -1609,7 +1623,9 @@ impl Aggregate { let group_expr = enumerate_grouping_sets(group_expr)?; let grouping_expr: Vec = grouping_set_to_exprlist(group_expr.as_slice())?; let all_expr = grouping_expr.iter().chain(aggr_expr.iter()); - let primary_keys = exprlist_to_primary_keys(all_expr.clone(), &input)?; + // let primary_keys = exprlist_to_primary_keys(all_expr.clone(), &input)?; + // TODO: Add handling for aggregate primary key + let primary_keys = HashMap::new(); let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 1a0f79c76b1bb..71b32abaf719e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -39,7 +39,7 @@ use datafusion_common::{ }; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; use std::cmp::Ordering; -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::sync::Arc; /// The value to which `COUNT(*)` is expanded to in @@ -1081,31 +1081,6 @@ pub fn exprlist_to_fields<'a>( } } -/// Calculate updated primary key for the plan, and expression -pub fn exprlist_to_primary_keys<'a>( - expr: impl IntoIterator, - plan: &LogicalPlan, -) -> Result>> { - // TODO: Add handling for primary key propagation here - // if necessary similar to `exprlist_to_fields` implementation - let exprs: Vec = expr.into_iter().cloned().collect(); - match plan { - // Below operators associate existing primary key with all the fields - LogicalPlan::Window(_) - | LogicalPlan::SubqueryAlias(_) - | LogicalPlan::Filter(_) - | LogicalPlan::Limit(_) - | LogicalPlan::Repartition(_) => { - let mut primary_keys = plan.schema().primary_keys().clone(); - for (_pk, associations) in primary_keys.iter_mut() { - *associations = (0..exprs.len()).collect(); - } - Ok(primary_keys) - } - _ => Ok(HashMap::new()), - } -} - /// Convert an expression into Column expression if it's already provided as input plan. /// /// For example, it rewrites: diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 8b2d3bffe246f..172e9b9cfabd0 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -440,7 +440,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|elem| elem.qualified_name()) .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); - for (pk, associations) in primary_keys { + for (pk, (_is_unique, associations)) in primary_keys { for group_by_expr in &group_by_exprs { let expr_name = format!("{}", group_by_expr); if field_names[*pk] == expr_name { @@ -465,7 +465,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let plan = LogicalPlanBuilder::from(input.clone()) .aggregate(group_by_exprs.clone(), aggr_exprs.clone())? .build()?; - // 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 From d6cf41881692f28ea790796d06130380a91b1ed2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 6 Jul 2023 14:16:17 +0300 Subject: [PATCH 13/37] Keep primary key as vec --- datafusion/common/src/dfschema.rs | 20 +++++------ datafusion/common/src/lib.rs | 2 +- datafusion/core/src/execution/context.rs | 7 ++-- datafusion/core/src/physical_planner.rs | 38 +++++++++++++-------- datafusion/expr/src/logical_plan/builder.rs | 31 +++++++++++------ datafusion/sql/src/select.rs | 8 +++-- 6 files changed, 65 insertions(+), 41 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 7cefeeb21bcc3..e37e67c42e392 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,7 +33,7 @@ use std::fmt::{Display, Formatter}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -pub type PrimaryKeyToAssociations = HashMap)>; +pub type PrimaryKeysToAssociations = HashMap, (bool, Vec)>; /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] @@ -43,7 +43,7 @@ pub struct DFSchema { /// Additional metadata in form of key value pairs metadata: HashMap, /// primary key index and its associated fields indices - primary_keys: PrimaryKeyToAssociations, + primary_keys: PrimaryKeysToAssociations, } impl DFSchema { @@ -126,7 +126,7 @@ impl DFSchema { } /// Assing primary key - pub fn with_primary_keys(mut self, primary_keys: PrimaryKeyToAssociations) -> Self { + pub fn with_primary_keys(mut self, primary_keys: PrimaryKeysToAssociations) -> Self { self.primary_keys = primary_keys; self } @@ -496,7 +496,7 @@ impl DFSchema { } /// Get primary keys - pub fn primary_keys(&self) -> &PrimaryKeyToAssociations { + pub fn primary_keys(&self) -> &PrimaryKeysToAssociations { &self.primary_keys } } @@ -783,24 +783,24 @@ impl SchemaExt for Schema { /// Add offset value to primary_keys and its associated indices pub fn add_offset_to_primary_key( - primary_keys: &PrimaryKeyToAssociations, + primary_keys: &PrimaryKeysToAssociations, offset: usize, -) -> PrimaryKeyToAssociations { +) -> PrimaryKeysToAssociations { primary_keys .iter() - .map(|(idx, (is_unique, associated_indices))| { + .map(|(pk_indices, (is_unique, associated_indices))| { ( - idx + offset, + pk_indices.iter().map(|pk_idx| pk_idx + offset).collect(), ( *is_unique, associated_indices .iter() .map(|item| item + offset) - .collect::>(), + .collect(), ), ) }) - .collect::() + .collect::() } #[cfg(test)] diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index e52bba059b958..2707a54722a90 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -38,7 +38,7 @@ pub mod utils; pub use column::Column; pub use dfschema::{ add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, - PrimaryKeyToAssociations, SchemaExt, ToDFSchema, + PrimaryKeysToAssociations, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index f490d9848fa1b..a7ef06da93b7f 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -517,10 +517,9 @@ impl SessionContext { // all of the primary keys are associated with all of the fields (since it is source). let association_indices = (0..fields.len()).collect::>(); - let primary_keys_with_associations = primary_keys - .iter() - .map(|idx| (*idx, (true, association_indices.clone()))) - .collect(); + let mut primary_keys_with_associations = HashMap::new(); + primary_keys_with_associations + .insert(primary_keys.clone(), (true, association_indices)); let updated_schema = DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? .with_primary_keys(primary_keys_with_associations); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 818866d53853c..f3a6f5982d2dc 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -63,7 +63,7 @@ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; use datafusion_common::{ - add_offset_to_primary_key, DFSchema, PrimaryKeyToAssociations, ScalarValue, + add_offset_to_primary_key, DFSchema, PrimaryKeysToAssociations, ScalarValue, }; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, @@ -1319,20 +1319,30 @@ impl DefaultPhysicalPlanner { fn get_updated_primary_keys( df_schema: &DFSchema, field_indices: &[usize], -) -> PrimaryKeyToAssociations { +) -> PrimaryKeysToAssociations { let mut primary_keys = HashMap::new(); let existing_primary_keys = df_schema.primary_keys(); - for (idx, field_idx) in field_indices.iter().enumerate() { - if let Some((is_unique, associated_indices)) = - existing_primary_keys.get(field_idx) - { - let mut new_associated_indices = vec![]; - for (idx, field_idx) in field_indices.iter().enumerate() { - if associated_indices.contains(field_idx) { - new_associated_indices.push(idx); - } + for (pk_indices, (is_unique, associations)) in existing_primary_keys { + let mut new_pk_indices = vec![]; + for pk_idx in pk_indices { + if let Some(new_pk_idx) = field_indices + .iter() + .position(|field_idx| field_idx == pk_idx) + { + new_pk_indices.push(new_pk_idx); } - primary_keys.insert(idx, (*is_unique, new_associated_indices)); + } + let mut new_association_indices = vec![]; + for assoc_idx in associations { + if let Some(new_assoc_idx) = field_indices + .iter() + .position(|field_idx| field_idx == assoc_idx) + { + new_association_indices.push(new_assoc_idx); + } + } + if !new_pk_indices.is_empty() { + primary_keys.insert(new_pk_indices, (*is_unique, new_association_indices)); } } primary_keys @@ -2472,11 +2482,11 @@ mod tests { let mut schema = get_test_dfschema().unwrap(); let mut primary_keys = HashMap::new(); - primary_keys.insert(1, (true, vec![0, 1, 2])); + primary_keys.insert(vec![1], (true, vec![0, 1, 2])); schema = schema.with_primary_keys(primary_keys); let res = get_updated_primary_keys(&schema, &[1, 2]); let mut expected = HashMap::new(); - expected.insert(0, (true, vec![0, 1])); + expected.insert(vec![0], (true, vec![0, 1])); assert_eq!(res, expected); } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index df171aae3e6ea..fb78b5d032648 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -42,7 +42,7 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ add_offset_to_primary_key, display::ToStringifiedPlan, Column, DFField, DFSchema, - DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeyToAssociations, Result, + DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeysToAssociations, Result, ScalarValue, TableReference, ToDFSchema, }; use std::any::Any; @@ -265,11 +265,15 @@ impl LogicalPlanBuilder { let schema = table_source.schema(); let mut primary_keys = HashMap::new(); - let n_field = schema.fields.len(); - // All the field indices are associated, since it is source, - let associated_indices = (0..n_field).collect::>(); - for pk in table_source.primary_keys() { - primary_keys.insert(*pk, (true, associated_indices.clone())); + let pks = table_source.primary_keys(); + if !pks.is_empty() { + let n_field = schema.fields.len(); + // All the field indices are associated, since it is source, + let associated_indices = (0..n_field).collect::>(); + primary_keys.insert( + pks.to_vec(), + (true, associated_indices), + ); } let projected_schema = projection @@ -1260,7 +1264,7 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result { +) -> Result { let mut updated_primary_key = HashMap::new(); let mut input_to_proj_mapping: HashMap = HashMap::new(); @@ -1282,8 +1286,14 @@ pub(crate) fn project_primary_keys( } // expr.to_field(input_schema) } - for (idx, (is_unique, associations)) in input_schema.primary_keys() { - if let Some(target) = input_to_proj_mapping.get(idx) { + for (pk_indices, (is_unique, associations)) in input_schema.primary_keys() { + let mut target_pk_indices = vec![]; + for pk_idx in pk_indices { + if let Some(target_pk_idx) = input_to_proj_mapping.get(pk_idx) { + target_pk_indices.push(*target_pk_idx); + } + } + if !target_pk_indices.is_empty() { // Do mapping for associations let updated_associations = if *is_unique { (0..exprs.len()).collect() @@ -1298,7 +1308,8 @@ pub(crate) fn project_primary_keys( new_associations }; if !updated_associations.is_empty() { - updated_primary_key.insert(*target, (*is_unique, updated_associations)); + updated_primary_key + .insert(target_pk_indices, (*is_unique, updated_associations)); } } } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 172e9b9cfabd0..86b36b75da4f7 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -440,10 +440,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|elem| elem.qualified_name()) .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); - for (pk, (_is_unique, associations)) in primary_keys { + for (pk_indices, (_is_unique, associations)) in primary_keys { + let field_pk_names = pk_indices + .iter() + .map(|pk_idx| field_names[*pk_idx].as_str()) + .collect::>(); for group_by_expr in &group_by_exprs { let expr_name = format!("{}", group_by_expr); - if field_names[*pk] == expr_name { + if field_pk_names.contains(&expr_name.as_str()) { let associated_field_names = associations .iter() .map(|idx| field_names[*idx].clone()) From 9242581008df63b9165e82eab4d2bc048ed330ac Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 6 Jul 2023 15:17:53 +0300 Subject: [PATCH 14/37] Move hash map to struct --- datafusion/common/src/dfschema.rs | 75 +++++++++++++++------ datafusion/common/src/lib.rs | 2 +- datafusion/core/src/execution/context.rs | 12 ++-- datafusion/core/src/physical_planner.rs | 35 +++++----- datafusion/expr/src/logical_plan/builder.rs | 65 +++++++++--------- datafusion/expr/src/logical_plan/plan.rs | 14 ++-- datafusion/sql/src/select.rs | 13 ++-- 7 files changed, 136 insertions(+), 80 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e37e67c42e392..e767e89b6b9b1 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,7 +33,37 @@ use std::fmt::{Display, Formatter}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -pub type PrimaryKeysToAssociations = HashMap, (bool, Vec)>; +/// Stores primary keys and their associated indices +/// (where primary key property holds, this changes during intermediate +/// schemas) +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct PrimaryKeysAndAssociations { + pub primary_keys: Vec, + /// Flag indicating whether primary_key is exactly unique + /// if `false` it means that primary key isn't necessarily unique. + /// When primary_key is non-unique, all the associated fields at `associated_indices` + /// are same also. Primary key still refers to the unique set of associated fields, however same result + /// may occur replicated. + pub is_unique: bool, + pub associated_indices: Vec, +} + +impl PrimaryKeysAndAssociations { + pub fn new(primary_keys: Vec, associated_indices: Vec) -> Self { + Self { + primary_keys, + is_unique: true, + associated_indices, + } + } + + pub fn with_is_unique(mut self, is_unique: bool) -> Self { + self.is_unique = is_unique; + self + } +} + +pub type PrimaryKeysGroups = Vec; /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] @@ -42,8 +72,8 @@ pub struct DFSchema { fields: Vec, /// Additional metadata in form of key value pairs metadata: HashMap, - /// primary key index and its associated fields indices - primary_keys: PrimaryKeysToAssociations, + /// Stores distinct primary key groups + primary_keys: PrimaryKeysGroups, } impl DFSchema { @@ -52,7 +82,7 @@ impl DFSchema { Self { fields: vec![], metadata: HashMap::new(), - primary_keys: HashMap::new(), + primary_keys: vec![], } } @@ -105,7 +135,7 @@ impl DFSchema { Ok(Self { fields, metadata, - primary_keys: HashMap::new(), + primary_keys: vec![], }) } @@ -126,7 +156,7 @@ impl DFSchema { } /// Assing primary key - pub fn with_primary_keys(mut self, primary_keys: PrimaryKeysToAssociations) -> Self { + pub fn with_primary_keys(mut self, primary_keys: PrimaryKeysGroups) -> Self { self.primary_keys = primary_keys; self } @@ -496,7 +526,7 @@ impl DFSchema { } /// Get primary keys - pub fn primary_keys(&self) -> &PrimaryKeysToAssociations { + pub fn primary_keys(&self) -> &PrimaryKeysGroups { &self.primary_keys } } @@ -783,24 +813,31 @@ impl SchemaExt for Schema { /// Add offset value to primary_keys and its associated indices pub fn add_offset_to_primary_key( - primary_keys: &PrimaryKeysToAssociations, + primary_keys: &PrimaryKeysGroups, offset: usize, -) -> PrimaryKeysToAssociations { +) -> PrimaryKeysGroups { primary_keys .iter() - .map(|(pk_indices, (is_unique, associated_indices))| { - ( - pk_indices.iter().map(|pk_idx| pk_idx + offset).collect(), - ( - *is_unique, - associated_indices + .map( + |PrimaryKeysAndAssociations { + primary_keys, + is_unique, + associated_indices, + }| { + PrimaryKeysAndAssociations { + primary_keys: primary_keys + .iter() + .map(|pk_idx| pk_idx + offset) + .collect(), + is_unique: *is_unique, + associated_indices: associated_indices .iter() .map(|item| item + offset) .collect(), - ), - ) - }) - .collect::() + } + }, + ) + .collect::() } #[cfg(test)] diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 2707a54722a90..c368dd582c68e 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -38,7 +38,7 @@ pub mod utils; pub use column::Column; pub use dfschema::{ add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, - PrimaryKeysToAssociations, SchemaExt, ToDFSchema, + PrimaryKeysAndAssociations, PrimaryKeysGroups, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index a7ef06da93b7f..f100246c4c36b 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -86,7 +86,9 @@ use crate::physical_planner::PhysicalPlanner; use crate::variable::{VarProvider, VarType}; use async_trait::async_trait; use chrono::{DateTime, Utc}; -use datafusion_common::{DFSchema, OwnedTableReference, SchemaReference}; +use datafusion_common::{ + DFSchema, OwnedTableReference, PrimaryKeysAndAssociations, SchemaReference, +}; use datafusion_sql::{ parser::DFParser, planner::{ContextProvider, SqlToRel}, @@ -517,9 +519,11 @@ impl SessionContext { // all of the primary keys are associated with all of the fields (since it is source). let association_indices = (0..fields.len()).collect::>(); - let mut primary_keys_with_associations = HashMap::new(); - primary_keys_with_associations - .insert(primary_keys.clone(), (true, association_indices)); + let primary_keys_with_associations = + vec![PrimaryKeysAndAssociations::new( + primary_keys.clone(), + association_indices, + )]; let updated_schema = DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? .with_primary_keys(primary_keys_with_associations); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index f3a6f5982d2dc..6b05cbcdfd3ac 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -63,7 +63,8 @@ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; use datafusion_common::{ - add_offset_to_primary_key, DFSchema, PrimaryKeysToAssociations, ScalarValue, + add_offset_to_primary_key, DFSchema, PrimaryKeysAndAssociations, PrimaryKeysGroups, + ScalarValue, }; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, @@ -1034,9 +1035,7 @@ impl DefaultPhysicalPlanner { // after join, indices from the right table will increase with the size of the left table. hence add offset let right_primary_keys = add_offset_to_primary_key(&right_primary_keys, left_field_indices.len()); - let mut primary_keys = HashMap::new(); - primary_keys.extend(left_primary_keys); - primary_keys.extend(right_primary_keys); + let primary_keys = left_primary_keys.into_iter().chain(right_primary_keys).collect(); // Construct intermediate schemas used for filtering data and // convert logical expression to physical according to filter schema @@ -1319,12 +1318,17 @@ impl DefaultPhysicalPlanner { fn get_updated_primary_keys( df_schema: &DFSchema, field_indices: &[usize], -) -> PrimaryKeysToAssociations { - let mut primary_keys = HashMap::new(); +) -> PrimaryKeysGroups { + let mut updated_primary_keys = vec![]; let existing_primary_keys = df_schema.primary_keys(); - for (pk_indices, (is_unique, associations)) in existing_primary_keys { + for PrimaryKeysAndAssociations { + primary_keys, + is_unique, + associated_indices, + } in existing_primary_keys + { let mut new_pk_indices = vec![]; - for pk_idx in pk_indices { + for pk_idx in primary_keys { if let Some(new_pk_idx) = field_indices .iter() .position(|field_idx| field_idx == pk_idx) @@ -1333,7 +1337,7 @@ fn get_updated_primary_keys( } } let mut new_association_indices = vec![]; - for assoc_idx in associations { + for assoc_idx in associated_indices { if let Some(new_assoc_idx) = field_indices .iter() .position(|field_idx| field_idx == assoc_idx) @@ -1342,10 +1346,13 @@ fn get_updated_primary_keys( } } if !new_pk_indices.is_empty() { - primary_keys.insert(new_pk_indices, (*is_unique, new_association_indices)); + let new_pk_group = + PrimaryKeysAndAssociations::new(new_pk_indices, new_association_indices) + .with_is_unique(*is_unique); + updated_primary_keys.push(new_pk_group); } } - primary_keys + updated_primary_keys } /// Expand and align a GROUPING SET expression. @@ -2481,12 +2488,10 @@ mod tests { async fn test_get_updated_primary_keys() { let mut schema = get_test_dfschema().unwrap(); - let mut primary_keys = HashMap::new(); - primary_keys.insert(vec![1], (true, vec![0, 1, 2])); + let primary_keys = vec![PrimaryKeysAndAssociations::new(vec![1], vec![0, 1, 2])]; schema = schema.with_primary_keys(primary_keys); let res = get_updated_primary_keys(&schema, &[1, 2]); - let mut expected = HashMap::new(); - expected.insert(vec![0], (true, vec![0, 1])); + let expected = vec![PrimaryKeysAndAssociations::new(vec![0], vec![0, 1])]; assert_eq!(res, expected); } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index fb78b5d032648..3e1175d896bad 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -42,8 +42,8 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ add_offset_to_primary_key, display::ToStringifiedPlan, Column, DFField, DFSchema, - DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeysToAssociations, Result, - ScalarValue, TableReference, ToDFSchema, + DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeysAndAssociations, + PrimaryKeysGroups, Result, ScalarValue, TableReference, ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -264,16 +264,16 @@ impl LogicalPlanBuilder { } let schema = table_source.schema(); - let mut primary_keys = HashMap::new(); + let mut primary_keys = vec![]; let pks = table_source.primary_keys(); if !pks.is_empty() { let n_field = schema.fields.len(); // All the field indices are associated, since it is source, let associated_indices = (0..n_field).collect::>(); - primary_keys.insert( + primary_keys.push(PrimaryKeysAndAssociations::new( pks.to_vec(), - (true, associated_indices), - ); + associated_indices, + )); } let projected_schema = projection @@ -1101,7 +1101,6 @@ pub fn build_join_schema( } }; - // TODO: Update is_unique flag let mut right_primary_keys = add_offset_to_primary_key(right.primary_keys(), left_fields.len()); @@ -1110,29 +1109,31 @@ pub fn build_join_schema( // After join, primary key may be cloned. reset is_unique flag. left_primary_keys .iter_mut() - .for_each(|(_pk, (is_unique, _))| *is_unique = false); + .for_each(|pk_group| pk_group.is_unique = false); right_primary_keys .iter_mut() - .for_each(|(_pk, (is_unique, _))| *is_unique = false); - let mut primary_keys = HashMap::new(); - match join_type { + .for_each(|pk_group| pk_group.is_unique = false); + let primary_keys = match join_type { JoinType::Inner => { // left then right - primary_keys.extend(left_primary_keys); - primary_keys.extend(right_primary_keys); + left_primary_keys + .into_iter() + .chain(right_primary_keys.into_iter()) + .collect() } JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { // Only use the left side for the schema - primary_keys.extend(left_primary_keys); + left_primary_keys } JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { // Only use the right side for the schema - primary_keys.extend(right_primary_keys); + right_primary_keys } JoinType::Full => { // primary key is not preserved + vec![] } - } + }; let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); @@ -1264,19 +1265,14 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result { - let mut updated_primary_key = HashMap::new(); +) -> Result { + let mut updated_primary_key = vec![]; let mut input_to_proj_mapping: HashMap = HashMap::new(); let input_schema = &input.schema(); - // println!("exprs: {:?}", exprs); - // println!("plan schema: {:?}", input_schema); - // println!("existing primary keys: {:?}", input_schema.primary_keys()); // look for exact match in plan's output schema let input_fields = input_schema.fields(); for (idx, expr) in exprs.iter().enumerate() { - // println!("{}", format!("{:?}", expr)); - // println!("{}", format!("{}", expr)); let expr_name = format!("{}", expr); if let Some(match_idx) = input_fields .iter() @@ -1284,11 +1280,15 @@ pub(crate) fn project_primary_keys( { input_to_proj_mapping.insert(match_idx, idx); } - // expr.to_field(input_schema) } - for (pk_indices, (is_unique, associations)) in input_schema.primary_keys() { + for PrimaryKeysAndAssociations { + primary_keys, + is_unique, + associated_indices, + } in input_schema.primary_keys() + { let mut target_pk_indices = vec![]; - for pk_idx in pk_indices { + for pk_idx in primary_keys { if let Some(target_pk_idx) = input_to_proj_mapping.get(pk_idx) { target_pk_indices.push(*target_pk_idx); } @@ -1298,9 +1298,8 @@ pub(crate) fn project_primary_keys( let updated_associations = if *is_unique { (0..exprs.len()).collect() } else { - // println!("not unique, associations: {:?}", associations); let mut new_associations = vec![]; - for assoc_idx in associations { + for assoc_idx in associated_indices { if let Some(target_assoc_idx) = input_to_proj_mapping.get(assoc_idx) { new_associations.push(*target_assoc_idx); } @@ -1308,13 +1307,15 @@ pub(crate) fn project_primary_keys( new_associations }; if !updated_associations.is_empty() { - updated_primary_key - .insert(target_pk_indices, (*is_unique, updated_associations)); + let new_pk_group = PrimaryKeysAndAssociations::new( + target_pk_indices, + updated_associations, + ) + .with_is_unique(*is_unique); + updated_primary_key.push(new_pk_group); } } } - // println!("input_to_proj_mapping:{:?}", input_to_proj_mapping); - // println!("updated_primary_key:{:?}", updated_primary_key); Ok(updated_primary_key) } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index fcdb9d60e9c72..a6d09c4ccc3aa 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -36,7 +36,7 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{ plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - OwnedTableReference, Result, ScalarValue, + OwnedTableReference, PrimaryKeysAndAssociations, Result, ScalarValue, }; use std::collections::{HashMap, HashSet}; use std::fmt::{self, Debug, Display, Formatter}; @@ -1423,10 +1423,15 @@ impl Window { let n_window_expr = window_expr.len(); let new_associated_fields: Vec = (n_input_fields..n_input_fields + n_window_expr).collect(); - for (_pk, (is_unique, associations)) in primary_keys.iter_mut() { + for PrimaryKeysAndAssociations { + is_unique, + associated_indices, + .. + } in primary_keys.iter_mut() + { // if unique extend associations with new fields if *is_unique { - associations.extend(&new_associated_fields); + associated_indices.extend(&new_associated_fields); } } // let primary_keys = exprlist_to_primary_keys(window_expr.iter(), &input)?; @@ -1625,12 +1630,11 @@ impl Aggregate { let all_expr = grouping_expr.iter().chain(aggr_expr.iter()); // let primary_keys = exprlist_to_primary_keys(all_expr.clone(), &input)?; // TODO: Add handling for aggregate primary key - let primary_keys = HashMap::new(); let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), )? - .with_primary_keys(primary_keys); + .with_primary_keys(vec![]); Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 86b36b75da4f7..c0ef8e561d668 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -20,7 +20,7 @@ use crate::utils::{ check_columns_satisfy_exprs, extract_aliases, rebase_expr, resolve_aliases_to_exprs, resolve_columns, resolve_positions_to_exprs, }; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{DataFusionError, PrimaryKeysAndAssociations, Result}; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, }; @@ -440,15 +440,20 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|elem| elem.qualified_name()) .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); - for (pk_indices, (_is_unique, associations)) in primary_keys { - let field_pk_names = pk_indices + for PrimaryKeysAndAssociations { + primary_keys, + associated_indices, + .. + } in primary_keys + { + let field_pk_names = primary_keys .iter() .map(|pk_idx| field_names[*pk_idx].as_str()) .collect::>(); for group_by_expr in &group_by_exprs { let expr_name = format!("{}", group_by_expr); if field_pk_names.contains(&expr_name.as_str()) { - let associated_field_names = associations + let associated_field_names = associated_indices .iter() .map(|idx| field_names[*idx].clone()) .collect::>(); From 89719d98550ce76fb92408dcc3085206cf6d690c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 6 Jul 2023 16:11:49 +0300 Subject: [PATCH 15/37] simplifications --- datafusion/common/src/dfschema.rs | 44 +++-- datafusion/common/src/lib.rs | 2 +- .../src/datasource/default_table_source.rs | 2 +- datafusion/core/src/datasource/memory.rs | 12 +- datafusion/core/src/datasource/provider.rs | 6 +- datafusion/core/src/execution/context.rs | 11 +- datafusion/core/src/physical_planner.rs | 77 +-------- datafusion/expr/src/logical_plan/builder.rs | 152 ++++++++++-------- datafusion/expr/src/logical_plan/plan.rs | 4 +- datafusion/expr/src/table_source.rs | 6 +- datafusion/optimizer/src/optimizer.rs | 4 +- datafusion/sql/src/select.rs | 8 +- 12 files changed, 143 insertions(+), 185 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e767e89b6b9b1..8f9298123adef 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -37,8 +37,8 @@ pub type DFSchemaRef = Arc; /// (where primary key property holds, this changes during intermediate /// schemas) #[derive(Debug, Clone, PartialEq, Eq)] -pub struct PrimaryKeysAndAssociations { - pub primary_keys: Vec, +pub struct PrimaryKeyGroup { + pub primary_key_indices: Vec, /// Flag indicating whether primary_key is exactly unique /// if `false` it means that primary key isn't necessarily unique. /// When primary_key is non-unique, all the associated fields at `associated_indices` @@ -48,10 +48,10 @@ pub struct PrimaryKeysAndAssociations { pub associated_indices: Vec, } -impl PrimaryKeysAndAssociations { - pub fn new(primary_keys: Vec, associated_indices: Vec) -> Self { +impl PrimaryKeyGroup { + pub fn new(primary_key_indices: Vec, associated_indices: Vec) -> Self { Self { - primary_keys, + primary_key_indices, is_unique: true, associated_indices, } @@ -63,7 +63,7 @@ impl PrimaryKeysAndAssociations { } } -pub type PrimaryKeysGroups = Vec; +pub type PrimaryKeyGroups = Vec; /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] @@ -73,7 +73,7 @@ pub struct DFSchema { /// Additional metadata in form of key value pairs metadata: HashMap, /// Stores distinct primary key groups - primary_keys: PrimaryKeysGroups, + primary_keys: PrimaryKeyGroups, } impl DFSchema { @@ -156,7 +156,7 @@ impl DFSchema { } /// Assing primary key - pub fn with_primary_keys(mut self, primary_keys: PrimaryKeysGroups) -> Self { + pub fn with_primary_keys(mut self, primary_keys: PrimaryKeyGroups) -> Self { self.primary_keys = primary_keys; self } @@ -169,8 +169,18 @@ impl DFSchema { fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); let mut primary_keys = self.primary_keys.clone(); - let other_primary_keys = + primary_keys + .iter_mut() + .for_each(|PrimaryKeyGroup { is_unique, .. }| { + *is_unique = false; + }); + let mut other_primary_keys = add_offset_to_primary_key(schema.primary_keys(), self.fields.len()); + other_primary_keys.iter_mut().for_each( + |PrimaryKeyGroup { is_unique, .. }| { + *is_unique = false; + }, + ); primary_keys.extend(other_primary_keys); Ok(Self::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) } @@ -526,7 +536,7 @@ impl DFSchema { } /// Get primary keys - pub fn primary_keys(&self) -> &PrimaryKeysGroups { + pub fn primary_keys(&self) -> &PrimaryKeyGroups { &self.primary_keys } } @@ -813,19 +823,19 @@ impl SchemaExt for Schema { /// Add offset value to primary_keys and its associated indices pub fn add_offset_to_primary_key( - primary_keys: &PrimaryKeysGroups, + primary_keys: &PrimaryKeyGroups, offset: usize, -) -> PrimaryKeysGroups { +) -> PrimaryKeyGroups { primary_keys .iter() .map( - |PrimaryKeysAndAssociations { - primary_keys, + |PrimaryKeyGroup { + primary_key_indices, is_unique, associated_indices, }| { - PrimaryKeysAndAssociations { - primary_keys: primary_keys + PrimaryKeyGroup { + primary_key_indices: primary_key_indices .iter() .map(|pk_idx| pk_idx + offset) .collect(), @@ -837,7 +847,7 @@ pub fn add_offset_to_primary_key( } }, ) - .collect::() + .collect::() } #[cfg(test)] diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index c368dd582c68e..9254e45082f23 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -38,7 +38,7 @@ pub mod utils; pub use column::Column; pub use dfschema::{ add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, - PrimaryKeysAndAssociations, PrimaryKeysGroups, SchemaExt, ToDFSchema, + PrimaryKeyGroup, PrimaryKeyGroups, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 36b49dee94de6..6a09ae2f9d3cc 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -53,7 +53,7 @@ impl TableSource for DefaultTableSource { } /// Get a reference to the primary key indices - fn primary_keys(&self) -> &[usize] { + fn primary_keys(&self) -> Option<&[usize]> { self.table_provider.primary_keys() } diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 60bfe599366f4..225113a923a4e 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -52,7 +52,7 @@ pub type PartitionData = Arc>>; pub struct MemTable { schema: SchemaRef, pub(crate) batches: Vec, - primary_keys: Vec, + primary_keys: Option>, } impl MemTable { @@ -77,13 +77,15 @@ impl MemTable { .into_iter() .map(|e| Arc::new(RwLock::new(e))) .collect::>(), - primary_keys: vec![], + primary_keys: None, }) } /// Assign primary keys pub fn with_primary_keys(mut self, primary_keys: Vec) -> Self { - self.primary_keys = primary_keys; + if !primary_keys.is_empty() { + self.primary_keys = Some(primary_keys); + } self } @@ -156,8 +158,8 @@ impl TableProvider for MemTable { self.schema.clone() } - fn primary_keys(&self) -> &[usize] { - &self.primary_keys + fn primary_keys(&self) -> Option<&[usize]> { + self.primary_keys.as_deref() } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index f727b55870cd8..eb21d773aca1b 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -42,9 +42,9 @@ pub trait TableProvider: Sync + Send { fn schema(&self) -> SchemaRef; /// Get a reference to the primary_key indices - fn primary_keys(&self) -> &[usize] { - // By default return empty reference - &[] + fn primary_keys(&self) -> Option<&[usize]> { + // By default return None + None } /// Get the type of this table for metadata/catalog purposes. diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index f100246c4c36b..49a3480e28754 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -87,7 +87,7 @@ use crate::variable::{VarProvider, VarType}; use async_trait::async_trait; use chrono::{DateTime, Utc}; use datafusion_common::{ - DFSchema, OwnedTableReference, PrimaryKeysAndAssociations, SchemaReference, + DFSchema, OwnedTableReference, PrimaryKeyGroup, SchemaReference, }; use datafusion_sql::{ parser::DFParser, @@ -519,11 +519,10 @@ impl SessionContext { // all of the primary keys are associated with all of the fields (since it is source). let association_indices = (0..fields.len()).collect::>(); - let primary_keys_with_associations = - vec![PrimaryKeysAndAssociations::new( - primary_keys.clone(), - association_indices, - )]; + let primary_keys_with_associations = vec![PrimaryKeyGroup::new( + primary_keys.clone(), + association_indices, + )]; let updated_schema = DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? .with_primary_keys(primary_keys_with_associations); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 6b05cbcdfd3ac..34ff6b7d240d0 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -62,10 +62,8 @@ use crate::{ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; -use datafusion_common::{ - add_offset_to_primary_key, DFSchema, PrimaryKeysAndAssociations, PrimaryKeysGroups, - ScalarValue, -}; +use datafusion_common::{add_offset_to_primary_key, DFSchema, ScalarValue}; +use datafusion_expr::builder::project_primary_key_indices; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, WindowFunction, @@ -1029,8 +1027,8 @@ impl DefaultPhysicalPlanner { ) .unzip(); // Update indices of the fields according to filter schema (the index seen in the filter schema). - let left_primary_keys = get_updated_primary_keys(left_df_schema, &left_field_indices); - let right_primary_keys = get_updated_primary_keys(right_df_schema, &right_field_indices); + let left_primary_keys = project_primary_key_indices(left_df_schema.primary_keys(), &left_field_indices, left_field_indices.len()); + let right_primary_keys = project_primary_key_indices(right_df_schema.primary_keys(), &right_field_indices, right_field_indices.len()); // after join, indices from the right table will increase with the size of the left table. hence add offset let right_primary_keys = add_offset_to_primary_key(&right_primary_keys, left_field_indices.len()); @@ -1310,51 +1308,6 @@ impl DefaultPhysicalPlanner { } } -/// Update primary key indices, with index of the number in `field_indices` -/// If `field_indices` is [2, 5, 8], primary keys are 5 -> [5, 8] in the `df_schema`. -/// return value will be 1 -> [1, 2]. This means that 1st index of the `field_indices`(5) is primary key, -/// and this primary key is associated with columns at indices 1 and 2 (in the updated schema). -/// In the updated schema, fields at the indices [2, 5, 8] will be at [0, 1, 2]. -fn get_updated_primary_keys( - df_schema: &DFSchema, - field_indices: &[usize], -) -> PrimaryKeysGroups { - let mut updated_primary_keys = vec![]; - let existing_primary_keys = df_schema.primary_keys(); - for PrimaryKeysAndAssociations { - primary_keys, - is_unique, - associated_indices, - } in existing_primary_keys - { - let mut new_pk_indices = vec![]; - for pk_idx in primary_keys { - if let Some(new_pk_idx) = field_indices - .iter() - .position(|field_idx| field_idx == pk_idx) - { - new_pk_indices.push(new_pk_idx); - } - } - let mut new_association_indices = vec![]; - for assoc_idx in associated_indices { - if let Some(new_assoc_idx) = field_indices - .iter() - .position(|field_idx| field_idx == assoc_idx) - { - new_association_indices.push(new_assoc_idx); - } - } - if !new_pk_indices.is_empty() { - let new_pk_group = - PrimaryKeysAndAssociations::new(new_pk_indices, new_association_indices) - .with_is_unique(*is_unique); - updated_primary_keys.push(new_pk_group); - } - } - updated_primary_keys -} - /// Expand and align a GROUPING SET expression. /// (see ) /// @@ -2259,7 +2212,7 @@ mod tests { dict_id: 0, \ dict_is_ordered: false, \ metadata: {} } }\ - ], metadata: {}, primary_keys: {} }, \ + ], metadata: {}, primary_keys: [] }, \ ExecutionPlan schema: Schema { fields: [\ Field { \ name: \"b\", \ @@ -2484,26 +2437,6 @@ mod tests { } } - #[tokio::test] - async fn test_get_updated_primary_keys() { - let mut schema = get_test_dfschema().unwrap(); - - let primary_keys = vec![PrimaryKeysAndAssociations::new(vec![1], vec![0, 1, 2])]; - schema = schema.with_primary_keys(primary_keys); - let res = get_updated_primary_keys(&schema, &[1, 2]); - let expected = vec![PrimaryKeysAndAssociations::new(vec![0], vec![0, 1])]; - assert_eq!(res, expected); - } - - fn get_test_dfschema() -> Result { - let fields = vec![ - DFField::new_unqualified("a", DataType::Int64, false), - DFField::new_unqualified("b", DataType::Int64, false), - DFField::new_unqualified("c", DataType::Int64, false), - ]; - DFSchema::new_with_metadata(fields, HashMap::new()) - } - struct ErrorExtensionPlanner {} #[async_trait] diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 3e1175d896bad..8ea8cce880afb 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -42,8 +42,8 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ add_offset_to_primary_key, display::ToStringifiedPlan, Column, DFField, DFSchema, - DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeysAndAssociations, - PrimaryKeysGroups, Result, ScalarValue, TableReference, ToDFSchema, + DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeyGroup, PrimaryKeyGroups, + Result, ScalarValue, TableReference, ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -265,20 +265,18 @@ impl LogicalPlanBuilder { let schema = table_source.schema(); let mut primary_keys = vec![]; - let pks = table_source.primary_keys(); - if !pks.is_empty() { + if let Some(pks) = table_source.primary_keys() { let n_field = schema.fields.len(); // All the field indices are associated, since it is source, let associated_indices = (0..n_field).collect::>(); - primary_keys.push(PrimaryKeysAndAssociations::new( - pks.to_vec(), - associated_indices, - )); + primary_keys.push(PrimaryKeyGroup::new(pks.to_vec(), associated_indices)); } let projected_schema = projection .as_ref() .map(|p| { + let projected_primary_keys = + project_primary_key_indices(&primary_keys, p, p.len()); DFSchema::new_with_metadata( p.iter() .map(|i| { @@ -290,7 +288,7 @@ impl LogicalPlanBuilder { .collect(), schema.metadata().clone(), ) - .map(|df_schema| df_schema.with_primary_keys(primary_keys.clone())) + .map(|df_schema| df_schema.with_primary_keys(projected_primary_keys)) }) .unwrap_or_else(|| { DFSchema::try_from_qualified_schema(table_name.clone(), &schema) @@ -1262,61 +1260,81 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result { - let mut updated_primary_key = vec![]; - let mut input_to_proj_mapping: HashMap = HashMap::new(); +// Update entries inside the `entries` vector, with their corresponding index, inside `proj_indices` vector. +fn update_elements_with_matching_indices( + entries: &[usize], + proj_indices: &[usize], +) -> Vec { + entries + .iter() + .filter_map(|val| proj_indices.iter().position(|proj_idx| proj_idx == val)) + .collect() +} - let input_schema = &input.schema(); - // look for exact match in plan's output schema - let input_fields = input_schema.fields(); - for (idx, expr) in exprs.iter().enumerate() { - let expr_name = format!("{}", expr); - if let Some(match_idx) = input_fields - .iter() - .position(|item| item.qualified_name() == expr_name) - { - input_to_proj_mapping.insert(match_idx, idx); - } - } - for PrimaryKeysAndAssociations { - primary_keys, +/// Update primary key indices, with index of the number in `field_indices` +/// If `proj_indices` is \[2, 5, 8\], primary key groups is \[5\] -> \[5, 8\] in the `df_schema`. +/// return value will be \[1\] -> \[1, 2\]. This means that 1st index of the `field_indices`(5) is primary key, +/// and this primary key is associated with columns at indices 1 and 2 (in the updated schema). +/// In the updated schema, fields at the indices \[2, 5, 8\] will be at \[0, 1, 2\]. +pub fn project_primary_key_indices( + primary_key_groups: &PrimaryKeyGroups, + proj_indices: &[usize], + // If is_unique flag of primary key group is true. Association covers whole table. This + // number stores table size to be able to correctly associate with whole table. + n_out: usize, +) -> PrimaryKeyGroups { + let mut updated_primary_groups = vec![]; + for PrimaryKeyGroup { + primary_key_indices, is_unique, associated_indices, - } in input_schema.primary_keys() + } in primary_key_groups { - let mut target_pk_indices = vec![]; - for pk_idx in primary_keys { - if let Some(target_pk_idx) = input_to_proj_mapping.get(pk_idx) { - target_pk_indices.push(*target_pk_idx); - } + let new_pk_indices = + update_elements_with_matching_indices(primary_key_indices, proj_indices); + // Do mapping for associations + let new_association_indices = if *is_unique { + (0..n_out).collect() + } else { + // new_associations + update_elements_with_matching_indices(associated_indices, proj_indices) + }; + if !new_pk_indices.is_empty() { + let new_pk_group = + PrimaryKeyGroup::new(new_pk_indices, new_association_indices) + .with_is_unique(*is_unique); + updated_primary_groups.push(new_pk_group); } - if !target_pk_indices.is_empty() { - // Do mapping for associations - let updated_associations = if *is_unique { - (0..exprs.len()).collect() - } else { - let mut new_associations = vec![]; - for assoc_idx in associated_indices { - if let Some(target_assoc_idx) = input_to_proj_mapping.get(assoc_idx) { - new_associations.push(*target_assoc_idx); - } + } + updated_primary_groups +} + +pub(crate) fn project_primary_keys( + exprs: &[Expr], + input: &LogicalPlan, +) -> Result { + // look for exact match in plan's output schema + let input_fields = input.schema().fields(); + // Calculate expression indices (if found) in the input schema. + let proj_indices = exprs + .iter() + .filter_map(|expr| { + let expr_name = match expr { + Expr::Alias(alias) => { + format!("{}", alias.expr) } - new_associations + _ => format!("{}", expr), }; - if !updated_associations.is_empty() { - let new_pk_group = PrimaryKeysAndAssociations::new( - target_pk_indices, - updated_associations, - ) - .with_is_unique(*is_unique); - updated_primary_key.push(new_pk_group); - } - } - } - Ok(updated_primary_key) + input_fields + .iter() + .position(|item| item.qualified_name() == expr_name) + }) + .collect::>(); + Ok(project_primary_key_indices( + input.schema().primary_keys(), + &proj_indices, + exprs.len(), + )) } /// Create Projection @@ -1345,19 +1363,7 @@ pub fn project( } validate_unique_names("Projections", projected_expr.iter())?; - // println!("projected exprs"); - // for item in &projected_expr { - // println!("item:{:?}", item); - // } - // println!("projected exprs"); - let primary_keys = project_primary_keys(&projected_expr, &plan)?; - // let primary_keys = exprlist_to_primary_keys(&projected_expr, &plan)?; - - // println!("---------------PROJECTION--------------"); - // print_primary_key(&plan); - // println!("projection primary_keys: {:?}", primary_keys); - // println!("---------------PROJECTION--------------"); let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), @@ -2047,4 +2053,12 @@ mod tests { Ok(()) } + + #[test] + fn test_get_updated_primary_keys() { + let primary_keys = vec![PrimaryKeyGroup::new(vec![1], vec![0, 1, 2])]; + let res = project_primary_key_indices(&primary_keys, &[1, 2], 2); + let expected = vec![PrimaryKeyGroup::new(vec![0], vec![0, 1])]; + assert_eq!(res, expected); + } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index a6d09c4ccc3aa..68893d448af60 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -36,7 +36,7 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{ plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - OwnedTableReference, PrimaryKeysAndAssociations, Result, ScalarValue, + OwnedTableReference, PrimaryKeyGroup, Result, ScalarValue, }; use std::collections::{HashMap, HashSet}; use std::fmt::{self, Debug, Display, Formatter}; @@ -1423,7 +1423,7 @@ impl Window { let n_window_expr = window_expr.len(); let new_associated_fields: Vec = (n_input_fields..n_input_fields + n_window_expr).collect(); - for PrimaryKeysAndAssociations { + for PrimaryKeyGroup { is_unique, associated_indices, .. diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 77fe161f5e05f..a037e9c49bbca 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -65,9 +65,9 @@ pub trait TableSource: Sync + Send { fn schema(&self) -> SchemaRef; /// Get primary key index if any - fn primary_keys(&self) -> &[usize] { - // By default, return empty reference - &[] + fn primary_keys(&self) -> Option<&[usize]> { + // By default return None. + None } /// Get the type of this table for metadata/catalog purposes. diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 771a6e64aa0e3..fe159bf54b56f 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -493,12 +493,12 @@ mod tests { assert_eq!( "get table_scan rule\ncaused by\n\ Internal error: Optimizer rule 'get table_scan rule' failed, due to generate a different schema, \ - original schema: DFSchema { fields: [], metadata: {}, primary_keys: {} }, \ + original schema: DFSchema { fields: [], metadata: {}, primary_keys: [] }, \ 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: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \ - metadata: {}, primary_keys: {} }. \ + metadata: {}, primary_keys: [] }. \ This was likely caused by a bug in DataFusion's code \ and we would welcome that you file an bug report in our issue tracker", err.to_string() diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index c0ef8e561d668..74db42e4ec732 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -20,7 +20,7 @@ use crate::utils::{ check_columns_satisfy_exprs, extract_aliases, rebase_expr, resolve_aliases_to_exprs, resolve_columns, resolve_positions_to_exprs, }; -use datafusion_common::{DataFusionError, PrimaryKeysAndAssociations, Result}; +use datafusion_common::{DataFusionError, PrimaryKeyGroup, Result}; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, }; @@ -440,13 +440,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|elem| elem.qualified_name()) .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); - for PrimaryKeysAndAssociations { - primary_keys, + for PrimaryKeyGroup { + primary_key_indices, associated_indices, .. } in primary_keys { - let field_pk_names = primary_keys + let field_pk_names = primary_key_indices .iter() .map(|pk_idx| field_names[*pk_idx].as_str()) .collect::>(); From b3632da6b3d29de12bad3735a1543753c4f370b4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 6 Jul 2023 17:57:30 +0300 Subject: [PATCH 16/37] Update comments --- datafusion/core/src/physical_planner.rs | 1 - datafusion/expr/src/logical_plan/builder.rs | 17 +++++++++++------ datafusion/expr/src/logical_plan/plan.rs | 15 +++++++-------- datafusion/expr/src/utils.rs | 2 ++ datafusion/sql/src/select.rs | 9 +++++++-- 5 files changed, 27 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 34ff6b7d240d0..2a730b3117495 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2438,7 +2438,6 @@ mod tests { } struct ErrorExtensionPlanner {} - #[async_trait] impl ExtensionPlanner for ErrorExtensionPlanner { /// Create a physical plan for an extension node diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 8ea8cce880afb..955f86da8b245 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1104,7 +1104,9 @@ pub fn build_join_schema( let mut left_primary_keys = left.primary_keys().clone(); - // After join, primary key may be cloned. reset is_unique flag. + // After join, primary key may no longer be unique + // (However, it still defines unique set of column values, just same values may be replicated). + // reset is_unique flag to false. left_primary_keys .iter_mut() .for_each(|pk_group| pk_group.is_unique = false); @@ -1279,8 +1281,8 @@ fn update_elements_with_matching_indices( pub fn project_primary_key_indices( primary_key_groups: &PrimaryKeyGroups, proj_indices: &[usize], - // If is_unique flag of primary key group is true. Association covers whole table. This - // number stores table size to be able to correctly associate with whole table. + // If is_unique flag of primary key group is true. Association covers whole table. `n_out` + // stores schema field length to be able to correctly associate with whole table. n_out: usize, ) -> PrimaryKeyGroups { let mut updated_primary_groups = vec![]; @@ -1292,11 +1294,11 @@ pub fn project_primary_key_indices( { let new_pk_indices = update_elements_with_matching_indices(primary_key_indices, proj_indices); - // Do mapping for associations let new_association_indices = if *is_unique { + // Associate with all of the fields in the schema (0..n_out).collect() } else { - // new_associations + // Update associations according to projection update_elements_with_matching_indices(associated_indices, proj_indices) }; if !new_pk_indices.is_empty() { @@ -1309,11 +1311,12 @@ pub fn project_primary_key_indices( updated_primary_groups } +/// This function projects primary key of the +/// `input`, according to projection expressions `exprs` pub(crate) fn project_primary_keys( exprs: &[Expr], input: &LogicalPlan, ) -> Result { - // look for exact match in plan's output schema let input_fields = input.schema().fields(); // Calculate expression indices (if found) in the input schema. let proj_indices = exprs @@ -1363,6 +1366,7 @@ pub fn project( } validate_unique_names("Projections", projected_expr.iter())?; + // Update input primary keys according to projection. let primary_keys = project_primary_keys(&projected_expr, &plan)?; let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, @@ -1533,6 +1537,7 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new( DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? + // we can use existing primary key, .with_primary_keys(input_schema.primary_keys().clone()), ); diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 68893d448af60..014c12c87d3bb 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1259,7 +1259,7 @@ pub struct Projection { impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { - // let primary_keys = exprlist_to_primary_keys(&expr, &input)?; + // update primary key of `input` according to projection exprs. let primary_keys = project_primary_keys(&expr, &input)?; let schema = Arc::new( DFSchema::new_with_metadata( @@ -1330,6 +1330,7 @@ impl SubqueryAlias { ) -> Result { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); + // Since schema is same, other than qualifier, we can use existing primary keys let primary_keys = plan.schema().primary_keys().clone(); let schema = DFSchemaRef::new( DFSchema::try_from_qualified_schema(&alias, &schema)? @@ -1420,21 +1421,21 @@ impl Window { // Update primary key for window let mut primary_keys = input.schema().primary_keys().clone(); let n_input_fields = input.schema().fields().len(); - let n_window_expr = window_expr.len(); let new_associated_fields: Vec = - (n_input_fields..n_input_fields + n_window_expr).collect(); + (n_input_fields..window_fields.len()).collect(); for PrimaryKeyGroup { is_unique, associated_indices, .. } in primary_keys.iter_mut() { - // if unique extend associations with new fields + // if unique, extend associations such that they cover + // new window expressions if *is_unique { associated_indices.extend(&new_associated_fields); } } - // let primary_keys = exprlist_to_primary_keys(window_expr.iter(), &input)?; + Ok(Window { input, window_expr, @@ -1628,13 +1629,11 @@ impl Aggregate { let group_expr = enumerate_grouping_sets(group_expr)?; let grouping_expr: Vec = grouping_set_to_exprlist(group_expr.as_slice())?; let all_expr = grouping_expr.iter().chain(aggr_expr.iter()); - // let primary_keys = exprlist_to_primary_keys(all_expr.clone(), &input)?; // TODO: Add handling for aggregate primary key let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), - )? - .with_primary_keys(vec![]); + )?; Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 71b32abaf719e..ad7785833969c 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -448,6 +448,7 @@ pub fn expand_qualified_wildcard( } let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? + // we can use primary key as is, since it only stores indices .with_primary_keys(schema.primary_keys().clone()); let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, @@ -1010,6 +1011,7 @@ pub fn from_plan( let schema = Arc::new( DFSchema::new_with_metadata(fields, input.schema().metadata().clone())? + // we can use existing primary keys as is .with_primary_keys(input.schema().primary_keys().clone()), ); diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 74db42e4ec732..eda63535666dd 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -452,15 +452,19 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .collect::>(); for group_by_expr in &group_by_exprs { let expr_name = format!("{}", group_by_expr); + // group by expression is primary key if field_pk_names.contains(&expr_name.as_str()) { let associated_field_names = associated_indices .iter() - .map(|idx| field_names[*idx].clone()) + .map(|idx| field_names[*idx].as_str()) .collect::>(); + // Expand group by exprs with select_exprs + // If one of the expressions inside group by is primary key and + // select expression is associated with that primary key. for expr in select_exprs { let expr_name = format!("{}", expr); if !new_group_by_exprs.contains(expr) - && associated_field_names.contains(&expr_name) + && associated_field_names.contains(&expr_name.as_str()) { new_group_by_exprs.push(expr.clone()); } @@ -474,6 +478,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let plan = LogicalPlanBuilder::from(input.clone()) .aggregate(group_by_exprs.clone(), aggr_exprs.clone())? .build()?; + // 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 From f152b5892453aea4dc7bbc32702c1e25c051f23e Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 18 Jul 2023 10:24:15 +0300 Subject: [PATCH 17/37] Update datafusion/core/src/physical_planner.rs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Metehan Yıldırım <100111937+metesynnada@users.noreply.github.com> --- datafusion/core/src/physical_planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 0c60442dcc023..10778886143e1 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1037,7 +1037,7 @@ impl DefaultPhysicalPlanner { // Construct intermediate schemas used for filtering data and // convert logical expression to physical according to filter schema - let filter_df_schema = DFSchema::new_with_metadata(filter_df_fields, HashMap::new())?.with_primary_keys(primary_keys); + let filter_df_schema = DFSchema::new_with_metadata(filter_df_fields, HashMap::new())?; let filter_schema = Schema::new_with_metadata(filter_fields, HashMap::new()); let filter_expr = create_physical_expr( expr, From 639c55c3c697cd9c9f15508b3aa21ac4d2d99d1b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 18 Jul 2023 10:38:00 +0300 Subject: [PATCH 18/37] Remove unnecessary code --- datafusion/core/src/physical_planner.rs | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 10778886143e1..64540fef7d42f 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -62,8 +62,7 @@ use crate::{ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; -use datafusion_common::{add_offset_to_primary_key, DFSchema, ScalarValue}; -use datafusion_expr::builder::project_primary_key_indices; +use datafusion_common::{DFSchema, ScalarValue}; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, WindowFunction, @@ -1026,14 +1025,6 @@ impl DefaultPhysicalPlanner { )) ) .unzip(); - // Update indices of the fields according to filter schema (the index seen in the filter schema). - let left_primary_keys = project_primary_key_indices(left_df_schema.primary_keys(), &left_field_indices, left_field_indices.len()); - let right_primary_keys = project_primary_key_indices(right_df_schema.primary_keys(), &right_field_indices, right_field_indices.len()); - - // after join, indices from the right table will increase with the size of the left table. hence add offset - let right_primary_keys = add_offset_to_primary_key(&right_primary_keys, left_field_indices.len()); - - let primary_keys = left_primary_keys.into_iter().chain(right_primary_keys).collect(); // Construct intermediate schemas used for filtering data and // convert logical expression to physical according to filter schema From dabf833a7dd462599c79e3814b61ca0fa806c5aa Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 18 Jul 2023 17:10:48 +0300 Subject: [PATCH 19/37] Rename, update comments --- datafusion/common/src/dfschema.rs | 99 ++++++++------- datafusion/common/src/lib.rs | 4 +- datafusion/core/src/execution/context.rs | 6 +- datafusion/expr/src/logical_plan/builder.rs | 115 +++++++++--------- datafusion/expr/src/logical_plan/plan.rs | 18 +-- datafusion/expr/src/utils.rs | 6 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- datafusion/sql/src/select.rs | 8 +- 8 files changed, 137 insertions(+), 121 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 0c926b33960ae..384b8354525c3 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,25 +33,27 @@ use std::fmt::{Display, Formatter}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -/// Stores primary keys and their associated indices -/// (where primary key property holds, this changes during intermediate -/// schemas) +/// Stores identifier keys and their associated indices +/// (where identifier key property holds, this may change during intermediate +/// schemas, such as join) #[derive(Debug, Clone, PartialEq, Eq)] -pub struct PrimaryKeyGroup { - pub primary_key_indices: Vec, - /// Flag indicating whether primary_key is exactly unique - /// if `false` it means that primary key isn't necessarily unique. - /// When primary_key is non-unique, all the associated fields at `associated_indices` - /// are same also. Primary key still refers to the unique set of associated fields, however same result - /// may occur replicated. +pub struct IdentifierKeyGroup { + pub identifier_key_indices: Vec, + /// Flag indicating whether identifiers are unique + /// if `true` it means that, these indices are primary keys. + /// if `false` it means that identifier still uniquely identifies its associated column + /// results. However, same results may occur replicated. pub is_unique: bool, pub associated_indices: Vec, } -impl PrimaryKeyGroup { - pub fn new(primary_key_indices: Vec, associated_indices: Vec) -> Self { +impl IdentifierKeyGroup { + pub fn new( + identifier_key_indices: Vec, + associated_indices: Vec, + ) -> Self { Self { - primary_key_indices, + identifier_key_indices, is_unique: true, associated_indices, } @@ -63,7 +65,7 @@ impl PrimaryKeyGroup { } } -pub type PrimaryKeyGroups = Vec; +pub type IdentifierKeyGroups = Vec; /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] @@ -72,8 +74,8 @@ pub struct DFSchema { fields: Vec, /// Additional metadata in form of key value pairs metadata: HashMap, - /// Stores distinct primary key groups - primary_keys: PrimaryKeyGroups, + /// Stores distinct identifier key groups + identifier_key_groups: IdentifierKeyGroups, } impl DFSchema { @@ -82,7 +84,7 @@ impl DFSchema { Self { fields: vec![], metadata: HashMap::new(), - primary_keys: vec![], + identifier_key_groups: vec![], } } @@ -135,7 +137,7 @@ impl DFSchema { Ok(Self { fields, metadata, - primary_keys: vec![], + identifier_key_groups: vec![], }) } @@ -155,9 +157,12 @@ impl DFSchema { ) } - /// Assing primary key - pub fn with_primary_keys(mut self, primary_keys: PrimaryKeyGroups) -> Self { - self.primary_keys = primary_keys; + /// Assign identifier key groups + pub fn with_identifier_key_groups( + mut self, + identifier_key_groups: IdentifierKeyGroups, + ) -> Self { + self.identifier_key_groups = identifier_key_groups; self } @@ -168,21 +173,24 @@ impl DFSchema { let mut metadata = self.metadata.clone(); fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); - let mut primary_keys = self.primary_keys.clone(); - primary_keys - .iter_mut() - .for_each(|PrimaryKeyGroup { is_unique, .. }| { + let mut identifier_key_groups = self.identifier_key_groups.clone(); + identifier_key_groups.iter_mut().for_each( + |IdentifierKeyGroup { is_unique, .. }| { *is_unique = false; - }); - let mut other_primary_keys = - add_offset_to_primary_key(schema.primary_keys(), self.fields.len()); - other_primary_keys.iter_mut().for_each( - |PrimaryKeyGroup { is_unique, .. }| { + }, + ); + let mut other_identifier_key_groups = add_offset_to_identifier_key_groups( + schema.identifier_key_groups(), + self.fields.len(), + ); + other_identifier_key_groups.iter_mut().for_each( + |IdentifierKeyGroup { is_unique, .. }| { *is_unique = false; }, ); - primary_keys.extend(other_primary_keys); - Ok(Self::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) + identifier_key_groups.extend(other_identifier_key_groups); + Ok(Self::new_with_metadata(fields, metadata)? + .with_identifier_key_groups(identifier_key_groups)) } /// Modify this schema by appending the fields from the supplied schema, ignoring any @@ -531,9 +539,9 @@ impl DFSchema { &self.metadata } - /// Get primary keys - pub fn primary_keys(&self) -> &PrimaryKeyGroups { - &self.primary_keys + /// Get Identifier key groups + pub fn identifier_key_groups(&self) -> &IdentifierKeyGroups { + &self.identifier_key_groups } } @@ -839,21 +847,22 @@ impl SchemaExt for Schema { } } -/// Add offset value to primary_keys and its associated indices -pub fn add_offset_to_primary_key( - primary_keys: &PrimaryKeyGroups, +/// Add offset value to identifier key indices and its associated indices +/// for each identifier key group +pub fn add_offset_to_identifier_key_groups( + identifier_key_groups: &IdentifierKeyGroups, offset: usize, -) -> PrimaryKeyGroups { - primary_keys +) -> IdentifierKeyGroups { + identifier_key_groups .iter() .map( - |PrimaryKeyGroup { - primary_key_indices, + |IdentifierKeyGroup { + identifier_key_indices, is_unique, associated_indices, }| { - PrimaryKeyGroup { - primary_key_indices: primary_key_indices + IdentifierKeyGroup { + identifier_key_indices: identifier_key_indices .iter() .map(|pk_idx| pk_idx + offset) .collect(), @@ -865,7 +874,7 @@ pub fn add_offset_to_primary_key( } }, ) - .collect::() + .collect::() } #[cfg(test)] diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 9254e45082f23..a422577d4c711 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -37,8 +37,8 @@ pub mod utils; pub use column::Column; pub use dfschema::{ - add_offset_to_primary_key, DFField, DFSchema, DFSchemaRef, ExprSchema, - PrimaryKeyGroup, PrimaryKeyGroups, SchemaExt, ToDFSchema, + add_offset_to_identifier_key_groups, DFField, DFSchema, DFSchemaRef, ExprSchema, + IdentifierKeyGroup, IdentifierKeyGroups, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 49a3480e28754..39353cef01b79 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -87,7 +87,7 @@ use crate::variable::{VarProvider, VarType}; use async_trait::async_trait; use chrono::{DateTime, Utc}; use datafusion_common::{ - DFSchema, OwnedTableReference, PrimaryKeyGroup, SchemaReference, + DFSchema, IdentifierKeyGroup, OwnedTableReference, SchemaReference, }; use datafusion_sql::{ parser::DFParser, @@ -519,13 +519,13 @@ impl SessionContext { // all of the primary keys are associated with all of the fields (since it is source). let association_indices = (0..fields.len()).collect::>(); - let primary_keys_with_associations = vec![PrimaryKeyGroup::new( + let primary_keys_with_associations = vec![IdentifierKeyGroup::new( primary_keys.clone(), association_indices, )]; let updated_schema = DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? - .with_primary_keys(primary_keys_with_associations); + .with_identifier_key_groups(primary_keys_with_associations); let schema = Arc::new(updated_schema.into()); diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 955f86da8b245..b2d980a52ffb6 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -41,9 +41,9 @@ use crate::{ }; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ - add_offset_to_primary_key, display::ToStringifiedPlan, Column, DFField, DFSchema, - DFSchemaRef, DataFusionError, OwnedTableReference, PrimaryKeyGroup, PrimaryKeyGroups, - Result, ScalarValue, TableReference, ToDFSchema, + add_offset_to_identifier_key_groups, display::ToStringifiedPlan, Column, DFField, + DFSchema, DFSchemaRef, DataFusionError, IdentifierKeyGroup, IdentifierKeyGroups, + OwnedTableReference, Result, ScalarValue, TableReference, ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -264,19 +264,19 @@ impl LogicalPlanBuilder { } let schema = table_source.schema(); - let mut primary_keys = vec![]; + let mut id_key_groups = vec![]; if let Some(pks) = table_source.primary_keys() { let n_field = schema.fields.len(); // All the field indices are associated, since it is source, let associated_indices = (0..n_field).collect::>(); - primary_keys.push(PrimaryKeyGroup::new(pks.to_vec(), associated_indices)); + id_key_groups.push(IdentifierKeyGroup::new(pks.to_vec(), associated_indices)); } let projected_schema = projection .as_ref() .map(|p| { - let projected_primary_keys = - project_primary_key_indices(&primary_keys, p, p.len()); + let projected_id_key_groups = + project_identifier_key_indices(&id_key_groups, p, p.len()); DFSchema::new_with_metadata( p.iter() .map(|i| { @@ -288,11 +288,13 @@ impl LogicalPlanBuilder { .collect(), schema.metadata().clone(), ) - .map(|df_schema| df_schema.with_primary_keys(projected_primary_keys)) + .map(|df_schema| { + df_schema.with_identifier_key_groups(projected_id_key_groups) + }) }) .unwrap_or_else(|| { DFSchema::try_from_qualified_schema(table_name.clone(), &schema) - .map(|df_schema| df_schema.with_primary_keys(primary_keys)) + .map(|df_schema| df_schema.with_identifier_key_groups(id_key_groups)) })?; let table_scan = LogicalPlan::TableScan(TableScan { @@ -1099,45 +1101,48 @@ pub fn build_join_schema( } }; - let mut right_primary_keys = - add_offset_to_primary_key(right.primary_keys(), left_fields.len()); + let mut right_id_key_groups = add_offset_to_identifier_key_groups( + right.identifier_key_groups(), + left_fields.len(), + ); - let mut left_primary_keys = left.primary_keys().clone(); + let mut left_id_key_groups = left.identifier_key_groups().clone(); - // After join, primary key may no longer be unique + // After join, identifier key may no longer be unique // (However, it still defines unique set of column values, just same values may be replicated). // reset is_unique flag to false. - left_primary_keys + left_id_key_groups .iter_mut() .for_each(|pk_group| pk_group.is_unique = false); - right_primary_keys + right_id_key_groups .iter_mut() .for_each(|pk_group| pk_group.is_unique = false); - let primary_keys = match join_type { + let id_key_groups = match join_type { JoinType::Inner => { // left then right - left_primary_keys + left_id_key_groups .into_iter() - .chain(right_primary_keys.into_iter()) + .chain(right_id_key_groups.into_iter()) .collect() } JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { // Only use the left side for the schema - left_primary_keys + left_id_key_groups } JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { // Only use the right side for the schema - right_primary_keys + right_id_key_groups } JoinType::Full => { - // primary key is not preserved + // identifier keys are not preserved vec![] } }; let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); - Ok(DFSchema::new_with_metadata(fields, metadata)?.with_primary_keys(primary_keys)) + Ok(DFSchema::new_with_metadata(fields, metadata)? + .with_identifier_key_groups(id_key_groups)) } /// Errors if one or more expressions have equal names. @@ -1273,27 +1278,27 @@ fn update_elements_with_matching_indices( .collect() } -/// Update primary key indices, with index of the number in `field_indices` -/// If `proj_indices` is \[2, 5, 8\], primary key groups is \[5\] -> \[5, 8\] in the `df_schema`. -/// return value will be \[1\] -> \[1, 2\]. This means that 1st index of the `field_indices`(5) is primary key, -/// and this primary key is associated with columns at indices 1 and 2 (in the updated schema). +/// Update identifier key indices, with index of the number in `field_indices` +/// If `proj_indices` is \[2, 5, 8\], and identifier key groups is \[5\] -> \[5, 8\] in the `df_schema`. +/// return value will be \[1\] -> \[1, 2\]. This means that 1st index of the `field_indices`(5) is identifier key, +/// and this identifier key is associated with columns at indices 1 and 2 (in the updated schema). /// In the updated schema, fields at the indices \[2, 5, 8\] will be at \[0, 1, 2\]. -pub fn project_primary_key_indices( - primary_key_groups: &PrimaryKeyGroups, +pub fn project_identifier_key_indices( + id_key_groups: &IdentifierKeyGroups, proj_indices: &[usize], - // If is_unique flag of primary key group is true. Association covers whole table. `n_out` + // If is_unique flag of identifier key group is true. Association covers whole table. `n_out` // stores schema field length to be able to correctly associate with whole table. n_out: usize, -) -> PrimaryKeyGroups { - let mut updated_primary_groups = vec![]; - for PrimaryKeyGroup { - primary_key_indices, +) -> IdentifierKeyGroups { + let mut updated_id_key_groups = vec![]; + for IdentifierKeyGroup { + identifier_key_indices, is_unique, associated_indices, - } in primary_key_groups + } in id_key_groups { - let new_pk_indices = - update_elements_with_matching_indices(primary_key_indices, proj_indices); + let new_id_key_indices = + update_elements_with_matching_indices(identifier_key_indices, proj_indices); let new_association_indices = if *is_unique { // Associate with all of the fields in the schema (0..n_out).collect() @@ -1301,22 +1306,22 @@ pub fn project_primary_key_indices( // Update associations according to projection update_elements_with_matching_indices(associated_indices, proj_indices) }; - if !new_pk_indices.is_empty() { - let new_pk_group = - PrimaryKeyGroup::new(new_pk_indices, new_association_indices) + if !new_id_key_indices.is_empty() { + let new_id_key_group = + IdentifierKeyGroup::new(new_id_key_indices, new_association_indices) .with_is_unique(*is_unique); - updated_primary_groups.push(new_pk_group); + updated_id_key_groups.push(new_id_key_group); } } - updated_primary_groups + updated_id_key_groups } -/// This function projects primary key of the +/// This function projects identifier key groups of the /// `input`, according to projection expressions `exprs` -pub(crate) fn project_primary_keys( +pub(crate) fn project_identifier_keys( exprs: &[Expr], input: &LogicalPlan, -) -> Result { +) -> Result { let input_fields = input.schema().fields(); // Calculate expression indices (if found) in the input schema. let proj_indices = exprs @@ -1333,8 +1338,8 @@ pub(crate) fn project_primary_keys( .position(|item| item.qualified_name() == expr_name) }) .collect::>(); - Ok(project_primary_key_indices( - input.schema().primary_keys(), + Ok(project_identifier_key_indices( + input.schema().identifier_key_groups(), &proj_indices, exprs.len(), )) @@ -1366,13 +1371,13 @@ pub fn project( } validate_unique_names("Projections", projected_expr.iter())?; - // Update input primary keys according to projection. - let primary_keys = project_primary_keys(&projected_expr, &plan)?; + // Update input identifier keys according to projection. + let id_key_groups = project_identifier_keys(&projected_expr, &plan)?; let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), )? - .with_primary_keys(primary_keys); + .with_identifier_key_groups(id_key_groups); Ok(LogicalPlan::Projection(Projection::try_new_with_schema( projected_expr, @@ -1537,8 +1542,8 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new( DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? - // we can use existing primary key, - .with_primary_keys(input_schema.primary_keys().clone()), + // we can use existing identifier key groups, + .with_identifier_key_groups(input_schema.identifier_key_groups().clone()), ); Ok(LogicalPlan::Unnest(Unnest { @@ -2060,10 +2065,10 @@ mod tests { } #[test] - fn test_get_updated_primary_keys() { - let primary_keys = vec![PrimaryKeyGroup::new(vec![1], vec![0, 1, 2])]; - let res = project_primary_key_indices(&primary_keys, &[1, 2], 2); - let expected = vec![PrimaryKeyGroup::new(vec![0], vec![0, 1])]; + fn test_get_updated_id_keys() { + let identifier_key_groups = vec![IdentifierKeyGroup::new(vec![1], vec![0, 1, 2])]; + let res = project_identifier_key_indices(&identifier_key_groups, &[1, 2], 2); + let expected = vec![IdentifierKeyGroup::new(vec![0], vec![0, 1])]; assert_eq!(res, expected); } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 0acf07ac1febc..0041318272c57 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -36,7 +36,7 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{ plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - OwnedTableReference, PrimaryKeyGroup, Result, ScalarValue, + IdentifierKeyGroup, OwnedTableReference, Result, ScalarValue, }; use std::collections::{HashMap, HashSet}; use std::fmt::{self, Debug, Display, Formatter}; @@ -44,7 +44,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; // backwards compatibility -use crate::builder::project_primary_keys; +use crate::builder::project_identifier_keys; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -1255,13 +1255,13 @@ impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { // update primary key of `input` according to projection exprs. - let primary_keys = project_primary_keys(&expr, &input)?; + let primary_keys = project_identifier_keys(&expr, &input)?; let schema = Arc::new( DFSchema::new_with_metadata( exprlist_to_fields(&expr, &input)?, input.schema().metadata().clone(), )? - .with_primary_keys(primary_keys), + .with_identifier_key_groups(primary_keys), ); Self::try_new_with_schema(expr, input, schema) } @@ -1326,10 +1326,10 @@ impl SubqueryAlias { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); // Since schema is same, other than qualifier, we can use existing primary keys - let primary_keys = plan.schema().primary_keys().clone(); + let primary_keys = plan.schema().identifier_key_groups().clone(); let schema = DFSchemaRef::new( DFSchema::try_from_qualified_schema(&alias, &schema)? - .with_primary_keys(primary_keys), + .with_identifier_key_groups(primary_keys), ); Ok(SubqueryAlias { input: Arc::new(plan), @@ -1414,11 +1414,11 @@ impl Window { let metadata = input.schema().metadata().clone(); // Update primary key for window - let mut primary_keys = input.schema().primary_keys().clone(); + let mut primary_keys = input.schema().identifier_key_groups().clone(); let n_input_fields = input.schema().fields().len(); let new_associated_fields: Vec = (n_input_fields..window_fields.len()).collect(); - for PrimaryKeyGroup { + for IdentifierKeyGroup { is_unique, associated_indices, .. @@ -1436,7 +1436,7 @@ impl Window { window_expr, schema: Arc::new( DFSchema::new_with_metadata(window_fields, metadata)? - .with_primary_keys(primary_keys), + .with_identifier_key_groups(primary_keys), ), }) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index d12b797be87fc..4da25e4a33878 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -449,7 +449,7 @@ pub fn expand_qualified_wildcard( let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? // we can use primary key as is, since it only stores indices - .with_primary_keys(schema.primary_keys().clone()); + .with_identifier_key_groups(schema.identifier_key_groups().clone()); let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, @@ -1022,7 +1022,9 @@ pub fn from_plan( let schema = Arc::new( DFSchema::new_with_metadata(fields, input.schema().metadata().clone())? // we can use existing primary keys as is - .with_primary_keys(input.schema().primary_keys().clone()), + .with_identifier_key_groups( + input.schema().identifier_key_groups().clone(), + ), ); Ok(LogicalPlan::Unnest(Unnest { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 4999b63f5f781..02dc20ac44c81 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -265,7 +265,7 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { DFSchemaRef::new( DFSchema::new_with_metadata(new_fields, schema.metadata().clone()) .unwrap() - .with_primary_keys(schema.primary_keys().clone()), + .with_identifier_key_groups(schema.identifier_key_groups().clone()), ) } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index eda63535666dd..dcebfe591c45f 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -20,7 +20,7 @@ use crate::utils::{ check_columns_satisfy_exprs, extract_aliases, rebase_expr, resolve_aliases_to_exprs, resolve_columns, resolve_positions_to_exprs, }; -use datafusion_common::{DataFusionError, PrimaryKeyGroup, Result}; +use datafusion_common::{DataFusionError, IdentifierKeyGroup, Result}; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, }; @@ -432,7 +432,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { let schema = input.schema(); - let primary_keys = schema.primary_keys(); + let primary_keys = schema.identifier_key_groups(); let field_names = schema .fields() @@ -440,8 +440,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|elem| elem.qualified_name()) .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); - for PrimaryKeyGroup { - primary_key_indices, + for IdentifierKeyGroup { + identifier_key_indices: primary_key_indices, associated_indices, .. } in primary_keys From 43d47f672bc518157d6569731e68095d63639a19 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 19 Jul 2023 10:51:30 +0300 Subject: [PATCH 20/37] Simplifications --- datafusion/common/src/dfschema.rs | 25 ++++++++++++--------- datafusion/core/src/execution/context.rs | 21 +++-------------- datafusion/expr/src/logical_plan/builder.rs | 6 ++++- 3 files changed, 23 insertions(+), 29 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 384b8354525c3..742a2d3effd0a 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -54,7 +54,7 @@ impl IdentifierKeyGroup { ) -> Self { Self { identifier_key_indices, - is_unique: true, + is_unique: false, associated_indices, } } @@ -847,6 +847,14 @@ impl SchemaExt for Schema { } } +/// Adds `offset` value to each entry inside `in_data`. +fn add_offset_to_vec>( + in_data: &[T], + offset: T, +) -> Vec { + in_data.iter().map(|&item| item + offset).collect() +} + /// Add offset value to identifier key indices and its associated indices /// for each identifier key group pub fn add_offset_to_identifier_key_groups( @@ -862,19 +870,16 @@ pub fn add_offset_to_identifier_key_groups( associated_indices, }| { IdentifierKeyGroup { - identifier_key_indices: identifier_key_indices - .iter() - .map(|pk_idx| pk_idx + offset) - .collect(), + identifier_key_indices: add_offset_to_vec( + identifier_key_indices, + offset, + ), is_unique: *is_unique, - associated_indices: associated_indices - .iter() - .map(|item| item + offset) - .collect(), + associated_indices: add_offset_to_vec(associated_indices, offset), } }, ) - .collect::() + .collect() } #[cfg(test)] diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 39353cef01b79..79bcc7f1a6bad 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -86,9 +86,7 @@ use crate::physical_planner::PhysicalPlanner; use crate::variable::{VarProvider, VarType}; use async_trait::async_trait; use chrono::{DateTime, Utc}; -use datafusion_common::{ - DFSchema, IdentifierKeyGroup, OwnedTableReference, SchemaReference, -}; +use datafusion_common::{OwnedTableReference, SchemaReference}; use datafusion_sql::{ parser::DFParser, planner::{ContextProvider, SqlToRel}, @@ -497,7 +495,7 @@ impl SessionContext { )), (_, _, Err(_)) => { let df_schema = input.schema(); - + // Get primary key indices in the schema let primary_keys = primary_key .iter() .map(|pk| { @@ -515,20 +513,7 @@ impl SessionContext { }) .collect::>>()?; - let fields = df_schema.fields().to_vec(); - - // all of the primary keys are associated with all of the fields (since it is source). - let association_indices = (0..fields.len()).collect::>(); - let primary_keys_with_associations = vec![IdentifierKeyGroup::new( - primary_keys.clone(), - association_indices, - )]; - let updated_schema = - DFSchema::new_with_metadata(fields, df_schema.metadata().clone())? - .with_identifier_key_groups(primary_keys_with_associations); - - let schema = Arc::new(updated_schema.into()); - + let schema = Arc::new(df_schema.as_ref().into()); let physical = DataFrame::new(self.state(), input); let batches: Vec<_> = physical.collect_partitioned().await?; diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index b2d980a52ffb6..c75ae287bd545 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -269,7 +269,11 @@ impl LogicalPlanBuilder { let n_field = schema.fields.len(); // All the field indices are associated, since it is source, let associated_indices = (0..n_field).collect::>(); - id_key_groups.push(IdentifierKeyGroup::new(pks.to_vec(), associated_indices)); + id_key_groups.push( + IdentifierKeyGroup::new(pks.to_vec(), associated_indices) + // Since primary keys are guaranteed to be unique, set `is_unique` flag to `true` + .with_is_unique(true), + ); } let projected_schema = projection From 170cb87d471423eb1946e1852b0a990c9542b3cf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 19 Jul 2023 11:00:00 +0300 Subject: [PATCH 21/37] Minor changes --- datafusion/core/src/physical_planner.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 20 ++++++++++---------- datafusion/optimizer/src/optimizer.rs | 4 ++-- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 64540fef7d42f..19ab04873164c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2203,7 +2203,7 @@ mod tests { dict_id: 0, \ dict_is_ordered: false, \ metadata: {} } }\ - ], metadata: {}, primary_keys: [] }, \ + ], metadata: {}, identifier_key_groups: [] }, \ ExecutionPlan schema: Schema { fields: [\ Field { \ name: \"b\", \ diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 0041318272c57..4ed419bce6201 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1254,14 +1254,14 @@ pub struct Projection { impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { - // update primary key of `input` according to projection exprs. - let primary_keys = project_identifier_keys(&expr, &input)?; + // update identifier key groups of `input` according to projection exprs. + let id_key_groups = project_identifier_keys(&expr, &input)?; let schema = Arc::new( DFSchema::new_with_metadata( exprlist_to_fields(&expr, &input)?, input.schema().metadata().clone(), )? - .with_identifier_key_groups(primary_keys), + .with_identifier_key_groups(id_key_groups), ); Self::try_new_with_schema(expr, input, schema) } @@ -1325,11 +1325,11 @@ impl SubqueryAlias { ) -> Result { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); - // Since schema is same, other than qualifier, we can use existing primary keys - let primary_keys = plan.schema().identifier_key_groups().clone(); + // Since schema is same, other than qualifier, we can use existing identifier key groups + let id_key_groups = plan.schema().identifier_key_groups().clone(); let schema = DFSchemaRef::new( DFSchema::try_from_qualified_schema(&alias, &schema)? - .with_identifier_key_groups(primary_keys), + .with_identifier_key_groups(id_key_groups), ); Ok(SubqueryAlias { input: Arc::new(plan), @@ -1413,8 +1413,8 @@ impl Window { .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); let metadata = input.schema().metadata().clone(); - // Update primary key for window - let mut primary_keys = input.schema().identifier_key_groups().clone(); + // Update identifier key groups for window + let mut id_key_groups = input.schema().identifier_key_groups().clone(); let n_input_fields = input.schema().fields().len(); let new_associated_fields: Vec = (n_input_fields..window_fields.len()).collect(); @@ -1422,7 +1422,7 @@ impl Window { is_unique, associated_indices, .. - } in primary_keys.iter_mut() + } in id_key_groups.iter_mut() { // if unique, extend associations such that they cover // new window expressions @@ -1436,7 +1436,7 @@ impl Window { window_expr, schema: Arc::new( DFSchema::new_with_metadata(window_fields, metadata)? - .with_identifier_key_groups(primary_keys), + .with_identifier_key_groups(id_key_groups), ), }) } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index fe159bf54b56f..dad1a28be7595 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -493,12 +493,12 @@ mod tests { assert_eq!( "get table_scan rule\ncaused by\n\ Internal error: Optimizer rule 'get table_scan rule' failed, due to generate a different schema, \ - original schema: DFSchema { fields: [], metadata: {}, primary_keys: [] }, \ + original schema: DFSchema { fields: [], metadata: {}, identifier_key_groups: [] }, \ 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: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \ - metadata: {}, primary_keys: [] }. \ + metadata: {}, identifier_key_groups: [] }. \ This was likely caused by a bug in DataFusion's code \ and we would welcome that you file an bug report in our issue tracker", err.to_string() From 3acef9641381a6ccf0d641addd852dd632489d4e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 19 Jul 2023 13:27:45 +0300 Subject: [PATCH 22/37] minor changes --- datafusion/sql/src/select.rs | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index dcebfe591c45f..cda93ffc1cf03 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -432,7 +432,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { let schema = input.schema(); - let primary_keys = schema.identifier_key_groups(); + let id_key_groups = schema.identifier_key_groups(); let field_names = schema .fields() @@ -441,26 +441,28 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); for IdentifierKeyGroup { - identifier_key_indices: primary_key_indices, + identifier_key_indices, associated_indices, .. - } in primary_keys + } in id_key_groups { - let field_pk_names = primary_key_indices + let id_key_names = identifier_key_indices .iter() - .map(|pk_idx| field_names[*pk_idx].as_str()) + .map(|id_key_idx| field_names[*id_key_idx].as_str()) .collect::>(); for group_by_expr in &group_by_exprs { let expr_name = format!("{}", group_by_expr); - // group by expression is primary key - if field_pk_names.contains(&expr_name.as_str()) { + // group by expression contains identifier key + // In these case, we can use associated fields after aggregation + // even if they are not part of group by expressions + if id_key_names.contains(&expr_name.as_str()) { let associated_field_names = associated_indices .iter() .map(|idx| field_names[*idx].as_str()) .collect::>(); // Expand group by exprs with select_exprs - // If one of the expressions inside group by is primary key and - // select expression is associated with that primary key. + // If one of the expressions inside group by is identifier key, and + // select expression is associated with that identifier key. for expr in select_exprs { let expr_name = format!("{}", expr); if !new_group_by_exprs.contains(expr) From 23e2938307b4d60420dbabd0aaa3adaca5897784 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 19 Jul 2023 15:27:13 +0300 Subject: [PATCH 23/37] Code style + improved comments --- datafusion/common/src/dfschema.rs | 51 +++++++++-------- .../src/datasource/default_table_source.rs | 8 ++- datafusion/core/src/datasource/provider.rs | 3 +- datafusion/core/src/execution/context.rs | 16 +++--- datafusion/core/src/physical_planner.rs | 1 + datafusion/expr/src/logical_plan/builder.rs | 57 ++++++++++--------- datafusion/expr/src/logical_plan/plan.rs | 2 +- datafusion/expr/src/table_source.rs | 3 +- datafusion/expr/src/utils.rs | 4 +- 9 files changed, 74 insertions(+), 71 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 742a2d3effd0a..1c690fd256a70 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -20,6 +20,7 @@ use std::collections::{HashMap, HashSet}; use std::convert::TryFrom; +use std::fmt::{Display, Formatter}; use std::hash::Hash; use std::sync::Arc; @@ -28,21 +29,25 @@ use crate::{field_not_found, Column, OwnedTableReference, TableReference}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; -use std::fmt::{Display, Formatter}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -/// Stores identifier keys and their associated indices -/// (where identifier key property holds, this may change during intermediate -/// schemas, such as join) +/// Stores identifier keys and their associated column indices. An identifier key +/// is a column whose value determines values of some other (dependent) columns. +/// These dependent columns are the "associated columns" of this identifier key. +/// If two rows have the same identifier key, associated columns in these rows +/// are necessarily the same. If the identifier key is unique, the set of +/// associated columns is equal to the entire schema and the identifier key can +/// serve as a primary key. Note that a primary key may "downgrade" into an +/// identifier key due to an operation such as a join, and this object is used to +/// track dependence relationships in such cases. #[derive(Debug, Clone, PartialEq, Eq)] pub struct IdentifierKeyGroup { pub identifier_key_indices: Vec, - /// Flag indicating whether identifiers are unique - /// if `true` it means that, these indices are primary keys. - /// if `false` it means that identifier still uniquely identifies its associated column - /// results. However, same results may occur replicated. + /// Flag indicating whether this identifier key is unique. + /// If true, these indices constitute a primary key. Otherwise, the + /// identifier key still uniquely determines its associated columns. pub is_unique: bool, pub associated_indices: Vec, } @@ -173,24 +178,20 @@ impl DFSchema { let mut metadata = self.metadata.clone(); fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); - let mut identifier_key_groups = self.identifier_key_groups.clone(); - identifier_key_groups.iter_mut().for_each( - |IdentifierKeyGroup { is_unique, .. }| { - *is_unique = false; - }, - ); - let mut other_identifier_key_groups = add_offset_to_identifier_key_groups( + let mut id_key_groups = self.identifier_key_groups.clone(); + for IdentifierKeyGroup { is_unique, .. } in id_key_groups.iter_mut() { + *is_unique = false; + } + let mut other_id_key_groups = add_offset_to_identifier_key_groups( schema.identifier_key_groups(), self.fields.len(), ); - other_identifier_key_groups.iter_mut().for_each( - |IdentifierKeyGroup { is_unique, .. }| { - *is_unique = false; - }, - ); - identifier_key_groups.extend(other_identifier_key_groups); + for IdentifierKeyGroup { is_unique, .. } in other_id_key_groups.iter_mut() { + *is_unique = false; + } + id_key_groups.extend(other_id_key_groups); Ok(Self::new_with_metadata(fields, metadata)? - .with_identifier_key_groups(identifier_key_groups)) + .with_identifier_key_groups(id_key_groups)) } /// Modify this schema by appending the fields from the supplied schema, ignoring any @@ -539,7 +540,7 @@ impl DFSchema { &self.metadata } - /// Get Identifier key groups + /// Get identifier key groups pub fn identifier_key_groups(&self) -> &IdentifierKeyGroups { &self.identifier_key_groups } @@ -855,8 +856,8 @@ fn add_offset_to_vec>( in_data.iter().map(|&item| item + offset).collect() } -/// Add offset value to identifier key indices and its associated indices -/// for each identifier key group +/// Add `offset` value to identifier key indices and its associated indices +/// for each identifier key group. pub fn add_offset_to_identifier_key_groups( identifier_key_groups: &IdentifierKeyGroups, offset: usize, diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 6a09ae2f9d3cc..2de6a43161290 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -17,12 +17,14 @@ //! Default TableSource implementation used in DataFusion physical plans +use std::any::Any; +use std::sync::Arc; + use crate::datasource::TableProvider; + use arrow::datatypes::SchemaRef; use datafusion_common::DataFusionError; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; -use std::any::Any; -use std::sync::Arc; /// DataFusion default table source, wrapping TableProvider /// @@ -52,7 +54,7 @@ impl TableSource for DefaultTableSource { self.table_provider.schema() } - /// Get a reference to the primary key indices + /// Get a reference to primary key indices, if a primary key exists. fn primary_keys(&self) -> Option<&[usize]> { self.table_provider.primary_keys() } diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index eb21d773aca1b..5b82ae9a3ef52 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -41,9 +41,8 @@ pub trait TableProvider: Sync + Send { /// Get a reference to the schema for this table fn schema(&self) -> SchemaRef; - /// Get a reference to the primary_key indices + /// Get a reference to primary key indices, if a primary key exists. fn primary_keys(&self) -> Option<&[usize]> { - // By default return None None } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 79bcc7f1a6bad..a8ebd19dd2b2a 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -495,21 +495,19 @@ impl SessionContext { )), (_, _, Err(_)) => { let df_schema = input.schema(); - // Get primary key indices in the schema + // Get primary key indices in the schema: let primary_keys = primary_key .iter() .map(|pk| { - if let Some(idx) = df_schema + df_schema .fields() .iter() .position(|item| item.qualified_name() == pk.flat_name()) - { - Ok(idx) - } else { - Err(DataFusionError::Execution( - "Primary Key doesn't exist".to_string(), - )) - } + .ok_or_else(|| { + DataFusionError::Execution( + "Primary key doesn't exist".to_string(), + ) + }) }) .collect::>>()?; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 19ab04873164c..eea215d0ea601 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2429,6 +2429,7 @@ mod tests { } struct ErrorExtensionPlanner {} + #[async_trait] impl ExtensionPlanner for ErrorExtensionPlanner { /// Create a physical plan for an extension node diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index c75ae287bd545..773deddd9e6bc 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -267,11 +267,11 @@ impl LogicalPlanBuilder { let mut id_key_groups = vec![]; if let Some(pks) = table_source.primary_keys() { let n_field = schema.fields.len(); - // All the field indices are associated, since it is source, + // All the field indices are associated since it is source. let associated_indices = (0..n_field).collect::>(); id_key_groups.push( IdentifierKeyGroup::new(pks.to_vec(), associated_indices) - // Since primary keys are guaranteed to be unique, set `is_unique` flag to `true` + // As primary keys are guaranteed to be unique, set `is_unique` flag to `true`. .with_is_unique(true), ); } @@ -1112,33 +1112,33 @@ pub fn build_join_schema( let mut left_id_key_groups = left.identifier_key_groups().clone(); - // After join, identifier key may no longer be unique - // (However, it still defines unique set of column values, just same values may be replicated). - // reset is_unique flag to false. - left_id_key_groups - .iter_mut() - .for_each(|pk_group| pk_group.is_unique = false); - right_id_key_groups - .iter_mut() - .for_each(|pk_group| pk_group.is_unique = false); + // After a join, a primary key may no longer be unique. However, as it still + // defines unique set of column values it is still an identifier key for its + // associated columns. + for group in left_id_key_groups.iter_mut() { + group.is_unique = false; + } + for group in right_id_key_groups.iter_mut() { + group.is_unique = false; + } let id_key_groups = match join_type { JoinType::Inner => { - // left then right + // Left then right: left_id_key_groups .into_iter() .chain(right_id_key_groups.into_iter()) .collect() } JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - // Only use the left side for the schema + // Only use the left side for the schema: left_id_key_groups } JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { - // Only use the right side for the schema + // Only use the right side for the schema: right_id_key_groups } JoinType::Full => { - // identifier keys are not preserved + // Identifier keys are not preserved. vec![] } }; @@ -1271,7 +1271,8 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result \[5, 8\] in the `df_schema`. -/// return value will be \[1\] -> \[1, 2\]. This means that 1st index of the `field_indices`(5) is identifier key, -/// and this identifier key is associated with columns at indices 1 and 2 (in the updated schema). -/// In the updated schema, fields at the indices \[2, 5, 8\] will be at \[0, 1, 2\]. +/// Update identifier key indices using the index mapping in `proj_indices`. +/// Assume that `proj_indices` is \[2, 5, 8\] and identifier key groups is +/// \[5\] -> \[5, 8\] in `df_schema`. Then, the return value will be \[1\] -> \[1, 2\]. +/// This means that the first index of the `proj_indices` (5) is an identifier key, +/// and this identifier key is associated with columns at indices 1 and 2 (in +/// the updated schema). In the updated schema, fields at indices \[2, 5, 8\] will +/// be at \[0, 1, 2\]. pub fn project_identifier_key_indices( id_key_groups: &IdentifierKeyGroups, proj_indices: &[usize], @@ -1304,10 +1307,10 @@ pub fn project_identifier_key_indices( let new_id_key_indices = update_elements_with_matching_indices(identifier_key_indices, proj_indices); let new_association_indices = if *is_unique { - // Associate with all of the fields in the schema + // Associate with all of the fields in the schema: (0..n_out).collect() } else { - // Update associations according to projection + // Update associations according to projection: update_elements_with_matching_indices(associated_indices, proj_indices) }; if !new_id_key_indices.is_empty() { @@ -1320,14 +1323,14 @@ pub fn project_identifier_key_indices( updated_id_key_groups } -/// This function projects identifier key groups of the -/// `input`, according to projection expressions `exprs` +/// This function projects identifier key groups of the plan `input` according +/// to projection expressions `exprs`. pub(crate) fn project_identifier_keys( exprs: &[Expr], input: &LogicalPlan, ) -> Result { let input_fields = input.schema().fields(); - // Calculate expression indices (if found) in the input schema. + // Calculate expression indices (if present) in the input schema. let proj_indices = exprs .iter() .filter_map(|expr| { @@ -1546,7 +1549,7 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new( DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? - // we can use existing identifier key groups, + // We can use existing identifier key groups: .with_identifier_key_groups(input_schema.identifier_key_groups().clone()), ); diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 4ed419bce6201..3820143f07a02 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -17,6 +17,7 @@ //! Logical plan types +use crate::builder::project_identifier_keys; use crate::expr::{Alias, Exists, InSubquery, Placeholder}; use crate::expr_rewriter::create_col_from_scalar_expr; use crate::expr_vec_fmt; @@ -44,7 +45,6 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; // backwards compatibility -use crate::builder::project_identifier_keys; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index a037e9c49bbca..0592c9ebf2edd 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -64,9 +64,8 @@ pub trait TableSource: Sync + Send { /// Get a reference to the schema for this table fn schema(&self) -> SchemaRef; - /// Get primary key index if any + /// Get primary key indices, if one exists. fn primary_keys(&self) -> Option<&[usize]> { - // By default return None. None } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 4da25e4a33878..2c7232e34c5c5 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -448,7 +448,7 @@ pub fn expand_qualified_wildcard( } let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? - // we can use primary key as is, since it only stores indices + // We can use the identifier key as is, since it only stores indices: .with_identifier_key_groups(schema.identifier_key_groups().clone()); let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, @@ -1021,7 +1021,7 @@ pub fn from_plan( let schema = Arc::new( DFSchema::new_with_metadata(fields, input.schema().metadata().clone())? - // we can use existing primary keys as is + // We can use the existing identifier keys as is: .with_identifier_key_groups( input.schema().identifier_key_groups().clone(), ), From b5cfc900b0d8bb79a26125a670d6f4c6d105ae46 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 20 Jul 2023 15:40:56 +0300 Subject: [PATCH 24/37] Add new tests, address TODO, Add comments --- .../sqllogictests/test_files/groupby.slt | 65 +++++++++++++++ datafusion/expr/src/logical_plan/builder.rs | 81 +++++++++++++++++++ datafusion/expr/src/logical_plan/plan.rs | 28 ++++--- 3 files changed, 163 insertions(+), 11 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index fef7bbf76904d..f45518c9c5fe6 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -3059,3 +3059,68 @@ FROM FROM sales_global_with_pk AS l JOIN sales_global_with_pk AS r ON l.sn >= r.sn)) GROUP BY r.sn + +# aggregate should propagate primary key successfully +query IPR +SELECT sn, ts, sum1 +FROM ( + SELECT ts, sn, SUM(amount) as sum1 + FROM sales_global_with_pk + GROUP BY sn) +GROUP BY sn +ORDER BY sn +---- +0 2022-01-01T06:00:00 30 +1 2022-01-01T08:00:00 50 +2 2022-01-01T11:30:00 75 +3 2022-01-02T12:00:00 200 +4 2022-01-03T10:00:00 100 + +# aggregate should be able to introduce identifier key +# (when group by contains single expression, group by expression +# becomes identifier key, after aggregation; since we are sure that +# it will consist of unique values.) +# please note that ts is not primary key, still +# we can use sum1, after outer aggregation because +# after inner aggregation, ts becomes identifier key. +query PR +SELECT ts, sum1 +FROM ( + SELECT ts, SUM(amount) as sum1 + FROM sales_global_with_pk + GROUP BY ts) +GROUP BY ts +ORDER BY ts +---- +2022-01-01T06:00:00 30 +2022-01-01T08:00:00 50 +2022-01-01T11:30:00 75 +2022-01-02T12:00:00 200 +2022-01-03T10:00:00 100 + +# aggregate should reset its identifier keys +# if it is guaranteed that, after aggregation +# group by expressions will be unique. +query IRI +SELECT * +FROM ( + SELECT *, ROW_NUMBER() OVER(ORDER BY l.sn) AS rn1 + FROM ( + SELECT l.sn, SUM(l.amount) + FROM ( + SELECT l.sn, l.amount, SUM(l.amount) as sum1 + FROM + (SELECT * + FROM sales_global_with_pk AS l + JOIN sales_global_with_pk AS r ON l.sn >= r.sn) + GROUP BY l.sn) + GROUP BY l.sn) + ) +GROUP BY l.sn +ORDER BY l.sn +---- +0 30 1 +1 50 2 +2 75 3 +3 200 4 +4 100 5 diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 773deddd9e6bc..2662c006412ec 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1352,6 +1352,87 @@ pub(crate) fn project_identifier_keys( )) } +/// Check whether expression contains `Expr::GroupingSet` +fn contains_grouping_set(group_expr: &[Expr]) -> bool { + group_expr + .iter() + .any(|expr| matches!(expr, Expr::GroupingSet(_))) +} + +/// Calculate identifier key groups for aggregate expressions +pub(crate) fn aggregate_identifier_keys( + // Expressions inside GROUP BY + group_expr: &[Expr], + // input of aggregate + input: &LogicalPlan, + // If is_unique flag of identifier key group is true. Association covers whole table. `n_out` + // stores schema field length to be able to correctly associate with whole table. + n_out: usize, +) -> Result { + // Calculate identifier keys for the aggregate + let mut identifier_key_groups = vec![]; + // We can do analysis for how to propagate identifier key through aggregate + // when group by doesn't contain grouping set expression. + // When group by contains grouping set expression; identifier key will be empty. + // because we cannot guarantee group by expression results will be unique. + if !contains_grouping_set(group_expr) { + // association covers whole table. + let associated_indices = (0..n_out).collect::>(); + // Get id key groups of input + let id_key_groups = input.schema().identifier_key_groups(); + let fields = input.schema().fields(); + for id_key_group in id_key_groups { + let mut new_identifier_key_indices = HashSet::new(); + let id_key_indices = &id_key_group.identifier_key_indices; + let id_key_field_names = id_key_indices + .iter() + .map(|&idx| fields[idx].qualified_name()) + .collect::>(); + for (idx, group_by_expr) in group_expr.iter().enumerate() { + // when one of input identifier key matches with group by expression + // Add index of group by expression as identifier key + if id_key_field_names.contains(&group_by_expr.display_name()?) { + new_identifier_key_indices.insert(idx); + } + } + if !new_identifier_key_indices.is_empty() { + identifier_key_groups.push( + IdentifierKeyGroup::new( + new_identifier_key_indices.into_iter().collect(), + associated_indices.clone(), + ) // input uniqueness keeps same, when group by matches with + // input identifier keys + .with_is_unique(id_key_group.is_unique), + ); + } + } + // Guaranteed to be unique after aggregation, since it has single expression such GROUP BY a + if group_expr.len() == 1 { + // If identifier key groups contain 0, delete this identifier key from identifier key groups + // because it will be added anyway with is_unique flag true. + if let Some(idx) = identifier_key_groups + .iter() + .position(|item| item.identifier_key_indices.contains(&0)) + { + identifier_key_groups[idx] + .identifier_key_indices + .retain(|&x| x != idx); + if identifier_key_groups[idx].identifier_key_indices.is_empty() { + // identifier key is empty (e.g `identifier_key_indices` was vec![0], deleting 0 + // emptied `identifier_key_indices` vector.), delete this identifier group + identifier_key_groups.remove(idx); + } + } + // Add new identifier group with, that is associated with whole table. Where it is guaranteed + // to be unique. + identifier_key_groups.push( + IdentifierKeyGroup::new(vec![0], associated_indices).with_is_unique(true), + ); + } + } + Ok(identifier_key_groups) +} + /// Create Projection /// # Errors /// This function errors under any of the following conditions: diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 3820143f07a02..33b2a28f44346 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -17,7 +17,7 @@ //! Logical plan types -use crate::builder::project_identifier_keys; +use crate::builder::{aggregate_identifier_keys, project_identifier_keys}; use crate::expr::{Alias, Exists, InSubquery, Placeholder}; use crate::expr_rewriter::create_col_from_scalar_expr; use crate::expr_vec_fmt; @@ -1254,15 +1254,10 @@ pub struct Projection { impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { - // update identifier key groups of `input` according to projection exprs. - let id_key_groups = project_identifier_keys(&expr, &input)?; - let schema = Arc::new( - DFSchema::new_with_metadata( - exprlist_to_fields(&expr, &input)?, - input.schema().metadata().clone(), - )? - .with_identifier_key_groups(id_key_groups), - ); + 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) } @@ -1275,6 +1270,10 @@ impl Projection { if expr.len() != schema.fields().len() { return Err(DataFusionError::Plan(format!("Projection has mismatch between number of expressions ({}) and number of fields in schema ({})", expr.len(), schema.fields().len()))); } + // update identifier key groups of `input` according to projection exprs. + let id_key_groups = project_identifier_keys(&expr, &input)?; + let schema = schema.as_ref().clone(); + let schema = Arc::new(schema.with_identifier_key_groups(id_key_groups)); Ok(Self { expr, input, @@ -1624,11 +1623,12 @@ impl Aggregate { let group_expr = enumerate_grouping_sets(group_expr)?; let grouping_expr: Vec = grouping_set_to_exprlist(group_expr.as_slice())?; let all_expr = grouping_expr.iter().chain(aggr_expr.iter()); - // TODO: Add handling for aggregate primary key + let schema = DFSchema::new_with_metadata( exprlist_to_fields(all_expr, &input)?, input.schema().metadata().clone(), )?; + Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } @@ -1657,6 +1657,12 @@ impl Aggregate { schema.fields().len() ))); } + + let identifier_key_groups = + aggregate_identifier_keys(&group_expr, &input, schema.fields().len())?; + let new_schema = schema.as_ref().clone(); + let schema = + Arc::new(new_schema.with_identifier_key_groups(identifier_key_groups)); Ok(Self { input, group_expr, From fd000a0b16a15e68cea205a01d986c1ca8bdd578 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 20 Jul 2023 16:57:16 +0300 Subject: [PATCH 25/37] Final review --- datafusion/expr/src/logical_plan/builder.rs | 51 +++++++++++---------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 2662c006412ec..db8d81d86d0fb 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1352,33 +1352,32 @@ pub(crate) fn project_identifier_keys( )) } -/// Check whether expression contains `Expr::GroupingSet` +/// Checks whether any expression in `group_expr` contains `Expr::GroupingSet`. fn contains_grouping_set(group_expr: &[Expr]) -> bool { group_expr .iter() .any(|expr| matches!(expr, Expr::GroupingSet(_))) } -/// Calculate identifier key groups for aggregate expressions +/// Calculates identifier key groups for aggregate expressions. pub(crate) fn aggregate_identifier_keys( - // Expressions inside GROUP BY + // Expressions in the GROUP BY clause: group_expr: &[Expr], - // input of aggregate + // Input plan of the aggregate: input: &LogicalPlan, - // If is_unique flag of identifier key group is true. Association covers whole table. `n_out` - // stores schema field length to be able to correctly associate with whole table. + // Schema field length: n_out: usize, ) -> Result { - // Calculate identifier keys for the aggregate let mut identifier_key_groups = vec![]; - // We can do analysis for how to propagate identifier key through aggregate - // when group by doesn't contain grouping set expression. - // When group by contains grouping set expression; identifier key will be empty. - // because we cannot guarantee group by expression results will be unique. + // We can do a case analysis on how to propagate identifier keys based on + // whether the GROUP BY in question contains a grouping set expression: + // - If so, the identifier key will be empty because we cannot guarantee + // that GROUP BY expression results will be unique. + // - Otherwise, it may be possible to propagate identifier keys. if !contains_grouping_set(group_expr) { - // association covers whole table. + // Association covers whole table: let associated_indices = (0..n_out).collect::>(); - // Get id key groups of input + // Get identifier key groups of the input: let id_key_groups = input.schema().identifier_key_groups(); let fields = input.schema().fields(); for id_key_group in id_key_groups { @@ -1389,8 +1388,9 @@ pub(crate) fn aggregate_identifier_keys( .map(|&idx| fields[idx].qualified_name()) .collect::>(); for (idx, group_by_expr) in group_expr.iter().enumerate() { - // when one of input identifier key matches with group by expression - // Add index of group by expression as identifier key + // When one of the input identifier keys matches with the GROUP + // BY expression, add the index of the GROUP BY expression as + // an identifier key. if id_key_field_names.contains(&group_by_expr.display_name()?) { new_identifier_key_indices.insert(idx); } @@ -1400,16 +1400,18 @@ pub(crate) fn aggregate_identifier_keys( IdentifierKeyGroup::new( new_identifier_key_indices.into_iter().collect(), associated_indices.clone(), - ) // input uniqueness keeps same, when group by matches with - // input identifier keys + ) + // input uniqueness stays the same when GROUP BY matches with input identifier keys .with_is_unique(id_key_group.is_unique), ); } } - // Guaranteed to be unique after aggregation, since it has single expression such GROUP BY a + // Guaranteed to be unique after aggregation, since it has a single + // expression (e.g. GROUP BY a): if group_expr.len() == 1 { - // If identifier key groups contain 0, delete this identifier key from identifier key groups - // because it will be added anyway with is_unique flag true. + // If identifier key groups contain 0, delete this identifier key from + // identifier key groups because it will be added anyway with is_unique + // flag set. if let Some(idx) = identifier_key_groups .iter() .position(|item| item.identifier_key_indices.contains(&0)) @@ -1418,13 +1420,14 @@ pub(crate) fn aggregate_identifier_keys( .identifier_key_indices .retain(|&x| x != idx); if identifier_key_groups[idx].identifier_key_indices.is_empty() { - // identifier key is empty (e.g `identifier_key_indices` was vec![0], deleting 0 - // emptied `identifier_key_indices` vector.), delete this identifier group + // Identifier key is empty, delete this identifier group. + // Example: `identifier_key_indices` was equal to vec![0], + // and deleting 0 emptied it. identifier_key_groups.remove(idx); } } - // Add new identifier group with, that is associated with whole table. Where it is guaranteed - // to be unique. + // Add a new identifier group that is associated with whole table. + // Here, it is guaranteed to be unique. identifier_key_groups.push( IdentifierKeyGroup::new(vec![0], associated_indices).with_is_unique(true), ); From ad6714c3c5e1e6f025c26cba533e771ffa8e041d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 21 Jul 2023 15:52:21 +0300 Subject: [PATCH 26/37] Address reviews, change design to encapsulate functional dependency --- datafusion/common/src/dfschema.rs | 410 ++++++++++++++---- datafusion/common/src/lib.rs | 4 +- datafusion/core/src/physical_planner.rs | 2 +- .../sqllogictests/test_files/groupby.slt | 11 +- datafusion/expr/src/logical_plan/builder.rs | 247 +++-------- datafusion/expr/src/logical_plan/plan.rs | 53 +-- datafusion/expr/src/utils.rs | 10 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- datafusion/optimizer/src/optimizer.rs | 4 +- datafusion/sql/src/select.rs | 59 +-- 10 files changed, 452 insertions(+), 350 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 1c690fd256a70..3b6b60678e1a8 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -25,7 +25,7 @@ use std::hash::Hash; use std::sync::Arc; use crate::error::{unqualified_field_not_found, DataFusionError, Result, SchemaError}; -use crate::{field_not_found, Column, OwnedTableReference, TableReference}; +use crate::{field_not_found, Column, JoinType, OwnedTableReference, TableReference}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; @@ -33,44 +33,222 @@ use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -/// Stores identifier keys and their associated column indices. An identifier key +/// Stores functional dependence in the schema. +/// Functional dependence defines relationship between determinants and dependents. +/// The value of determinant keys uniquely determines the value of dependents. A determinant key /// is a column whose value determines values of some other (dependent) columns. -/// These dependent columns are the "associated columns" of this identifier key. -/// If two rows have the same identifier key, associated columns in these rows -/// are necessarily the same. If the identifier key is unique, the set of -/// associated columns is equal to the entire schema and the identifier key can +/// If two rows have the same determinant key, dependent columns in these rows +/// are necessarily the same. If the determinant key is unique, the set of +/// dependent columns is equal to the entire schema and the determinant key can /// serve as a primary key. Note that a primary key may "downgrade" into an -/// identifier key due to an operation such as a join, and this object is used to +/// determinant key due to an operation such as a join, and this object is used to /// track dependence relationships in such cases. +/// For more information about functional dependency see +/// Indices of determinants are stored in `source_indices`, and indices of +/// dependents are stored in `target_indices` +/// #[derive(Debug, Clone, PartialEq, Eq)] -pub struct IdentifierKeyGroup { - pub identifier_key_indices: Vec, - /// Flag indicating whether this identifier key is unique. - /// If true, these indices constitute a primary key. Otherwise, the - /// identifier key still uniquely determines its associated columns. - pub is_unique: bool, - pub associated_indices: Vec, +pub struct FunctionalDependence { + pub source_indices: Vec, + pub mode: Dependency, + pub target_indices: Vec, } -impl IdentifierKeyGroup { - pub fn new( - identifier_key_indices: Vec, - associated_indices: Vec, - ) -> Self { +/// Describes functional dependency mode +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum Dependency { + Single, // unique constraint + Multi, // non unique (e.g. multipart primary key) +} + +impl FunctionalDependence { + pub fn new(source_indices: Vec, target_indices: Vec) -> Self { Self { - identifier_key_indices, - is_unique: false, - associated_indices, + source_indices, + // Dependency::Multi is the most relaxed mode + // hence, if not specified this is the safest approach + mode: Dependency::Multi, + target_indices, } } - pub fn with_is_unique(mut self, is_unique: bool) -> Self { - self.is_unique = is_unique; + pub fn with_mode(mut self, mode: Dependency) -> Self { + self.mode = mode; self } } -pub type IdentifierKeyGroups = Vec; +/// Encapsulate functional dependencies +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct FunctionalDependencies { + inner: Vec, +} + +impl FunctionalDependencies { + /// Create an empty `FunctionalDependencies` + pub fn empty() -> Self { + Self { inner: vec![] } + } + + /// Create a new `FunctionalDependencies` from vector of `FunctionalDependence`s. + pub fn new(functional_dependencies: Vec) -> Self { + Self { + inner: functional_dependencies, + } + } + + pub fn new_from_primary_keys( + primary_key_indices: Option<&[usize]>, + n_field: usize, + ) -> Self { + if let Some(primary_key_indices) = primary_key_indices { + // All the field indices are associated, with whole table. since it is primary key. + let target_indices = (0..n_field).collect::>(); + let dependencies = vec![FunctionalDependence::new( + primary_key_indices.to_vec(), + target_indices, + ) + // As primary keys are guaranteed to be unique, set `mode` of functional dependency to `Dependency::Single`. + .with_mode(Dependency::Single)]; + Self::new(dependencies) + } else { + Self::empty() + } + } + + pub fn with_dependency(mut self, mode: Dependency) -> Self { + self.inner.iter_mut().for_each(|item| item.mode = mode); + self + } + + pub fn extend(&mut self, other: FunctionalDependencies) { + self.inner.extend(other.inner); + } + + /// Add `offset` value to `source_indices` and `target_indices` + /// for each functional dependency + pub fn add_offset(&mut self, offset: usize) { + self.inner.iter_mut().for_each( + |FunctionalDependence { + source_indices, + target_indices, + .. + }| { + *source_indices = add_offset_to_vec(source_indices, offset); + *target_indices = add_offset_to_vec(target_indices, offset); + }, + ) + } + + /// Update `source_indices`, and `target_indices` of each `FunctionalDependence` + /// using the index mapping in `proj_indices`. + /// Assume that `proj_indices` is \[2, 5, 8\] and functional dependence is + /// \[5\] (`source_indices`) -> \[5, 8\] (`target_indices`) in `df_schema`. + /// Then, the return value will be \[1\] -> \[1, 2\]. + /// This means that the first index of the `proj_indices` (5) is a determinant key + /// (stored at `source_indices`), and its key is dependent keys are at indices 1 and 2 (in + /// the updated schema) (stored at `target_indices`). + /// Please note that: In the updated schema, fields at indices \[2, 5, 8\] will be at \[0, 1, 2\]. + pub fn project_functional_dependencies( + &self, + proj_indices: &[usize], + // If dependency mode is Single. Association covers whole table. `n_out` + // stores schema field length to be able to correctly associate with whole table. + n_out: usize, + ) -> FunctionalDependencies { + let mut projected_func_dependencies = vec![]; + for FunctionalDependence { + source_indices, + mode, + target_indices, + } in &self.inner + { + let new_source_indices = + update_elements_with_matching_indices(source_indices, proj_indices); + let new_target_indices = if *mode == Dependency::Single { + // Associate with all of the fields in the schema: + (0..n_out).collect() + } else { + // Update associations according to projection: + update_elements_with_matching_indices(target_indices, proj_indices) + }; + if !new_source_indices.is_empty() { + let new_func_dependence = + FunctionalDependence::new(new_source_indices, new_target_indices) + .with_mode(*mode); + projected_func_dependencies.push(new_func_dependence); + } + } + FunctionalDependencies::new(projected_func_dependencies) + } + + pub fn join( + &self, + other: &FunctionalDependencies, + join_type: &JoinType, + left_cols_len: usize, + ) -> FunctionalDependencies { + let mut right_func_dependencies = other.clone(); + right_func_dependencies.add_offset(left_cols_len); + + let mut left_func_dependencies = self.clone(); + + // After a join, a primary key may no longer be unique. However, as it still + // defines unique set of column values for its target columns. + // Reset mode to `Dependency::Multi` + left_func_dependencies = + left_func_dependencies.with_dependency(Dependency::Multi); + right_func_dependencies = + right_func_dependencies.with_dependency(Dependency::Multi); + match join_type { + JoinType::Inner => { + // Left then right: + left_func_dependencies.extend(right_func_dependencies); + left_func_dependencies + } + JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + // Only use the left side for the schema: + left_func_dependencies + } + JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { + // Only use the right side for the schema: + right_func_dependencies + } + JoinType::Full => { + // All of the functional dependencies are lost. + FunctionalDependencies::empty() + } + } + } + + pub fn extend_target_indices(&mut self, n_out: usize) { + // if unique cover whole table + let new_target_indices = (0..n_out).collect::>(); + self.inner.iter_mut().for_each( + |FunctionalDependence { + mode, + target_indices, + .. + }| { + if *mode == Dependency::Single { + *target_indices = new_target_indices.clone() + } + }, + ) + } +} + +// Update entries inside the `entries` vector with their corresponding index +// inside the `proj_indices` vector. +fn update_elements_with_matching_indices( + entries: &[usize], + proj_indices: &[usize], +) -> Vec { + entries + .iter() + .filter_map(|val| proj_indices.iter().position(|proj_idx| proj_idx == val)) + .collect() +} /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] @@ -79,8 +257,8 @@ pub struct DFSchema { fields: Vec, /// Additional metadata in form of key value pairs metadata: HashMap, - /// Stores distinct identifier key groups - identifier_key_groups: IdentifierKeyGroups, + /// Stores functional dependencies in the schema. + functional_dependencies: FunctionalDependencies, } impl DFSchema { @@ -89,7 +267,7 @@ impl DFSchema { Self { fields: vec![], metadata: HashMap::new(), - identifier_key_groups: vec![], + functional_dependencies: FunctionalDependencies::empty(), } } @@ -142,7 +320,7 @@ impl DFSchema { Ok(Self { fields, metadata, - identifier_key_groups: vec![], + functional_dependencies: FunctionalDependencies::empty(), }) } @@ -162,12 +340,130 @@ impl DFSchema { ) } - /// Assign identifier key groups - pub fn with_identifier_key_groups( + pub fn join_functional_dependencies( + &self, + other: &DFSchema, + ) -> FunctionalDependencies { + let mut functional_dependencies = self.functional_dependencies.clone(); + // Make mode multi, since after join same rows may get replicated. + functional_dependencies = + functional_dependencies.with_dependency(Dependency::Multi); + let mut other_functional_dependencies = other.functional_dependencies().clone(); + other_functional_dependencies.add_offset(self.fields.len()); + other_functional_dependencies = + other_functional_dependencies.with_dependency(Dependency::Multi); + + functional_dependencies.extend(other_functional_dependencies); + functional_dependencies + } + + pub fn aggregate_functional_dependencies( + &self, + group_by_expr_names: &[String], + n_out: usize, + ) -> FunctionalDependencies { + let mut aggregate_func_dependencies = vec![]; + // Association covers whole table: + let target_indices = (0..n_out).collect::>(); + // Get functional dependencies of the schema. + let func_dependencies = self.functional_dependencies(); + let fields = self.fields(); + for FunctionalDependence { + source_indices, + mode, + .. + } in &func_dependencies.inner + { + // keep source_indices in HashSet to prevent duplicate entries + let mut new_source_indices = HashSet::new(); + let source_field_names = source_indices + .iter() + .map(|&idx| 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); + } + } + if !new_source_indices.is_empty() { + aggregate_func_dependencies.push( + FunctionalDependence::new( + new_source_indices.into_iter().collect(), + target_indices.clone(), + ) + // input uniqueness stays the same when GROUP BY matches with input functional dependence determinants + .with_mode(*mode), + ); + } + } + // Guaranteed to be unique after aggregation, since it has a single + // expression (e.g. GROUP BY a): + if group_by_expr_names.len() == 1 { + // If `source_indices` contain 0, delete this functional group + // because it will be added anyway with mode `Dependency::Single` + // flag set. + if let Some(idx) = aggregate_func_dependencies + .iter() + .position(|item| item.source_indices.contains(&0)) + { + aggregate_func_dependencies[idx] + .source_indices + .retain(|&x| x != idx); + if aggregate_func_dependencies[idx].source_indices.is_empty() { + // `source_indices` is empty, delete this identifier group. + // Example: `source_indices` was equal to vec![0], + // and deleting 0 emptied it. + aggregate_func_dependencies.remove(idx); + } + } + // Add a new functional dependence that is associated with whole table. + aggregate_func_dependencies.push( + FunctionalDependence::new(vec![0], target_indices) + .with_mode(Dependency::Single), + ); + } + FunctionalDependencies::new(aggregate_func_dependencies) + } + + pub fn get_target_functional_dependencies( + &self, + expr_name: &str, + ) -> Option<&[usize]> { + let func_dependencies = self.functional_dependencies(); + let field_names = self + .fields() + .iter() + .map(|elem| elem.qualified_name()) + .collect::>(); + for FunctionalDependence { + source_indices, + target_indices, + .. + } in &func_dependencies.inner + { + let source_key_names = source_indices + .iter() + .map(|id_key_idx| field_names[*id_key_idx].as_str()) + .collect::>(); + // group by expression contains a determinant from a functional dependency + // In this case, we can use associated fields after aggregation + // even if they are not part of group by expressions + if source_key_names.contains(&expr_name) { + return Some(target_indices); + } + } + None + } + + /// Assign functional dependencies + pub fn with_functional_dependencies( mut self, - identifier_key_groups: IdentifierKeyGroups, + functional_dependencies: FunctionalDependencies, ) -> Self { - self.identifier_key_groups = identifier_key_groups; + self.functional_dependencies = functional_dependencies; self } @@ -178,20 +474,9 @@ impl DFSchema { let mut metadata = self.metadata.clone(); fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); - let mut id_key_groups = self.identifier_key_groups.clone(); - for IdentifierKeyGroup { is_unique, .. } in id_key_groups.iter_mut() { - *is_unique = false; - } - let mut other_id_key_groups = add_offset_to_identifier_key_groups( - schema.identifier_key_groups(), - self.fields.len(), - ); - for IdentifierKeyGroup { is_unique, .. } in other_id_key_groups.iter_mut() { - *is_unique = false; - } - id_key_groups.extend(other_id_key_groups); + let join_functional_dependencies = self.join_functional_dependencies(schema); Ok(Self::new_with_metadata(fields, metadata)? - .with_identifier_key_groups(id_key_groups)) + .with_functional_dependencies(join_functional_dependencies)) } /// Modify this schema by appending the fields from the supplied schema, ignoring any @@ -540,9 +825,9 @@ impl DFSchema { &self.metadata } - /// Get identifier key groups - pub fn identifier_key_groups(&self) -> &IdentifierKeyGroups { - &self.identifier_key_groups + /// Get functional dependencies + pub fn functional_dependencies(&self) -> &FunctionalDependencies { + &self.functional_dependencies } } @@ -856,33 +1141,6 @@ fn add_offset_to_vec>( in_data.iter().map(|&item| item + offset).collect() } -/// Add `offset` value to identifier key indices and its associated indices -/// for each identifier key group. -pub fn add_offset_to_identifier_key_groups( - identifier_key_groups: &IdentifierKeyGroups, - offset: usize, -) -> IdentifierKeyGroups { - identifier_key_groups - .iter() - .map( - |IdentifierKeyGroup { - identifier_key_indices, - is_unique, - associated_indices, - }| { - IdentifierKeyGroup { - identifier_key_indices: add_offset_to_vec( - identifier_key_indices, - offset, - ), - is_unique: *is_unique, - associated_indices: add_offset_to_vec(associated_indices, offset), - } - }, - ) - .collect() -} - #[cfg(test)] mod tests { use crate::assert_contains; diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index a422577d4c711..18ece01a60b69 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -37,8 +37,8 @@ pub mod utils; pub use column::Column; pub use dfschema::{ - add_offset_to_identifier_key_groups, DFField, DFSchema, DFSchemaRef, ExprSchema, - IdentifierKeyGroup, IdentifierKeyGroups, SchemaExt, ToDFSchema, + DFField, DFSchema, DFSchemaRef, Dependency, ExprSchema, FunctionalDependence, + FunctionalDependencies, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index eea215d0ea601..46a8691176216 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2203,7 +2203,7 @@ mod tests { dict_id: 0, \ dict_is_ordered: false, \ metadata: {} } }\ - ], metadata: {}, identifier_key_groups: [] }, \ + ], metadata: {}, functional_dependencies: FunctionalDependencies { inner: [] } }, \ ExecutionPlan schema: Schema { fields: [\ Field { \ name: \"b\", \ diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index f45518c9c5fe6..7c6328f7883a1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -3076,13 +3076,14 @@ ORDER BY sn 3 2022-01-02T12:00:00 200 4 2022-01-03T10:00:00 100 -# aggregate should be able to introduce identifier key +# aggregate should be able to introduce functional dependence # (when group by contains single expression, group by expression -# becomes identifier key, after aggregation; since we are sure that +# becomes determinant, after aggregation; since we are sure that # it will consist of unique values.) # please note that ts is not primary key, still # we can use sum1, after outer aggregation because -# after inner aggregation, ts becomes identifier key. +# after inner aggregation, ts becomes determinant +# of functional dependence. query PR SELECT ts, sum1 FROM ( @@ -3098,8 +3099,8 @@ ORDER BY ts 2022-01-02T12:00:00 200 2022-01-03T10:00:00 100 -# aggregate should reset its identifier keys -# if it is guaranteed that, after aggregation +# aggregate should update its functional dependence +# mode, if it is guaranteed that, after aggregation # group by expressions will be unique. query IRI SELECT * diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index db8d81d86d0fb..e6348f9f04639 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -41,9 +41,9 @@ use crate::{ }; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ - add_offset_to_identifier_key_groups, display::ToStringifiedPlan, Column, DFField, - DFSchema, DFSchemaRef, DataFusionError, IdentifierKeyGroup, IdentifierKeyGroups, - OwnedTableReference, Result, ScalarValue, TableReference, ToDFSchema, + display::ToStringifiedPlan, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, + FunctionalDependencies, OwnedTableReference, Result, ScalarValue, TableReference, + ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -264,23 +264,17 @@ impl LogicalPlanBuilder { } let schema = table_source.schema(); - let mut id_key_groups = vec![]; - if let Some(pks) = table_source.primary_keys() { - let n_field = schema.fields.len(); - // All the field indices are associated since it is source. - let associated_indices = (0..n_field).collect::>(); - id_key_groups.push( - IdentifierKeyGroup::new(pks.to_vec(), associated_indices) - // As primary keys are guaranteed to be unique, set `is_unique` flag to `true`. - .with_is_unique(true), - ); - } + let n_field = schema.fields.len(); + let func_dependencies = FunctionalDependencies::new_from_primary_keys( + table_source.primary_keys(), + n_field, + ); let projected_schema = projection .as_ref() .map(|p| { let projected_id_key_groups = - project_identifier_key_indices(&id_key_groups, p, p.len()); + func_dependencies.project_functional_dependencies(p, p.len()); DFSchema::new_with_metadata( p.iter() .map(|i| { @@ -293,12 +287,13 @@ impl LogicalPlanBuilder { schema.metadata().clone(), ) .map(|df_schema| { - df_schema.with_identifier_key_groups(projected_id_key_groups) + df_schema.with_functional_dependencies(projected_id_key_groups) }) }) .unwrap_or_else(|| { - DFSchema::try_from_qualified_schema(table_name.clone(), &schema) - .map(|df_schema| df_schema.with_identifier_key_groups(id_key_groups)) + 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 { @@ -1104,49 +1099,15 @@ pub fn build_join_schema( right_fields.clone() } }; - - let mut right_id_key_groups = add_offset_to_identifier_key_groups( - right.identifier_key_groups(), + let func_dependencies = left.functional_dependencies().join( + right.functional_dependencies(), + join_type, left_fields.len(), ); - - let mut left_id_key_groups = left.identifier_key_groups().clone(); - - // After a join, a primary key may no longer be unique. However, as it still - // defines unique set of column values it is still an identifier key for its - // associated columns. - for group in left_id_key_groups.iter_mut() { - group.is_unique = false; - } - for group in right_id_key_groups.iter_mut() { - group.is_unique = false; - } - let id_key_groups = match join_type { - JoinType::Inner => { - // Left then right: - left_id_key_groups - .into_iter() - .chain(right_id_key_groups.into_iter()) - .collect() - } - JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - // Only use the left side for the schema: - left_id_key_groups - } - JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { - // Only use the right side for the schema: - right_id_key_groups - } - JoinType::Full => { - // Identifier keys are not preserved. - vec![] - } - }; - let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); Ok(DFSchema::new_with_metadata(fields, metadata)? - .with_identifier_key_groups(id_key_groups)) + .with_functional_dependencies(func_dependencies)) } /// Errors if one or more expressions have equal names. @@ -1271,64 +1232,12 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Vec { - entries - .iter() - .filter_map(|val| proj_indices.iter().position(|proj_idx| proj_idx == val)) - .collect() -} - -/// Update identifier key indices using the index mapping in `proj_indices`. -/// Assume that `proj_indices` is \[2, 5, 8\] and identifier key groups is -/// \[5\] -> \[5, 8\] in `df_schema`. Then, the return value will be \[1\] -> \[1, 2\]. -/// This means that the first index of the `proj_indices` (5) is an identifier key, -/// and this identifier key is associated with columns at indices 1 and 2 (in -/// the updated schema). In the updated schema, fields at indices \[2, 5, 8\] will -/// be at \[0, 1, 2\]. -pub fn project_identifier_key_indices( - id_key_groups: &IdentifierKeyGroups, - proj_indices: &[usize], - // If is_unique flag of identifier key group is true. Association covers whole table. `n_out` - // stores schema field length to be able to correctly associate with whole table. - n_out: usize, -) -> IdentifierKeyGroups { - let mut updated_id_key_groups = vec![]; - for IdentifierKeyGroup { - identifier_key_indices, - is_unique, - associated_indices, - } in id_key_groups - { - let new_id_key_indices = - update_elements_with_matching_indices(identifier_key_indices, proj_indices); - let new_association_indices = if *is_unique { - // Associate with all of the fields in the schema: - (0..n_out).collect() - } else { - // Update associations according to projection: - update_elements_with_matching_indices(associated_indices, proj_indices) - }; - if !new_id_key_indices.is_empty() { - let new_id_key_group = - IdentifierKeyGroup::new(new_id_key_indices, new_association_indices) - .with_is_unique(*is_unique); - updated_id_key_groups.push(new_id_key_group); - } - } - updated_id_key_groups -} - -/// This function projects identifier key groups of the plan `input` according +/// This function projects functional dependencies of `input` according /// to projection expressions `exprs`. -pub(crate) fn project_identifier_keys( +pub(crate) fn project_functional_dependencies( exprs: &[Expr], input: &LogicalPlan, -) -> Result { +) -> Result { let input_fields = input.schema().fields(); // Calculate expression indices (if present) in the input schema. let proj_indices = exprs @@ -1345,11 +1254,9 @@ pub(crate) fn project_identifier_keys( .position(|item| item.qualified_name() == expr_name) }) .collect::>(); - Ok(project_identifier_key_indices( - input.schema().identifier_key_groups(), - &proj_indices, - exprs.len(), - )) + let input_func_dependencies = input.schema().functional_dependencies(); + Ok(input_func_dependencies + .project_functional_dependencies(&proj_indices, exprs.len())) } /// Checks whether any expression in `group_expr` contains `Expr::GroupingSet`. @@ -1359,81 +1266,32 @@ fn contains_grouping_set(group_expr: &[Expr]) -> bool { .any(|expr| matches!(expr, Expr::GroupingSet(_))) } -/// Calculates identifier key groups for aggregate expressions. -pub(crate) fn aggregate_identifier_keys( +/// Calculates functional dependencies for aggregate expressions +pub(crate) fn aggregate_functional_dependencies( // Expressions in the GROUP BY clause: group_expr: &[Expr], // Input plan of the aggregate: input: &LogicalPlan, // Schema field length: n_out: usize, -) -> Result { - let mut identifier_key_groups = vec![]; - // We can do a case analysis on how to propagate identifier keys based on +) -> Result { + // We can do a case analysis on how to propagate functional dependencies based on // whether the GROUP BY in question contains a grouping set expression: - // - If so, the identifier key will be empty because we cannot guarantee + // - If so, the functional dependencies will be empty because we cannot guarantee // that GROUP BY expression results will be unique. - // - Otherwise, it may be possible to propagate identifier keys. + // - Otherwise, it may be possible to propagate functional dependencies if !contains_grouping_set(group_expr) { - // Association covers whole table: - let associated_indices = (0..n_out).collect::>(); - // Get identifier key groups of the input: - let id_key_groups = input.schema().identifier_key_groups(); - let fields = input.schema().fields(); - for id_key_group in id_key_groups { - let mut new_identifier_key_indices = HashSet::new(); - let id_key_indices = &id_key_group.identifier_key_indices; - let id_key_field_names = id_key_indices - .iter() - .map(|&idx| fields[idx].qualified_name()) - .collect::>(); - for (idx, group_by_expr) in group_expr.iter().enumerate() { - // When one of the input identifier keys matches with the GROUP - // BY expression, add the index of the GROUP BY expression as - // an identifier key. - if id_key_field_names.contains(&group_by_expr.display_name()?) { - new_identifier_key_indices.insert(idx); - } - } - if !new_identifier_key_indices.is_empty() { - identifier_key_groups.push( - IdentifierKeyGroup::new( - new_identifier_key_indices.into_iter().collect(), - associated_indices.clone(), - ) - // input uniqueness stays the same when GROUP BY matches with input identifier keys - .with_is_unique(id_key_group.is_unique), - ); - } - } - // Guaranteed to be unique after aggregation, since it has a single - // expression (e.g. GROUP BY a): - if group_expr.len() == 1 { - // If identifier key groups contain 0, delete this identifier key from - // identifier key groups because it will be added anyway with is_unique - // flag set. - if let Some(idx) = identifier_key_groups - .iter() - .position(|item| item.identifier_key_indices.contains(&0)) - { - identifier_key_groups[idx] - .identifier_key_indices - .retain(|&x| x != idx); - if identifier_key_groups[idx].identifier_key_indices.is_empty() { - // Identifier key is empty, delete this identifier group. - // Example: `identifier_key_indices` was equal to vec![0], - // and deleting 0 emptied it. - identifier_key_groups.remove(idx); - } - } - // Add a new identifier group that is associated with whole table. - // Here, it is guaranteed to be unique. - identifier_key_groups.push( - IdentifierKeyGroup::new(vec![0], associated_indices).with_is_unique(true), - ); - } + let group_by_expr_names = group_expr + .iter() + .map(|item| item.display_name()) + .collect::>>()?; + let aggregate_func_dependencies = input + .schema() + .aggregate_functional_dependencies(&group_by_expr_names, n_out); + Ok(aggregate_func_dependencies) + } else { + Ok(FunctionalDependencies::empty()) } - Ok(identifier_key_groups) } /// Create Projection @@ -1462,13 +1320,13 @@ pub fn project( } validate_unique_names("Projections", projected_expr.iter())?; - // Update input identifier keys according to projection. - let id_key_groups = project_identifier_keys(&projected_expr, &plan)?; + // Update input functional dependencies according to projection. + let id_key_groups = project_functional_dependencies(&projected_expr, &plan)?; let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), )? - .with_identifier_key_groups(id_key_groups); + .with_functional_dependencies(id_key_groups); Ok(LogicalPlan::Projection(Projection::try_new_with_schema( projected_expr, @@ -1633,8 +1491,8 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new( DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? - // We can use existing identifier key groups: - .with_identifier_key_groups(input_schema.identifier_key_groups().clone()), + // We can use existing functional dependencies: + .with_functional_dependencies(input_schema.functional_dependencies().clone()), ); Ok(LogicalPlan::Unnest(Unnest { @@ -1648,7 +1506,9 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { mod tests { use crate::{expr, expr_fn::exists}; use arrow::datatypes::{DataType, Field}; - use datafusion_common::{OwnedTableReference, SchemaError, TableReference}; + use datafusion_common::{ + FunctionalDependence, OwnedTableReference, SchemaError, TableReference, + }; use crate::logical_plan::StringifiedPlan; @@ -2157,9 +2017,16 @@ mod tests { #[test] fn test_get_updated_id_keys() { - let identifier_key_groups = vec![IdentifierKeyGroup::new(vec![1], vec![0, 1, 2])]; - let res = project_identifier_key_indices(&identifier_key_groups, &[1, 2], 2); - let expected = vec![IdentifierKeyGroup::new(vec![0], vec![0, 1])]; + let fund_dependencies = + FunctionalDependencies::new(vec![FunctionalDependence::new( + vec![1], + vec![0, 1, 2], + )]); + let res = fund_dependencies.project_functional_dependencies(&[1, 2], 2); + let expected = FunctionalDependencies::new(vec![FunctionalDependence::new( + vec![0], + vec![0, 1], + )]); assert_eq!(res, expected); } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 33b2a28f44346..8e9d19d9ce121 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -17,7 +17,9 @@ //! Logical plan types -use crate::builder::{aggregate_identifier_keys, project_identifier_keys}; +use crate::builder::{ + aggregate_functional_dependencies, project_functional_dependencies, +}; use crate::expr::{Alias, Exists, InSubquery, Placeholder}; use crate::expr_rewriter::create_col_from_scalar_expr; use crate::expr_vec_fmt; @@ -37,7 +39,7 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{ plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - IdentifierKeyGroup, OwnedTableReference, Result, ScalarValue, + OwnedTableReference, Result, ScalarValue, }; use std::collections::{HashMap, HashSet}; use std::fmt::{self, Debug, Display, Formatter}; @@ -1270,10 +1272,10 @@ impl Projection { if expr.len() != schema.fields().len() { return Err(DataFusionError::Plan(format!("Projection has mismatch between number of expressions ({}) and number of fields in schema ({})", expr.len(), schema.fields().len()))); } - // update identifier key groups of `input` according to projection exprs. - let id_key_groups = project_identifier_keys(&expr, &input)?; + // update functional dependencies of `input` according to projection exprs. + let id_key_groups = project_functional_dependencies(&expr, &input)?; let schema = schema.as_ref().clone(); - let schema = Arc::new(schema.with_identifier_key_groups(id_key_groups)); + let schema = Arc::new(schema.with_functional_dependencies(id_key_groups)); Ok(Self { expr, input, @@ -1324,11 +1326,11 @@ impl SubqueryAlias { ) -> Result { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); - // Since schema is same, other than qualifier, we can use existing identifier key groups - let id_key_groups = plan.schema().identifier_key_groups().clone(); + // Since schema is same, other than qualifier, we can use existing functional dependencies + let func_dependencies = plan.schema().functional_dependencies().clone(); let schema = DFSchemaRef::new( DFSchema::try_from_qualified_schema(&alias, &schema)? - .with_identifier_key_groups(id_key_groups), + .with_functional_dependencies(func_dependencies), ); Ok(SubqueryAlias { input: Arc::new(plan), @@ -1412,30 +1414,17 @@ impl Window { .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); let metadata = input.schema().metadata().clone(); - // Update identifier key groups for window - let mut id_key_groups = input.schema().identifier_key_groups().clone(); - let n_input_fields = input.schema().fields().len(); - let new_associated_fields: Vec = - (n_input_fields..window_fields.len()).collect(); - for IdentifierKeyGroup { - is_unique, - associated_indices, - .. - } in id_key_groups.iter_mut() - { - // if unique, extend associations such that they cover - // new window expressions - if *is_unique { - associated_indices.extend(&new_associated_fields); - } - } + // Update functional dependencies for window + let mut window_func_dependencies = + input.schema().functional_dependencies().clone(); + window_func_dependencies.extend_target_indices(window_fields.len()); Ok(Window { input, window_expr, schema: Arc::new( DFSchema::new_with_metadata(window_fields, metadata)? - .with_identifier_key_groups(id_key_groups), + .with_functional_dependencies(window_func_dependencies), ), }) } @@ -1658,11 +1647,15 @@ impl Aggregate { ))); } - let identifier_key_groups = - aggregate_identifier_keys(&group_expr, &input, schema.fields().len())?; + let aggregate_func_dependencies = aggregate_functional_dependencies( + &group_expr, + &input, + schema.fields().len(), + )?; let new_schema = schema.as_ref().clone(); - let schema = - Arc::new(new_schema.with_identifier_key_groups(identifier_key_groups)); + let schema = Arc::new( + new_schema.with_functional_dependencies(aggregate_func_dependencies), + ); Ok(Self { input, group_expr, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 2c7232e34c5c5..556dcc2541633 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -448,8 +448,8 @@ pub fn expand_qualified_wildcard( } let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? - // We can use the identifier key as is, since it only stores indices: - .with_identifier_key_groups(schema.identifier_key_groups().clone()); + // We can use the functional dependencies as is, since it only stores indices: + .with_functional_dependencies(schema.functional_dependencies().clone()); let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, @@ -1021,9 +1021,9 @@ pub fn from_plan( let schema = Arc::new( DFSchema::new_with_metadata(fields, input.schema().metadata().clone())? - // We can use the existing identifier keys as is: - .with_identifier_key_groups( - input.schema().identifier_key_groups().clone(), + // We can use the existing functional dependencies as is: + .with_functional_dependencies( + input.schema().functional_dependencies().clone(), ), ); diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 02dc20ac44c81..0f6b5e2451832 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -265,7 +265,7 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { DFSchemaRef::new( DFSchema::new_with_metadata(new_fields, schema.metadata().clone()) .unwrap() - .with_identifier_key_groups(schema.identifier_key_groups().clone()), + .with_functional_dependencies(schema.functional_dependencies().clone()), ) } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index dad1a28be7595..4ff855a014614 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -493,12 +493,12 @@ mod tests { assert_eq!( "get table_scan rule\ncaused by\n\ Internal error: Optimizer rule 'get table_scan rule' failed, due to generate a different schema, \ - original schema: DFSchema { fields: [], metadata: {}, identifier_key_groups: [] }, \ + original schema: DFSchema { fields: [], metadata: {}, functional_dependencies: FunctionalDependencies { inner: [] } }, \ 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: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \ - metadata: {}, identifier_key_groups: [] }. \ + metadata: {}, functional_dependencies: FunctionalDependencies { inner: [] } }. \ This was likely caused by a bug in DataFusion's code \ and we would welcome that you file an bug report in our issue tracker", err.to_string() diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index cda93ffc1cf03..3b845cd7e2392 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -20,7 +20,7 @@ use crate::utils::{ check_columns_satisfy_exprs, extract_aliases, rebase_expr, resolve_aliases_to_exprs, resolve_columns, resolve_positions_to_exprs, }; -use datafusion_common::{DataFusionError, IdentifierKeyGroup, Result}; +use datafusion_common::{DataFusionError, Result}; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, }; @@ -432,44 +432,27 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { let schema = input.schema(); - let id_key_groups = schema.identifier_key_groups(); - - let field_names = schema - .fields() - .iter() - .map(|elem| elem.qualified_name()) - .collect::>(); let mut new_group_by_exprs = group_by_exprs.clone(); - for IdentifierKeyGroup { - identifier_key_indices, - associated_indices, - .. - } in id_key_groups - { - let id_key_names = identifier_key_indices - .iter() - .map(|id_key_idx| field_names[*id_key_idx].as_str()) - .collect::>(); - for group_by_expr in &group_by_exprs { - let expr_name = format!("{}", group_by_expr); - // group by expression contains identifier key - // In these case, we can use associated fields after aggregation - // even if they are not part of group by expressions - if id_key_names.contains(&expr_name.as_str()) { - let associated_field_names = associated_indices - .iter() - .map(|idx| field_names[*idx].as_str()) - .collect::>(); - // Expand group by exprs with select_exprs - // If one of the expressions inside group by is identifier key, and - // select expression is associated with that identifier key. - for expr in select_exprs { - let expr_name = format!("{}", expr); - if !new_group_by_exprs.contains(expr) - && associated_field_names.contains(&expr_name.as_str()) - { - new_group_by_exprs.push(expr.clone()); - } + let fields = schema.fields(); + for group_by_expr in &group_by_exprs { + if let Some(target_indices) = schema.get_target_functional_dependencies( + group_by_expr.display_name()?.as_str(), + ) { + // 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 exprs with select_exprs + // If groupby expr is determinant key, we can use its + // dependent keys in select statements also. + // Expand group expression with dependent select expressions. + 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()); } } } From f440adf0c67ed88aa1e2c6c8921c24536d1fe238 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 21 Jul 2023 18:58:38 +0300 Subject: [PATCH 27/37] Rename variables --- datafusion/expr/src/logical_plan/builder.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index e6348f9f04639..b0ed303ff57c7 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -273,7 +273,7 @@ impl LogicalPlanBuilder { let projected_schema = projection .as_ref() .map(|p| { - let projected_id_key_groups = + let projected_func_dependencies = func_dependencies.project_functional_dependencies(p, p.len()); DFSchema::new_with_metadata( p.iter() @@ -287,7 +287,7 @@ impl LogicalPlanBuilder { schema.metadata().clone(), ) .map(|df_schema| { - df_schema.with_functional_dependencies(projected_id_key_groups) + df_schema.with_functional_dependencies(projected_func_dependencies) }) }) .unwrap_or_else(|| { @@ -1321,12 +1321,13 @@ pub fn project( validate_unique_names("Projections", projected_expr.iter())?; // Update input functional dependencies according to projection. - let id_key_groups = project_functional_dependencies(&projected_expr, &plan)?; + let projected_func_dependencies = + project_functional_dependencies(&projected_expr, &plan)?; let input_schema = DFSchema::new_with_metadata( exprlist_to_fields(&projected_expr, &plan)?, plan.schema().metadata().clone(), )? - .with_functional_dependencies(id_key_groups); + .with_functional_dependencies(projected_func_dependencies); Ok(LogicalPlan::Projection(Projection::try_new_with_schema( projected_expr, From 349910f2b11b6deb3cdd6da0c20a2d120e61fcd4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 24 Jul 2023 13:45:16 +0300 Subject: [PATCH 28/37] Change primary key API --- datafusion/common/src/dfschema.rs | 172 +++++++++++------- datafusion/common/src/lib.rs | 4 +- .../src/datasource/default_table_source.rs | 6 +- datafusion/core/src/datasource/memory.rs | 18 +- datafusion/core/src/datasource/provider.rs | 6 +- datafusion/core/src/execution/context.rs | 22 +-- .../sqllogictests/test_files/groupby.slt | 119 ++++++++++++ datafusion/expr/src/logical_plan/builder.rs | 7 +- datafusion/expr/src/logical_plan/ddl.rs | 14 +- datafusion/expr/src/table_source.rs | 4 +- datafusion/expr/src/utils.rs | 2 +- datafusion/sql/src/query.rs | 2 +- datafusion/sql/src/select.rs | 44 +++-- datafusion/sql/src/statement.rs | 75 ++++---- 14 files changed, 325 insertions(+), 170 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 3b6b60678e1a8..6241995d313ac 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,6 +33,15 @@ use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; +/// Stores constraint about the table. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub enum Constraint { + // columns together form a primary key (are unique and not null) + PrimaryKey(Vec), + // columns together form a unique key + Unique(Vec), +} + /// Stores functional dependence in the schema. /// Functional dependence defines relationship between determinants and dependents. /// The value of determinant keys uniquely determines the value of dependents. A determinant key @@ -50,6 +59,12 @@ pub type DFSchemaRef = Arc; #[derive(Debug, Clone, PartialEq, Eq)] pub struct FunctionalDependence { pub source_indices: Vec, + /// Nullable keeps track of whether one of the `source_indices` can receive NULL values. + /// if constraint at the source is `unique`, this flag is `true`. + /// if constraint at the source is `primary key`, this flag is `false`. + /// Please note that, as schema changes between different stages in the plan, + /// this property may change. (Such as LEFT JOIN, RIGHT JOIN). + pub nullable: bool, pub mode: Dependency, pub target_indices: Vec, } @@ -62,9 +77,14 @@ pub enum Dependency { } impl FunctionalDependence { - pub fn new(source_indices: Vec, target_indices: Vec) -> Self { + pub fn new( + source_indices: Vec, + target_indices: Vec, + nullable: bool, + ) -> Self { Self { source_indices, + nullable, // Dependency::Multi is the most relaxed mode // hence, if not specified this is the safest approach mode: Dependency::Multi, @@ -98,18 +118,31 @@ impl FunctionalDependencies { } pub fn new_from_primary_keys( - primary_key_indices: Option<&[usize]>, + constraints: Option<&[Constraint]>, n_field: usize, ) -> Self { - if let Some(primary_key_indices) = primary_key_indices { + if let Some(constraints) = constraints { // All the field indices are associated, with whole table. since it is primary key. let target_indices = (0..n_field).collect::>(); - let dependencies = vec![FunctionalDependence::new( - primary_key_indices.to_vec(), - target_indices, - ) - // As primary keys are guaranteed to be unique, set `mode` of functional dependency to `Dependency::Single`. - .with_mode(Dependency::Single)]; + let dependencies = constraints + .iter() + .map(|constraint| { + let dependency = match constraint { + Constraint::PrimaryKey(indices) => FunctionalDependence::new( + indices.to_vec(), + target_indices.clone(), + false, + ), + Constraint::Unique(indices) => FunctionalDependence::new( + indices.to_vec(), + target_indices.clone(), + true, + ), + }; + // As primary keys are guaranteed to be unique, set `mode` of functional dependency to `Dependency::Single`. + dependency.with_mode(Dependency::Single) + }) + .collect::>(); Self::new(dependencies) } else { Self::empty() @@ -159,6 +192,7 @@ impl FunctionalDependencies { let mut projected_func_dependencies = vec![]; for FunctionalDependence { source_indices, + nullable, mode, target_indices, } in &self.inner @@ -172,10 +206,15 @@ impl FunctionalDependencies { // Update associations according to projection: update_elements_with_matching_indices(target_indices, proj_indices) }; - if !new_source_indices.is_empty() { - let new_func_dependence = - FunctionalDependence::new(new_source_indices, new_target_indices) - .with_mode(*mode); + // All of the composite indices should still valid after projection, + // otherwise functional dependency cannot be propagated. + if new_source_indices.len() == source_indices.len() { + let new_func_dependence = FunctionalDependence::new( + new_source_indices, + new_target_indices, + *nullable, + ) + .with_mode(*mode); projected_func_dependencies.push(new_func_dependence); } } @@ -188,30 +227,38 @@ impl FunctionalDependencies { join_type: &JoinType, left_cols_len: usize, ) -> FunctionalDependencies { + // Get mutable versions of left and right side dependencies let mut right_func_dependencies = other.clone(); - right_func_dependencies.add_offset(left_cols_len); - let mut left_func_dependencies = self.clone(); - // After a join, a primary key may no longer be unique. However, as it still - // defines unique set of column values for its target columns. - // Reset mode to `Dependency::Multi` - left_func_dependencies = - left_func_dependencies.with_dependency(Dependency::Multi); - right_func_dependencies = - right_func_dependencies.with_dependency(Dependency::Multi); match join_type { - JoinType::Inner => { - // Left then right: + JoinType::Inner | JoinType::Left | JoinType::Right => { + // Add offset to right schema + right_func_dependencies.add_offset(left_cols_len); + + // result may have multiple values, hence set dependency mode to multi + left_func_dependencies = + left_func_dependencies.with_dependency(Dependency::Multi); + right_func_dependencies = + right_func_dependencies.with_dependency(Dependency::Multi); + + if *join_type == JoinType::Left { + // downgrade right side, since this side may have additional null values + right_func_dependencies.downgrade_dependencies(); + } else if *join_type == JoinType::Right { + // downgrade left side, since this side may have additional null values + left_func_dependencies.downgrade_dependencies(); + } + // Combine left and right functional dependencies left_func_dependencies.extend(right_func_dependencies); left_func_dependencies } - JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - // Only use the left side for the schema: + JoinType::LeftSemi | JoinType::LeftAnti => { + // LeftSemi and LeftAnti preserves functional dependencies of left side. left_func_dependencies } - JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { - // Only use the right side for the schema: + JoinType::RightSemi | JoinType::RightAnti => { + // RightSemi and RightAnti preserves functional dependencies of right side. right_func_dependencies } JoinType::Full => { @@ -221,6 +268,17 @@ impl FunctionalDependencies { } } + // This case occurs, when current dependency gets a new null value. + // If dependency is UNIQUE(e.g nullable), new null value invalidates dependency + // If dependency is PRIMARY KEY(e.g not nullable), new null value turns it into + // UNIQUE mode. + fn downgrade_dependencies(&mut self) { + // Delete nullable dependencies, since they will no longer be + // functional dependency + self.inner.retain(|item| !item.nullable); + self.inner.iter_mut().for_each(|item| item.nullable = true); + } + pub fn extend_target_indices(&mut self, n_out: usize) { // if unique cover whole table let new_target_indices = (0..n_out).collect::>(); @@ -340,23 +398,6 @@ impl DFSchema { ) } - pub fn join_functional_dependencies( - &self, - other: &DFSchema, - ) -> FunctionalDependencies { - let mut functional_dependencies = self.functional_dependencies.clone(); - // Make mode multi, since after join same rows may get replicated. - functional_dependencies = - functional_dependencies.with_dependency(Dependency::Multi); - let mut other_functional_dependencies = other.functional_dependencies().clone(); - other_functional_dependencies.add_offset(self.fields.len()); - other_functional_dependencies = - other_functional_dependencies.with_dependency(Dependency::Multi); - - functional_dependencies.extend(other_functional_dependencies); - functional_dependencies - } - pub fn aggregate_functional_dependencies( &self, group_by_expr_names: &[String], @@ -370,6 +411,7 @@ impl DFSchema { let fields = self.fields(); for FunctionalDependence { source_indices, + nullable, mode, .. } in &func_dependencies.inner @@ -388,11 +430,13 @@ impl DFSchema { new_source_indices.insert(idx); } } - if !new_source_indices.is_empty() { + // All of the composite, indices occur in group by expression + if new_source_indices.len() == source_indices.len() { aggregate_func_dependencies.push( FunctionalDependence::new( new_source_indices.into_iter().collect(), target_indices.clone(), + *nullable, ) // input uniqueness stays the same when GROUP BY matches with input functional dependence determinants .with_mode(*mode), @@ -409,19 +453,14 @@ impl DFSchema { .iter() .position(|item| item.source_indices.contains(&0)) { - aggregate_func_dependencies[idx] - .source_indices - .retain(|&x| x != idx); - if aggregate_func_dependencies[idx].source_indices.is_empty() { - // `source_indices` is empty, delete this identifier group. - // Example: `source_indices` was equal to vec![0], - // and deleting 0 emptied it. - aggregate_func_dependencies.remove(idx); - } + // delete the functional dependency that contains 0th idx. + aggregate_func_dependencies.remove(idx); } // Add a new functional dependence that is associated with whole table. aggregate_func_dependencies.push( - FunctionalDependence::new(vec![0], target_indices) + // Safest behaviour to do is add new functional dependency as nullable. + // TODO: use nullable property of group by expr here. + FunctionalDependence::new(vec![0], target_indices, true) .with_mode(Dependency::Single), ); } @@ -430,8 +469,9 @@ impl DFSchema { pub fn get_target_functional_dependencies( &self, - expr_name: &str, - ) -> Option<&[usize]> { + group_by_expr_names: &[String], + ) -> Option> { + let mut combined_target_indices = HashSet::new(); let func_dependencies = self.functional_dependencies(); let field_names = self .fields() @@ -446,16 +486,20 @@ impl DFSchema { { let source_key_names = source_indices .iter() - .map(|id_key_idx| field_names[*id_key_idx].as_str()) + .map(|id_key_idx| field_names[*id_key_idx].clone()) .collect::>(); // group by expression contains a determinant from a functional dependency // In this case, we can use associated fields after aggregation // even if they are not part of group by expressions - if source_key_names.contains(&expr_name) { - return Some(target_indices); + if source_key_names + .iter() + .all(|source_key_name| group_by_expr_names.contains(source_key_name)) + { + combined_target_indices.extend(target_indices.iter()); } } - None + (!combined_target_indices.is_empty()) + .then_some(combined_target_indices.iter().cloned().collect::>()) } /// Assign functional dependencies @@ -474,9 +518,7 @@ impl DFSchema { let mut metadata = self.metadata.clone(); fields.extend_from_slice(schema.fields().as_slice()); metadata.extend(schema.metadata.clone()); - let join_functional_dependencies = self.join_functional_dependencies(schema); - Ok(Self::new_with_metadata(fields, metadata)? - .with_functional_dependencies(join_functional_dependencies)) + Self::new_with_metadata(fields, metadata) } /// Modify this schema by appending the fields from the supplied schema, ignoring any diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 18ece01a60b69..4dbafc91faf2c 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -37,8 +37,8 @@ pub mod utils; pub use column::Column; pub use dfschema::{ - DFField, DFSchema, DFSchemaRef, Dependency, ExprSchema, FunctionalDependence, - FunctionalDependencies, SchemaExt, ToDFSchema, + Constraint, DFField, DFSchema, DFSchemaRef, Dependency, ExprSchema, + FunctionalDependence, FunctionalDependencies, SchemaExt, ToDFSchema, }; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 2de6a43161290..b784bb8ed705d 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use crate::datasource::TableProvider; use arrow::datatypes::SchemaRef; -use datafusion_common::DataFusionError; +use datafusion_common::{Constraint, DataFusionError}; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; /// DataFusion default table source, wrapping TableProvider @@ -55,8 +55,8 @@ impl TableSource for DefaultTableSource { } /// Get a reference to primary key indices, if a primary key exists. - fn primary_keys(&self) -> Option<&[usize]> { - self.table_provider.primary_keys() + fn primary_keys(&self) -> Option<&[Constraint]> { + self.table_provider.constraints() } /// Tests whether the table provider can make use of any or all filter expressions diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 3710053788acd..929fe857cd56a 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -26,7 +26,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use async_trait::async_trait; -use datafusion_common::SchemaExt; +use datafusion_common::{Constraint, SchemaExt}; use datafusion_execution::TaskContext; use tokio::sync::RwLock; use tokio::task::JoinSet; @@ -52,7 +52,7 @@ pub type PartitionData = Arc>>; pub struct MemTable { schema: SchemaRef, pub(crate) batches: Vec, - primary_keys: Option>, + constraints: Option>, } impl MemTable { @@ -77,14 +77,14 @@ impl MemTable { .into_iter() .map(|e| Arc::new(RwLock::new(e))) .collect::>(), - primary_keys: None, + constraints: None, }) } - /// Assign primary keys - pub fn with_primary_keys(mut self, primary_keys: Vec) -> Self { - if !primary_keys.is_empty() { - self.primary_keys = Some(primary_keys); + /// Assign constraints + pub fn with_constraints(mut self, constraints: Vec) -> Self { + if !constraints.is_empty() { + self.constraints = Some(constraints); } self } @@ -163,8 +163,8 @@ impl TableProvider for MemTable { self.schema.clone() } - fn primary_keys(&self) -> Option<&[usize]> { - self.primary_keys.as_deref() + fn constraints(&self) -> Option<&[Constraint]> { + self.constraints.as_deref() } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 5b82ae9a3ef52..c4450b4723c44 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::sync::Arc; use async_trait::async_trait; -use datafusion_common::{DataFusionError, Statistics}; +use datafusion_common::{Constraint, DataFusionError, Statistics}; use datafusion_expr::{CreateExternalTable, LogicalPlan}; pub use datafusion_expr::{TableProviderFilterPushDown, TableType}; @@ -41,8 +41,8 @@ pub trait TableProvider: Sync + Send { /// Get a reference to the schema for this table fn schema(&self) -> SchemaRef; - /// Get a reference to primary key indices, if a primary key exists. - fn primary_keys(&self) -> Option<&[usize]> { + /// Get a reference to the constraints of the table. + fn constraints(&self) -> Option<&[Constraint]> { None } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index a8ebd19dd2b2a..d639560c4da19 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -470,13 +470,12 @@ impl SessionContext { input, if_not_exists, or_replace, - primary_key, + constraints, } = cmd; let input = Arc::try_unwrap(input).unwrap_or_else(|e| e.as_ref().clone()); let input = self.state().optimize(&input)?; let table = self.table(&name).await; - match (if_not_exists, or_replace, table) { (true, false, Ok(_)) => self.return_empty_dataframe(), (false, true, Ok(_)) => { @@ -495,28 +494,13 @@ impl SessionContext { )), (_, _, Err(_)) => { let df_schema = input.schema(); - // Get primary key indices in the schema: - let primary_keys = primary_key - .iter() - .map(|pk| { - df_schema - .fields() - .iter() - .position(|item| item.qualified_name() == pk.flat_name()) - .ok_or_else(|| { - DataFusionError::Execution( - "Primary key doesn't exist".to_string(), - ) - }) - }) - .collect::>>()?; - let schema = Arc::new(df_schema.as_ref().into()); let physical = DataFrame::new(self.state(), input); let batches: Vec<_> = physical.collect_partitioned().await?; let table = Arc::new( - MemTable::try_new(schema, batches)?.with_primary_keys(primary_keys), + // pass constrains to the mem table. + MemTable::try_new(schema, batches)?.with_constraints(constraints), ); self.register_table(&name, table)?; diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 7c6328f7883a1..8c32aa529df48 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2874,6 +2874,39 @@ CREATE TABLE sales_global_with_pk (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) +# create a table for testing, where primary key is composite +statement ok +CREATE TABLE sales_global_with_composite_pk (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT, + primary key(sn, ts) + ) 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 sn is unique key +statement ok +CREATE TABLE sales_global_with_unique (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT, + unique(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), + (1, 'TUR', NULL, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0) + # when group by contains primary key expression # we can use all the expressions in the table during selection # (not just group by expressions + aggregation result) @@ -2958,6 +2991,92 @@ SELECT r.sn, SUM(l.amount), r.amount 3 300 200 4 100 100 +# when primary key consists of composite columns +# to associate it with other fields, aggregate should contain all the composite columns +query IRR +SELECT r.sn, SUM(l.amount), r.amount + FROM sales_global_with_composite_pk AS l + JOIN sales_global_with_composite_pk AS r + ON l.sn >= r.sn + GROUP BY r.sn, r.ts + ORDER BY r.sn +---- +0 455 30 +1 425 50 +2 375 75 +3 300 200 +4 100 100 + +# when primary key consists of composite columns +# to associate it with other fields, aggregate should contain all the composite columns +# if any of the composite column is missing, we cannot use associated indices, inside select expression +# below query should fail +statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression r.amount could not be resolved from available columns: r.sn, SUM\(l.amount\) +SELECT r.sn, SUM(l.amount), r.amount + FROM sales_global_with_composite_pk AS l + JOIN sales_global_with_composite_pk AS r + ON l.sn >= r.sn + GROUP BY r.sn + ORDER BY r.sn + +# left join should propagate right side constraint, +# if right side is a primary key (unique and doesn't contain null) +query IRR +SELECT r.sn, r.amount, SUM(r.amount) + FROM (SELECT * + FROM sales_global_with_pk as l + LEFT JOIN sales_global_with_pk as r + ON l.amount >= r.amount + 10) + GROUP BY r.sn +ORDER BY r.sn +---- +0 30 120 +1 50 150 +2 75 150 +4 100 100 +NULL NULL NULL + +# left join shouldn't propagate right side constraint, +# if right side is a unique key (unique and can contain null) +# Please note that, above query and this one is same except the constraint in the table. +statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression r.amount could not be resolved from available columns: r.sn, SUM\(r.amount\) +SELECT r.sn, r.amount, SUM(r.amount) + FROM (SELECT * + FROM sales_global_with_unique as l + LEFT JOIN sales_global_with_unique as r + ON l.amount >= r.amount + 10) + GROUP BY r.sn +ORDER BY r.sn + +# left semi join should propagate constraint of left side as is. +query IRR +SELECT l.sn, l.amount, SUM(l.amount) + FROM (SELECT * + FROM sales_global_with_unique as l + LEFT SEMI JOIN sales_global_with_unique as r + ON l.amount >= r.amount + 10) + GROUP BY l.sn +ORDER BY l.sn +---- +1 50 50 +2 75 75 +3 200 200 +4 100 100 +NULL 100 100 + +# Similarly, left anti join should propagate constraint of left side as is. +query IRR +SELECT l.sn, l.amount, SUM(l.amount) + FROM (SELECT * + FROM sales_global_with_unique as l + LEFT ANTI JOIN sales_global_with_unique as r + ON l.amount >= r.amount + 10) + GROUP BY l.sn +ORDER BY l.sn +---- +0 30 30 + + # 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\) SELECT l.sn, r.sn, SUM(l.amount), r.amount diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index b0ed303ff57c7..148d35bf9ae7f 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -816,11 +816,12 @@ impl LogicalPlanBuilder { /// Apply a cross join pub fn cross_join(self, right: LogicalPlan) -> Result { - let schema = self.plan.schema().join(right.schema())?; + let join_schema = + build_join_schema(self.plan.schema(), right.schema(), &JoinType::Inner)?; Ok(Self::from(LogicalPlan::CrossJoin(CrossJoin { left: Arc::new(self.plan), right: Arc::new(right), - schema: DFSchemaRef::new(schema), + schema: DFSchemaRef::new(join_schema), }))) } @@ -2022,11 +2023,13 @@ mod tests { 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/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index e005f114719df..8e413bbb179c9 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. +use datafusion_common::OwnedSchemaReference; use datafusion_common::{ - parsers::CompressionTypeVariant, DFSchemaRef, OwnedTableReference, + parsers::CompressionTypeVariant, Constraint, DFSchemaRef, OwnedTableReference, }; -use datafusion_common::{Column, OwnedSchemaReference}; use std::collections::HashMap; use std::sync::Arc; use std::{ @@ -117,14 +117,14 @@ impl DdlStatement { } DdlStatement::CreateMemoryTable(CreateMemoryTable { name, - primary_key, + constraints, .. }) => { let pk: Vec = - primary_key.iter().map(|c| c.name.to_string()).collect(); + constraints.iter().map(|c| format!("{:?}", c)).collect(); let mut pk = pk.join(", "); if !pk.is_empty() { - pk = format!(" primary_key=[{pk}]"); + pk = format!(" constraints=[{pk}]"); } write!(f, "CreateMemoryTable: {name:?}{pk}") } @@ -222,8 +222,8 @@ impl Hash for CreateExternalTable { pub struct CreateMemoryTable { /// The table name pub name: OwnedTableReference, - /// The ordered list of columns in the primary key, or an empty vector if none - pub primary_key: Vec, + /// The list of constraints in the schema, such as primary key, unique, etc. + pub constraints: Vec, /// The logical plan pub input: Arc, /// Option to not error if table already exists diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 0592c9ebf2edd..e6bebbb488090 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -19,7 +19,7 @@ use crate::{Expr, LogicalPlan}; use arrow::datatypes::SchemaRef; -use datafusion_common::Result; +use datafusion_common::{Constraint, Result}; use std::any::Any; /// Indicates whether and how a filter expression can be handled by a @@ -65,7 +65,7 @@ pub trait TableSource: Sync + Send { fn schema(&self) -> SchemaRef; /// Get primary key indices, if one exists. - fn primary_keys(&self) -> Option<&[usize]> { + fn primary_keys(&self) -> Option<&[Constraint]> { None } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 556dcc2541633..c716be82d0b61 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -924,7 +924,7 @@ pub fn from_plan( })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { input: Arc::new(inputs[0].clone()), - primary_key: vec![], + constraints: vec![], name: name.clone(), if_not_exists: *if_not_exists, or_replace: *or_replace, diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 2d7771d8c753c..6c4c9095032fa 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -86,7 +86,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let select_into = select.into.unwrap(); LogicalPlan::Ddl(DdlStatement::CreateMemoryTable(CreateMemoryTable { name: self.object_name_to_table_reference(select_into.name)?, - primary_key: Vec::new(), + constraints: Vec::new(), input: Arc::new(plan), if_not_exists: false, or_replace: false, diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 3b845cd7e2392..2d11b8fd02248 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -434,29 +434,33 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let schema = input.schema(); let mut new_group_by_exprs = group_by_exprs.clone(); let fields = schema.fields(); - for group_by_expr in &group_by_exprs { - if let Some(target_indices) = schema.get_target_functional_dependencies( - group_by_expr.display_name()?.as_str(), - ) { - // 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 exprs with select_exprs - // If groupby expr is determinant key, we can use its - // dependent keys in select statements also. - // Expand group expression with dependent select expressions. - 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()); - } + 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 select, even if they do not occur in aggregation. + if let Some(target_indices) = + schema.get_target_functional_dependencies(&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 exprs with select_exprs + // If groupby expr is determinant key, we can use its + // dependent keys in select statements also. + // Expand group expression with dependent select expressions. + 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()); } } } + let group_by_exprs = new_group_by_exprs; // create the aggregate plan diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index a5ff3633acad9..ba1c22d2bb9a2 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -26,8 +26,9 @@ use crate::utils::normalize_ident; use arrow_schema::DataType; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - unqualified_field_not_found, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - ExprSchema, OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, + unqualified_field_not_found, Column, Constraint, DFField, DFSchema, DFSchemaRef, + DataFusionError, ExprSchema, OwnedTableReference, Result, SchemaReference, + TableReference, ToDFSchema, }; use datafusion_expr::expr_rewriter::normalize_col_with_schemas_and_ambiguity_check; use datafusion_expr::logical_plan::builder::project; @@ -132,8 +133,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .. } if table_properties.is_empty() && with_options.is_empty() => match query { Some(query) => { - let primary_key = Self::primary_key_from_constraints(&constraints)?; - let plan = self.query_to_plan(*query, planner_context)?; let input_schema = plan.schema(); @@ -163,10 +162,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { plan }; + let constraints = + Self::primary_key_from_constraints(&constraints, plan.schema())?; + Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { name: self.object_name_to_table_reference(name)?, - primary_key, + constraints, input: Arc::new(plan), if_not_exists, or_replace, @@ -175,19 +177,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } None => { - let primary_key = Self::primary_key_from_constraints(&constraints)?; - let schema = self.build_schema(columns)?.to_dfschema_ref()?; let plan = EmptyRelation { produce_one_row: false, schema, }; let plan = LogicalPlan::EmptyRelation(plan); - + let constraints = + Self::primary_key_from_constraints(&constraints, plan.schema())?; Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { name: self.object_name_to_table_reference(name)?, - primary_key, + constraints, input: Arc::new(plan), if_not_exists, or_replace, @@ -1163,20 +1164,40 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { fn primary_key_from_constraints( constraints: &[TableConstraint], - ) -> Result> { - let pk: Result>> = constraints + df_schema: &DFSchemaRef, + ) -> Result> { + constraints .iter() .map(|c: &TableConstraint| match c { TableConstraint::Unique { columns, is_primary, .. - } => match is_primary { - true => Ok(columns), - false => Err(DataFusionError::Plan( - "Non-primary unique constraints are not supported".to_string(), - )), - }, + } => { + // Get primary key indices in the schema: + let indices = columns + .iter() + .map(|pk| { + let idx = df_schema + .fields() + .iter() + .position(|item| { + item.qualified_name() == pk.value.clone() + }) + .ok_or_else(|| { + DataFusionError::Execution( + "Primary key doesn't exist".to_string(), + ) + })?; + Ok(idx) + }) + .collect::>>()?; + Ok(if *is_primary { + Constraint::PrimaryKey(indices) + } else { + Constraint::Unique(indices) + }) + } TableConstraint::ForeignKey { .. } => Err(DataFusionError::Plan( "Foreign key constraints are not currently supported".to_string(), )), @@ -1190,24 +1211,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { "Indexes are not currently supported".to_string(), )), }) - .collect(); - let pk = pk?; - let pk = match pk.as_slice() { - [] => return Ok(vec![]), - [pk] => pk, - _ => { - return Err(DataFusionError::Plan( - "Only one primary key is supported!".to_string(), - ))? - } - }; - let primary_key: Vec = pk - .iter() - .map(|c| Column { - relation: None, - name: c.value.clone(), - }) - .collect(); - Ok(primary_key) + .collect::>>() } } From 9c0f6d009818a942b4f41bd86e99a0dc7c332779 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 24 Jul 2023 17:01:02 +0300 Subject: [PATCH 29/37] Fix test --- datafusion/sql/tests/sql_integration.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 652011fbac318..3bafc7b6fb931 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -232,7 +232,7 @@ fn cast_to_invalid_decimal_type_precision_lt_scale() { fn plan_create_table_with_pk() { let sql = "create table person (id int, name string, primary key(id))"; let plan = r#" -CreateMemoryTable: Bare { table: "person" } primary_key=[id] +CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])] EmptyRelation "# .trim(); @@ -251,10 +251,9 @@ CreateMemoryTable: Bare { table: "person" } } #[test] -#[should_panic(expected = "Non-primary unique constraints are not supported")] fn plan_create_table_check_constraint() { let sql = "create table person (id int, name string, unique(id))"; - let plan = ""; + let plan = "CreateMemoryTable: Bare { table: \"person\" } constraints=[Unique([0])]\n EmptyRelation"; quick_test(sql, plan); } From 6e64b0cd0912a13c5e2fd09b553ef9a950b91637 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 25 Jul 2023 17:04:50 +0300 Subject: [PATCH 30/37] Functional dependency review --- datafusion/common/src/dfschema.rs | 226 +++++++++--------- .../src/datasource/default_table_source.rs | 4 +- datafusion/expr/src/logical_plan/builder.rs | 30 +-- datafusion/expr/src/logical_plan/ddl.rs | 9 +- datafusion/expr/src/logical_plan/plan.rs | 21 +- datafusion/expr/src/table_source.rs | 4 +- datafusion/sql/src/select.rs | 20 +- datafusion/sql/src/statement.rs | 5 +- 8 files changed, 169 insertions(+), 150 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 6241995d313ac..0e89c39f3470d 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -33,50 +33,55 @@ use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -/// Stores constraint about the table. +/// This object defines a constraint on a table. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum Constraint { - // columns together form a primary key (are unique and not null) + /// Columns with the given indices form a composite primary key (they are + /// jointly unique and not nullable): PrimaryKey(Vec), - // columns together form a unique key + /// Columns with the given indices form a composite unique key: Unique(Vec), } -/// Stores functional dependence in the schema. -/// Functional dependence defines relationship between determinants and dependents. -/// The value of determinant keys uniquely determines the value of dependents. A determinant key -/// is a column whose value determines values of some other (dependent) columns. -/// If two rows have the same determinant key, dependent columns in these rows -/// are necessarily the same. If the determinant key is unique, the set of +/// This object defines a functional dependence in the schema. A functional +/// dependence defines a relationship between determinant keys and dependent +/// columns. A determinant key is a column, or a set of columns, whose value +/// uniquely determines values of some other (dependent) columns. If two rows +/// have the same determinant key, dependent columns in these rows are +/// necessarily the same. If the determinant key is unique, the set of /// dependent columns is equal to the entire schema and the determinant key can -/// serve as a primary key. Note that a primary key may "downgrade" into an -/// determinant key due to an operation such as a join, and this object is used to -/// track dependence relationships in such cases. -/// For more information about functional dependency see -/// Indices of determinants are stored in `source_indices`, and indices of -/// dependents are stored in `target_indices` +/// serve as a primary key. Note that a primary key may "downgrade" into a +/// determinant key due to an operation such as a join, and this object is +/// used to track dependence relationships in such cases. For more information +/// on functional dependencies, see: /// #[derive(Debug, Clone, PartialEq, Eq)] pub struct FunctionalDependence { + // Column indices of the (possibly composite) determinant key: pub source_indices: Vec, - /// Nullable keeps track of whether one of the `source_indices` can receive NULL values. - /// if constraint at the source is `unique`, this flag is `true`. - /// if constraint at the source is `primary key`, this flag is `false`. - /// Please note that, as schema changes between different stages in the plan, - /// this property may change. (Such as LEFT JOIN, RIGHT JOIN). + // Column indices of dependent column(s): + pub target_indices: Vec, + /// Flag indicating whether one of the `source_indices` can receive NULL values. + /// For a data source, if the constraint in question is `Constraint::Unique`, + /// this flag is `true`. If the constraint in question is `Constraint::PrimaryKey`, + /// this flag is `false`. + /// Note that as the schema changes between different stages in a plan, + /// such as after LEFT JOIN or RIGHT JOIN operations, this property may + /// change. pub nullable: bool, + // The functional dependency mode: pub mode: Dependency, - pub target_indices: Vec, } -/// Describes functional dependency mode +/// Describes functional dependency mode. #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub enum Dependency { - Single, // unique constraint - Multi, // non unique (e.g. multipart primary key) + Single, // A determinant key may occur only once. + Multi, // A determinant key may occur multiple times (in multiple rows). } impl FunctionalDependence { + // Creates a new functional dependence. pub fn new( source_indices: Vec, target_indices: Vec, @@ -84,11 +89,10 @@ impl FunctionalDependence { ) -> Self { Self { source_indices, + target_indices, nullable, - // Dependency::Multi is the most relaxed mode - // hence, if not specified this is the safest approach + // Start with the least restrictive mode by default: mode: Dependency::Multi, - target_indices, } } @@ -98,70 +102,74 @@ impl FunctionalDependence { } } -/// Encapsulate functional dependencies +/// This object encapsulates all functional dependencies in a given relation. #[derive(Debug, Clone, PartialEq, Eq)] pub struct FunctionalDependencies { - inner: Vec, + deps: Vec, } impl FunctionalDependencies { - /// Create an empty `FunctionalDependencies` + /// Creates an empty `FunctionalDependencies` object. pub fn empty() -> Self { - Self { inner: vec![] } + Self { deps: vec![] } } - /// Create a new `FunctionalDependencies` from vector of `FunctionalDependence`s. - pub fn new(functional_dependencies: Vec) -> Self { - Self { - inner: functional_dependencies, - } + /// Creates a new `FunctionalDependencies` object from a vector of + /// `FunctionalDependence` objects. + pub fn new(dependencies: Vec) -> Self { + Self { deps: dependencies } } - pub fn new_from_primary_keys( + /// Creates a new `FunctionalDependencies` object from the given constraints. + pub fn new_from_constraints( constraints: Option<&[Constraint]>, n_field: usize, ) -> Self { if let Some(constraints) = constraints { - // All the field indices are associated, with whole table. since it is primary key. - let target_indices = (0..n_field).collect::>(); + // Construct dependency objects based on each individual constraint: let dependencies = constraints .iter() .map(|constraint| { + // All the field indices are associated with the whole table + // since we are dealing with table level constraints: let dependency = match constraint { Constraint::PrimaryKey(indices) => FunctionalDependence::new( indices.to_vec(), - target_indices.clone(), + (0..n_field).collect::>(), false, ), Constraint::Unique(indices) => FunctionalDependence::new( indices.to_vec(), - target_indices.clone(), + (0..n_field).collect::>(), true, ), }; - // As primary keys are guaranteed to be unique, set `mode` of functional dependency to `Dependency::Single`. + // As primary keys are guaranteed to be unique, set the + // functional dependency mode to `Dependency::Single`: dependency.with_mode(Dependency::Single) }) .collect::>(); Self::new(dependencies) } else { + // There is no constraint, return an empty object: Self::empty() } } pub fn with_dependency(mut self, mode: Dependency) -> Self { - self.inner.iter_mut().for_each(|item| item.mode = mode); + self.deps.iter_mut().for_each(|item| item.mode = mode); self } + /// Merges the given functional dependencies with these. pub fn extend(&mut self, other: FunctionalDependencies) { - self.inner.extend(other.inner); + self.deps.extend(other.deps); } - /// Add `offset` value to `source_indices` and `target_indices` - /// for each functional dependency + /// Adds the `offset` value to `source_indices` and `target_indices` for + /// each functional dependency. pub fn add_offset(&mut self, offset: usize) { - self.inner.iter_mut().for_each( + self.deps.iter_mut().for_each( |FunctionalDependence { source_indices, target_indices, @@ -173,29 +181,28 @@ impl FunctionalDependencies { ) } - /// Update `source_indices`, and `target_indices` of each `FunctionalDependence` - /// using the index mapping in `proj_indices`. - /// Assume that `proj_indices` is \[2, 5, 8\] and functional dependence is - /// \[5\] (`source_indices`) -> \[5, 8\] (`target_indices`) in `df_schema`. - /// Then, the return value will be \[1\] -> \[1, 2\]. - /// This means that the first index of the `proj_indices` (5) is a determinant key - /// (stored at `source_indices`), and its key is dependent keys are at indices 1 and 2 (in - /// the updated schema) (stored at `target_indices`). - /// Please note that: In the updated schema, fields at indices \[2, 5, 8\] will be at \[0, 1, 2\]. + /// Updates `source_indices` and `target_indices` of each functional + /// dependence using the index mapping given in `proj_indices`. + /// + /// Assume that `proj_indices` is \[2, 5, 8\] and we have a functional + /// dependence \[5\] (`source_indices`) -> \[5, 8\] (`target_indices`). + /// In the updated schema, fields at indices \[2, 5, 8\] will transform + /// to \[0, 1, 2\]. Therefore, the resulting functional dependence will + /// be \[1\] -> \[1, 2\]. pub fn project_functional_dependencies( &self, proj_indices: &[usize], - // If dependency mode is Single. Association covers whole table. `n_out` - // stores schema field length to be able to correctly associate with whole table. + // The argument `n_out` denotes the schema field length, which is needed + // to correctly associate a `Single`-mode dependence with the whole table. n_out: usize, ) -> FunctionalDependencies { let mut projected_func_dependencies = vec![]; for FunctionalDependence { source_indices, + target_indices, nullable, mode, - target_indices, - } in &self.inner + } in &self.deps { let new_source_indices = update_elements_with_matching_indices(source_indices, proj_indices); @@ -206,8 +213,8 @@ impl FunctionalDependencies { // Update associations according to projection: update_elements_with_matching_indices(target_indices, proj_indices) }; - // All of the composite indices should still valid after projection, - // otherwise functional dependency cannot be propagated. + // All of the composite indices should still be valid after projection; + // otherwise, functional dependency cannot be propagated. if new_source_indices.len() == source_indices.len() { let new_func_dependence = FunctionalDependence::new( new_source_indices, @@ -221,83 +228,88 @@ impl FunctionalDependencies { FunctionalDependencies::new(projected_func_dependencies) } + /// This function joins this set of functional dependencies with the `other` + /// according to the given `join_type`. pub fn join( &self, other: &FunctionalDependencies, join_type: &JoinType, left_cols_len: usize, ) -> FunctionalDependencies { - // Get mutable versions of left and right side dependencies + // Get mutable copies of left and right side dependencies: let mut right_func_dependencies = other.clone(); let mut left_func_dependencies = self.clone(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Right => { - // Add offset to right schema + // Add offset to right schema: right_func_dependencies.add_offset(left_cols_len); - // result may have multiple values, hence set dependency mode to multi + // Result may have multiple values, update the dependency mode: left_func_dependencies = left_func_dependencies.with_dependency(Dependency::Multi); right_func_dependencies = right_func_dependencies.with_dependency(Dependency::Multi); if *join_type == JoinType::Left { - // downgrade right side, since this side may have additional null values + // Downgrade the right side, since it may have additional NULL values: right_func_dependencies.downgrade_dependencies(); } else if *join_type == JoinType::Right { - // downgrade left side, since this side may have additional null values + // Downgrade the left side, since it may have additional NULL values: left_func_dependencies.downgrade_dependencies(); } - // Combine left and right functional dependencies + // Combine left and right functional dependencies: left_func_dependencies.extend(right_func_dependencies); left_func_dependencies } JoinType::LeftSemi | JoinType::LeftAnti => { - // LeftSemi and LeftAnti preserves functional dependencies of left side. + // These joins preserve functional dependencies of the left side: left_func_dependencies } JoinType::RightSemi | JoinType::RightAnti => { - // RightSemi and RightAnti preserves functional dependencies of right side. + // These joins preserve functional dependencies of the right side: right_func_dependencies } JoinType::Full => { - // All of the functional dependencies are lost. + // All of the functional dependencies are lost in a FULL join: FunctionalDependencies::empty() } } } - // This case occurs, when current dependency gets a new null value. - // If dependency is UNIQUE(e.g nullable), new null value invalidates dependency - // If dependency is PRIMARY KEY(e.g not nullable), new null value turns it into - // UNIQUE mode. + /// This function downgrades a functional dependency when nullability becomes + /// a possibility: + /// - If the dependency in question is UNIQUE (i.e. nullable), a new null value + /// invalidates the dependency. + /// - If the dependency in question is PRIMARY KEY (i.e. not nullable), a new + /// null value turns it into UNIQUE mode. fn downgrade_dependencies(&mut self) { - // Delete nullable dependencies, since they will no longer be - // functional dependency - self.inner.retain(|item| !item.nullable); - self.inner.iter_mut().for_each(|item| item.nullable = true); + // Delete nullable dependencies, since they are no longer valid: + self.deps.retain(|item| !item.nullable); + self.deps.iter_mut().for_each(|item| item.nullable = true); } + /// This function ensures that functional dependencies involving uniquely + /// occuring determinant keys cover their entire table in terms of + /// dependent columns. pub fn extend_target_indices(&mut self, n_out: usize) { - // if unique cover whole table - let new_target_indices = (0..n_out).collect::>(); - self.inner.iter_mut().for_each( + self.deps.iter_mut().for_each( |FunctionalDependence { mode, target_indices, .. }| { + // If unique, cover the whole table: if *mode == Dependency::Single { - *target_indices = new_target_indices.clone() + *target_indices = (0..n_out).collect::>(); } }, ) } } -// Update entries inside the `entries` vector with their corresponding index -// inside the `proj_indices` vector. +/// Updates entries inside the `entries` vector with their corresponding +/// indices inside the `proj_indices` vector. fn update_elements_with_matching_indices( entries: &[usize], proj_indices: &[usize], @@ -398,15 +410,16 @@ impl DFSchema { ) } + /// Updates functional dependencies when there is a GROUP BY expression. pub fn aggregate_functional_dependencies( &self, group_by_expr_names: &[String], n_out: usize, ) -> FunctionalDependencies { let mut aggregate_func_dependencies = vec![]; - // Association covers whole table: + // Association covers the whole table: let target_indices = (0..n_out).collect::>(); - // Get functional dependencies of the schema. + // Get functional dependencies of the schema: let func_dependencies = self.functional_dependencies(); let fields = self.fields(); for FunctionalDependence { @@ -414,23 +427,23 @@ impl DFSchema { nullable, mode, .. - } in &func_dependencies.inner + } in &func_dependencies.deps { - // keep source_indices in HashSet to prevent duplicate entries + // Keep source indices in a `HashSet` to prevent duplicate entries: let mut new_source_indices = HashSet::new(); let source_field_names = source_indices .iter() .map(|&idx| 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. + // 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); } } - // All of the composite, indices occur in group by expression + // 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( @@ -443,20 +456,19 @@ impl DFSchema { ); } } - // Guaranteed to be unique after aggregation, since it has a single - // expression (e.g. GROUP BY a): + // 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 group - // because it will be added anyway with mode `Dependency::Single` - // flag set. + // 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 0th idx. + // Delete the functional dependency that contains zeroth idx: aggregate_func_dependencies.remove(idx); } - // Add a new functional dependence that is associated with whole table. + // Add a new functional dependency associated with the whole table: aggregate_func_dependencies.push( // Safest behaviour to do is add new functional dependency as nullable. // TODO: use nullable property of group by expr here. @@ -472,25 +484,25 @@ impl DFSchema { group_by_expr_names: &[String], ) -> Option> { let mut combined_target_indices = HashSet::new(); - let func_dependencies = self.functional_dependencies(); + let dependencies = self.functional_dependencies(); let field_names = self .fields() .iter() - .map(|elem| elem.qualified_name()) + .map(|item| item.qualified_name()) .collect::>(); for FunctionalDependence { source_indices, target_indices, .. - } in &func_dependencies.inner + } in &dependencies.deps { let source_key_names = source_indices .iter() .map(|id_key_idx| field_names[*id_key_idx].clone()) .collect::>(); - // group by expression contains a determinant from a functional dependency - // In this case, we can use associated fields after aggregation - // even if they are not part of group by expressions + // If the GROUP BY expression contains a determinant key, we can use + // the associated fields after aggregation even if they are not part + // of the GROUP BY expression. if source_key_names .iter() .all(|source_key_name| group_by_expr_names.contains(source_key_name)) @@ -502,7 +514,7 @@ impl DFSchema { .then_some(combined_target_indices.iter().cloned().collect::>()) } - /// Assign functional dependencies + /// Assigns functional dependencies. pub fn with_functional_dependencies( mut self, functional_dependencies: FunctionalDependencies, diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index b784bb8ed705d..3c658ed6f57b1 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -54,8 +54,8 @@ impl TableSource for DefaultTableSource { self.table_provider.schema() } - /// Get a reference to primary key indices, if a primary key exists. - fn primary_keys(&self) -> Option<&[Constraint]> { + /// Get a reference to applicable constraints, if any exists. + fn constraints(&self) -> Option<&[Constraint]> { self.table_provider.constraints() } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 148d35bf9ae7f..eea37172579e6 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -264,10 +264,9 @@ impl LogicalPlanBuilder { } let schema = table_source.schema(); - let n_field = schema.fields.len(); - let func_dependencies = FunctionalDependencies::new_from_primary_keys( - table_source.primary_keys(), - n_field, + let func_dependencies = FunctionalDependencies::new_from_constraints( + table_source.constraints(), + schema.fields.len(), ); let projected_schema = projection @@ -1233,7 +1232,7 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result>(); - let input_func_dependencies = input.schema().functional_dependencies(); - Ok(input_func_dependencies + Ok(input + .schema() + .functional_dependencies() .project_functional_dependencies(&proj_indices, exprs.len())) } @@ -1267,7 +1267,7 @@ fn contains_grouping_set(group_expr: &[Expr]) -> bool { .any(|expr| matches!(expr, Expr::GroupingSet(_))) } -/// Calculates functional dependencies for aggregate expressions +/// Calculates functional dependencies for aggregate expressions. pub(crate) fn aggregate_functional_dependencies( // Expressions in the GROUP BY clause: group_expr: &[Expr], @@ -1280,7 +1280,7 @@ pub(crate) fn aggregate_functional_dependencies( // whether the GROUP BY in question contains a grouping set expression: // - If so, the functional dependencies will be empty because we cannot guarantee // that GROUP BY expression results will be unique. - // - Otherwise, it may be possible to propagate functional dependencies + // - Otherwise, it may be possible to propagate functional dependencies. if !contains_grouping_set(group_expr) { let group_by_expr_names = group_expr .iter() @@ -1493,7 +1493,7 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { let schema = Arc::new( DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? - // We can use existing functional dependencies: + // We can use the existing functional dependencies: .with_functional_dependencies(input_schema.functional_dependencies().clone()), ); @@ -1506,17 +1506,17 @@ pub fn unnest(input: LogicalPlan, column: Column) -> Result { #[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 arrow::datatypes::{DataType, Field}; use datafusion_common::{ FunctionalDependence, OwnedTableReference, SchemaError, TableReference, }; - use crate::logical_plan::StringifiedPlan; - - use super::*; - use crate::{col, in_subquery, lit, scalar_subquery, sum}; - #[test] fn plan_builder_simple() -> Result<()> { let plan = diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 8e413bbb179c9..fa02ffff4f6e4 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -15,10 +15,6 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::OwnedSchemaReference; -use datafusion_common::{ - parsers::CompressionTypeVariant, Constraint, DFSchemaRef, OwnedTableReference, -}; use std::collections::HashMap; use std::sync::Arc; use std::{ @@ -28,6 +24,11 @@ use std::{ use crate::{Expr, LogicalPlan}; +use datafusion_common::parsers::CompressionTypeVariant; +use datafusion_common::{ + Constraint, DFSchemaRef, OwnedSchemaReference, OwnedTableReference, +}; + /// Various types of DDL (CREATE / DROP) catalog manipulation #[derive(Clone, PartialEq, Eq, Hash)] pub enum DdlStatement { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 8e9d19d9ce121..39f4cd2b4a1a9 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -33,6 +33,9 @@ use crate::utils::{ use crate::{ build_join_schema, Expr, ExprSchemable, TableProviderFilterPushDown, TableSource, }; + +use super::DdlStatement; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeVisitor, VisitRecursion, @@ -41,16 +44,14 @@ use datafusion_common::{ plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, Result, ScalarValue, }; -use std::collections::{HashMap, HashSet}; -use std::fmt::{self, Debug, Display, Formatter}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; - // backwards compatibility pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; -use super::DdlStatement; +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 @@ -1272,7 +1273,8 @@ impl Projection { if expr.len() != schema.fields().len() { return Err(DataFusionError::Plan(format!("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 exprs. + // Update functional dependencies of `input` according to projection + // expressions: let id_key_groups = project_functional_dependencies(&expr, &input)?; let schema = schema.as_ref().clone(); let schema = Arc::new(schema.with_functional_dependencies(id_key_groups)); @@ -1326,7 +1328,8 @@ impl SubqueryAlias { ) -> Result { let alias = alias.into(); let schema: Schema = plan.schema().as_ref().clone().into(); - // Since schema is same, other than qualifier, we can use existing functional dependencies + // Since schema is the same, other than qualifier, we can use existing + // functional dependencies: let func_dependencies = plan.schema().functional_dependencies().clone(); let schema = DFSchemaRef::new( DFSchema::try_from_qualified_schema(&alias, &schema)? @@ -1414,7 +1417,7 @@ impl Window { .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); let metadata = input.schema().metadata().clone(); - // Update functional dependencies for window + // Update functional dependencies for window: let mut window_func_dependencies = input.schema().functional_dependencies().clone(); window_func_dependencies.extend_target_indices(window_fields.len()); diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index e6bebbb488090..84181383ceae4 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -18,8 +18,10 @@ //! Table source use crate::{Expr, LogicalPlan}; + use arrow::datatypes::SchemaRef; use datafusion_common::{Constraint, Result}; + use std::any::Any; /// Indicates whether and how a filter expression can be handled by a @@ -65,7 +67,7 @@ pub trait TableSource: Sync + Send { fn schema(&self) -> SchemaRef; /// Get primary key indices, if one exists. - fn primary_keys(&self) -> Option<&[Constraint]> { + fn constraints(&self) -> Option<&[Constraint]> { None } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 2d11b8fd02248..45f7d91f81314 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -15,12 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashSet; +use std::sync::Arc; + use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use crate::utils::{ check_columns_satisfy_exprs, extract_aliases, rebase_expr, resolve_aliases_to_exprs, resolve_columns, resolve_positions_to_exprs, }; + use datafusion_common::{DataFusionError, Result}; +use datafusion_expr::expr::Alias; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, }; @@ -32,12 +37,8 @@ use datafusion_expr::utils::{ use datafusion_expr::{ Expr, Filter, GroupingSet, LogicalPlan, LogicalPlanBuilder, Partitioning, }; - -use datafusion_expr::expr::Alias; use sqlparser::ast::{Distinct, Expr as SQLExpr, WildcardAdditionalOptions, WindowType}; use sqlparser::ast::{NamedWindowDefinition, Select, SelectItem, TableWithJoins}; -use std::collections::HashSet; -use std::sync::Arc; impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// Generate a logic plan from an SQL select @@ -438,19 +439,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .iter() .map(|group_by_expr| group_by_expr.display_name()) .collect::>>()?; - // get targets that can be used in select, even if they do not occur in aggregation. + // Get targets that can be used in a select, even if they do not occur in aggregation: if let Some(target_indices) = schema.get_target_functional_dependencies(&group_by_expr_names) { - // Calculate dependent fields names with determinant group by expression + // 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 exprs with select_exprs - // If groupby expr is determinant key, we can use its - // dependent keys in select statements also. - // Expand group expression with dependent select expressions. + // 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) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index ba1c22d2bb9a2..c438998a9861e 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -23,6 +23,7 @@ use crate::planner::{ object_name_to_qualifier, ContextProvider, PlannerContext, SqlToRel, }; use crate::utils::normalize_ident; + use arrow_schema::DataType; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ @@ -30,6 +31,7 @@ use datafusion_common::{ DataFusionError, ExprSchema, OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, }; +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; @@ -49,9 +51,8 @@ use sqlparser::ast::{ SetExpr, ShowCreateObject, ShowStatementFilter, Statement, TableConstraint, TableFactor, TableWithJoins, TransactionMode, UnaryOperator, Value, }; - -use datafusion_expr::expr::Placeholder; use sqlparser::parser::ParserError::ParserError; + use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; From ecb6b5bfc37a61b0cc79078f1bdb94ba1c533c24 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jul 2023 17:38:48 +0300 Subject: [PATCH 31/37] Address TODO, fix failing tests --- datafusion/common/src/dfschema.rs | 11 ++++++++--- datafusion/core/src/physical_planner.rs | 2 +- datafusion/optimizer/src/optimizer.rs | 4 ++-- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 0e89c39f3470d..3eb2865c38ce2 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -471,9 +471,14 @@ impl DFSchema { // Add a new functional dependency associated with the whole table: aggregate_func_dependencies.push( // Safest behaviour to do is add new functional dependency as nullable. - // TODO: use nullable property of group by expr here. - FunctionalDependence::new(vec![0], target_indices, true) - .with_mode(Dependency::Single), + // Use nullable property of group by expression (field at index 0). + // It determines functional dependency mode of aggregate expression output. + FunctionalDependence::new( + vec![0], + target_indices, + fields[0].is_nullable(), + ) + .with_mode(Dependency::Single), ); } FunctionalDependencies::new(aggregate_func_dependencies) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 46a8691176216..99f12f5cf9f12 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2203,7 +2203,7 @@ mod tests { dict_id: 0, \ dict_is_ordered: false, \ metadata: {} } }\ - ], metadata: {}, functional_dependencies: FunctionalDependencies { inner: [] } }, \ + ], metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }, \ ExecutionPlan schema: Schema { fields: [\ Field { \ name: \"b\", \ diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 4ff855a014614..920b9ea18f92b 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -493,12 +493,12 @@ mod tests { assert_eq!( "get table_scan rule\ncaused by\n\ Internal error: Optimizer rule 'get table_scan rule' failed, due to generate a different schema, \ - original schema: DFSchema { fields: [], metadata: {}, functional_dependencies: FunctionalDependencies { inner: [] } }, \ + 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: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \ - metadata: {}, functional_dependencies: FunctionalDependencies { inner: [] } }. \ + metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }. \ This was likely caused by a bug in DataFusion's code \ and we would welcome that you file an bug report in our issue tracker", err.to_string() From 8866b30600f17359c909a71c4fa85785d5740d09 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jul 2023 18:24:25 +0300 Subject: [PATCH 32/37] Fix TODO. --- datafusion/common/src/dfschema.rs | 18 +++++++++++------- datafusion/expr/src/logical_plan/builder.rs | 6 +++--- datafusion/expr/src/logical_plan/plan.rs | 7 ++----- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 8f86df92f7544..0a208ec9065de 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -414,14 +414,15 @@ impl DFSchema { pub fn aggregate_functional_dependencies( &self, group_by_expr_names: &[String], - n_out: usize, + aggr_schema: &DFSchema, ) -> FunctionalDependencies { let mut aggregate_func_dependencies = vec![]; + let fields = self.fields(); + let aggregate_fields = aggr_schema.fields(); // Association covers the whole table: - let target_indices = (0..n_out).collect::>(); + let target_indices = (0..aggr_schema.fields().len()).collect::>(); // Get functional dependencies of the schema: let func_dependencies = self.functional_dependencies(); - let fields = self.fields(); for FunctionalDependence { source_indices, nullable, @@ -470,10 +471,13 @@ impl DFSchema { } // Add a new functional dependency associated with the whole table: aggregate_func_dependencies.push( - // Safest behaviour to do is add new functional dependency as nullable. - // TODO: Use nullable property of group by expression - FunctionalDependence::new(vec![0], target_indices, true) - .with_mode(Dependency::Single), + // Use nullable property of the group by expression + FunctionalDependence::new( + vec![0], + target_indices, + aggregate_fields[0].is_nullable(), + ) + .with_mode(Dependency::Single), ); } FunctionalDependencies::new(aggregate_func_dependencies) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 258cf6743560d..4d59eeafded67 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1273,8 +1273,8 @@ pub(crate) fn aggregate_functional_dependencies( group_expr: &[Expr], // Input plan of the aggregate: input: &LogicalPlan, - // Schema field length: - n_out: usize, + // Aggregate schema + aggr_schema: &DFSchema, ) -> Result { // We can do a case analysis on how to propagate functional dependencies based on // whether the GROUP BY in question contains a grouping set expression: @@ -1288,7 +1288,7 @@ pub(crate) fn aggregate_functional_dependencies( .collect::>>()?; let aggregate_func_dependencies = input .schema() - .aggregate_functional_dependencies(&group_by_expr_names, n_out); + .aggregate_functional_dependencies(&group_by_expr_names, aggr_schema); Ok(aggregate_func_dependencies) } else { Ok(FunctionalDependencies::empty()) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 26c7607e25a7b..baef7c1deca28 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1667,11 +1667,8 @@ impl Aggregate { ))); } - let aggregate_func_dependencies = aggregate_functional_dependencies( - &group_expr, - &input, - schema.fields().len(), - )?; + let aggregate_func_dependencies = + aggregate_functional_dependencies(&group_expr, &input, &schema)?; let new_schema = schema.as_ref().clone(); let schema = Arc::new( new_schema.with_functional_dependencies(aggregate_func_dependencies), From 558e7dca4c72f23085c3aba994efba774745ee3b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 27 Jul 2023 10:27:16 +0300 Subject: [PATCH 33/37] Address reviews --- datafusion/common/src/dfschema.rs | 407 +---------------- .../common/src/functional_dependencies.rs | 427 ++++++++++++++++++ datafusion/common/src/lib.rs | 10 +- datafusion/expr/src/logical_plan/builder.rs | 18 +- datafusion/expr/src/logical_plan/plan.rs | 6 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- datafusion/sql/src/select.rs | 4 +- datafusion/sql/src/statement.rs | 9 +- 8 files changed, 457 insertions(+), 426 deletions(-) create mode 100644 datafusion/common/src/functional_dependencies.rs diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 0a208ec9065de..64937eafa158d 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -25,7 +25,9 @@ use std::hash::Hash; use std::sync::Arc; use crate::error::{unqualified_field_not_found, DataFusionError, Result, SchemaError}; -use crate::{field_not_found, Column, JoinType, OwnedTableReference, TableReference}; +use crate::{ + field_not_found, Column, FunctionalDependencies, OwnedTableReference, TableReference, +}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; @@ -33,293 +35,6 @@ use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; /// A reference-counted reference to a `DFSchema`. pub type DFSchemaRef = Arc; -/// This object defines a constraint on a table. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum Constraint { - /// Columns with the given indices form a composite primary key (they are - /// jointly unique and not nullable): - PrimaryKey(Vec), - /// Columns with the given indices form a composite unique key: - Unique(Vec), -} - -/// This object defines a functional dependence in the schema. A functional -/// dependence defines a relationship between determinant keys and dependent -/// columns. A determinant key is a column, or a set of columns, whose value -/// uniquely determines values of some other (dependent) columns. If two rows -/// have the same determinant key, dependent columns in these rows are -/// necessarily the same. If the determinant key is unique, the set of -/// dependent columns is equal to the entire schema and the determinant key can -/// serve as a primary key. Note that a primary key may "downgrade" into a -/// determinant key due to an operation such as a join, and this object is -/// used to track dependence relationships in such cases. For more information -/// on functional dependencies, see: -/// -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct FunctionalDependence { - // Column indices of the (possibly composite) determinant key: - pub source_indices: Vec, - // Column indices of dependent column(s): - pub target_indices: Vec, - /// Flag indicating whether one of the `source_indices` can receive NULL values. - /// For a data source, if the constraint in question is `Constraint::Unique`, - /// this flag is `true`. If the constraint in question is `Constraint::PrimaryKey`, - /// this flag is `false`. - /// Note that as the schema changes between different stages in a plan, - /// such as after LEFT JOIN or RIGHT JOIN operations, this property may - /// change. - pub nullable: bool, - // The functional dependency mode: - pub mode: Dependency, -} - -/// Describes functional dependency mode. -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub enum Dependency { - Single, // A determinant key may occur only once. - Multi, // A determinant key may occur multiple times (in multiple rows). -} - -impl FunctionalDependence { - // Creates a new functional dependence. - pub fn new( - source_indices: Vec, - target_indices: Vec, - nullable: bool, - ) -> Self { - Self { - source_indices, - target_indices, - nullable, - // Start with the least restrictive mode by default: - mode: Dependency::Multi, - } - } - - pub fn with_mode(mut self, mode: Dependency) -> Self { - self.mode = mode; - self - } -} - -/// This object encapsulates all functional dependencies in a given relation. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct FunctionalDependencies { - deps: Vec, -} - -impl FunctionalDependencies { - /// Creates an empty `FunctionalDependencies` object. - pub fn empty() -> Self { - Self { deps: vec![] } - } - - /// Creates a new `FunctionalDependencies` object from a vector of - /// `FunctionalDependence` objects. - pub fn new(dependencies: Vec) -> Self { - Self { deps: dependencies } - } - - /// Creates a new `FunctionalDependencies` object from the given constraints. - pub fn new_from_constraints( - constraints: Option<&[Constraint]>, - n_field: usize, - ) -> Self { - if let Some(constraints) = constraints { - // Construct dependency objects based on each individual constraint: - let dependencies = constraints - .iter() - .map(|constraint| { - // All the field indices are associated with the whole table - // since we are dealing with table level constraints: - let dependency = match constraint { - Constraint::PrimaryKey(indices) => FunctionalDependence::new( - indices.to_vec(), - (0..n_field).collect::>(), - false, - ), - Constraint::Unique(indices) => FunctionalDependence::new( - indices.to_vec(), - (0..n_field).collect::>(), - true, - ), - }; - // As primary keys are guaranteed to be unique, set the - // functional dependency mode to `Dependency::Single`: - dependency.with_mode(Dependency::Single) - }) - .collect::>(); - Self::new(dependencies) - } else { - // There is no constraint, return an empty object: - Self::empty() - } - } - - pub fn with_dependency(mut self, mode: Dependency) -> Self { - self.deps.iter_mut().for_each(|item| item.mode = mode); - self - } - - /// Merges the given functional dependencies with these. - pub fn extend(&mut self, other: FunctionalDependencies) { - self.deps.extend(other.deps); - } - - /// Adds the `offset` value to `source_indices` and `target_indices` for - /// each functional dependency. - pub fn add_offset(&mut self, offset: usize) { - self.deps.iter_mut().for_each( - |FunctionalDependence { - source_indices, - target_indices, - .. - }| { - *source_indices = add_offset_to_vec(source_indices, offset); - *target_indices = add_offset_to_vec(target_indices, offset); - }, - ) - } - - /// Updates `source_indices` and `target_indices` of each functional - /// dependence using the index mapping given in `proj_indices`. - /// - /// Assume that `proj_indices` is \[2, 5, 8\] and we have a functional - /// dependence \[5\] (`source_indices`) -> \[5, 8\] (`target_indices`). - /// In the updated schema, fields at indices \[2, 5, 8\] will transform - /// to \[0, 1, 2\]. Therefore, the resulting functional dependence will - /// be \[1\] -> \[1, 2\]. - pub fn project_functional_dependencies( - &self, - proj_indices: &[usize], - // The argument `n_out` denotes the schema field length, which is needed - // to correctly associate a `Single`-mode dependence with the whole table. - n_out: usize, - ) -> FunctionalDependencies { - let mut projected_func_dependencies = vec![]; - for FunctionalDependence { - source_indices, - target_indices, - nullable, - mode, - } in &self.deps - { - let new_source_indices = - update_elements_with_matching_indices(source_indices, proj_indices); - let new_target_indices = if *mode == Dependency::Single { - // Associate with all of the fields in the schema: - (0..n_out).collect() - } else { - // Update associations according to projection: - update_elements_with_matching_indices(target_indices, proj_indices) - }; - // All of the composite indices should still be valid after projection; - // otherwise, functional dependency cannot be propagated. - if new_source_indices.len() == source_indices.len() { - let new_func_dependence = FunctionalDependence::new( - new_source_indices, - new_target_indices, - *nullable, - ) - .with_mode(*mode); - projected_func_dependencies.push(new_func_dependence); - } - } - FunctionalDependencies::new(projected_func_dependencies) - } - - /// This function joins this set of functional dependencies with the `other` - /// according to the given `join_type`. - pub fn join( - &self, - other: &FunctionalDependencies, - join_type: &JoinType, - left_cols_len: usize, - ) -> FunctionalDependencies { - // Get mutable copies of left and right side dependencies: - let mut right_func_dependencies = other.clone(); - let mut left_func_dependencies = self.clone(); - - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right => { - // Add offset to right schema: - right_func_dependencies.add_offset(left_cols_len); - - // Result may have multiple values, update the dependency mode: - left_func_dependencies = - left_func_dependencies.with_dependency(Dependency::Multi); - right_func_dependencies = - right_func_dependencies.with_dependency(Dependency::Multi); - - if *join_type == JoinType::Left { - // Downgrade the right side, since it may have additional NULL values: - right_func_dependencies.downgrade_dependencies(); - } else if *join_type == JoinType::Right { - // Downgrade the left side, since it may have additional NULL values: - left_func_dependencies.downgrade_dependencies(); - } - // Combine left and right functional dependencies: - left_func_dependencies.extend(right_func_dependencies); - left_func_dependencies - } - JoinType::LeftSemi | JoinType::LeftAnti => { - // These joins preserve functional dependencies of the left side: - left_func_dependencies - } - JoinType::RightSemi | JoinType::RightAnti => { - // These joins preserve functional dependencies of the right side: - right_func_dependencies - } - JoinType::Full => { - // All of the functional dependencies are lost in a FULL join: - FunctionalDependencies::empty() - } - } - } - - /// This function downgrades a functional dependency when nullability becomes - /// a possibility: - /// - If the dependency in question is UNIQUE (i.e. nullable), a new null value - /// invalidates the dependency. - /// - If the dependency in question is PRIMARY KEY (i.e. not nullable), a new - /// null value turns it into UNIQUE mode. - fn downgrade_dependencies(&mut self) { - // Delete nullable dependencies, since they are no longer valid: - self.deps.retain(|item| !item.nullable); - self.deps.iter_mut().for_each(|item| item.nullable = true); - } - - /// This function ensures that functional dependencies involving uniquely - /// occuring determinant keys cover their entire table in terms of - /// dependent columns. - pub fn extend_target_indices(&mut self, n_out: usize) { - self.deps.iter_mut().for_each( - |FunctionalDependence { - mode, - target_indices, - .. - }| { - // If unique, cover the whole table: - if *mode == Dependency::Single { - *target_indices = (0..n_out).collect::>(); - } - }, - ) - } -} - -/// Updates entries inside the `entries` vector with their corresponding -/// indices inside the `proj_indices` vector. -fn update_elements_with_matching_indices( - entries: &[usize], - proj_indices: &[usize], -) -> Vec { - entries - .iter() - .filter_map(|val| proj_indices.iter().position(|proj_idx| proj_idx == val)) - .collect() -} - /// DFSchema wraps an Arrow schema and adds relation names #[derive(Debug, Clone, PartialEq, Eq)] pub struct DFSchema { @@ -410,114 +125,6 @@ impl DFSchema { ) } - /// Updates functional dependencies when there is a GROUP BY expression. - pub fn aggregate_functional_dependencies( - &self, - group_by_expr_names: &[String], - aggr_schema: &DFSchema, - ) -> FunctionalDependencies { - let mut aggregate_func_dependencies = vec![]; - let fields = self.fields(); - let aggregate_fields = aggr_schema.fields(); - // Association covers the whole table: - let target_indices = (0..aggr_schema.fields().len()).collect::>(); - // Get functional dependencies of the schema: - let func_dependencies = self.functional_dependencies(); - for FunctionalDependence { - source_indices, - nullable, - mode, - .. - } in &func_dependencies.deps - { - // Keep source indices in a `HashSet` to prevent duplicate entries: - let mut new_source_indices = HashSet::new(); - let source_field_names = source_indices - .iter() - .map(|&idx| 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); - } - } - // 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(), - target_indices.clone(), - *nullable, - ) - // input uniqueness stays the same when GROUP BY matches with input functional dependence determinants - .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); - } - // Add a new functional dependency associated with the whole table: - aggregate_func_dependencies.push( - // Use nullable property of the group by expression - FunctionalDependence::new( - vec![0], - target_indices, - aggregate_fields[0].is_nullable(), - ) - .with_mode(Dependency::Single), - ); - } - FunctionalDependencies::new(aggregate_func_dependencies) - } - - pub fn get_target_functional_dependencies( - &self, - group_by_expr_names: &[String], - ) -> Option> { - let mut combined_target_indices = HashSet::new(); - let dependencies = self.functional_dependencies(); - let field_names = self - .fields() - .iter() - .map(|item| item.qualified_name()) - .collect::>(); - for FunctionalDependence { - source_indices, - target_indices, - .. - } in &dependencies.deps - { - let source_key_names = source_indices - .iter() - .map(|id_key_idx| field_names[*id_key_idx].clone()) - .collect::>(); - // If the GROUP BY expression contains a determinant key, we can use - // the associated fields after aggregation even if they are not part - // of the GROUP BY expression. - if source_key_names - .iter() - .all(|source_key_name| group_by_expr_names.contains(source_key_name)) - { - combined_target_indices.extend(target_indices.iter()); - } - } - (!combined_target_indices.is_empty()) - .then_some(combined_target_indices.iter().cloned().collect::>()) - } - /// Assigns functional dependencies. pub fn with_functional_dependencies( mut self, @@ -1191,14 +798,6 @@ impl SchemaExt for Schema { } } -/// Adds `offset` value to each entry inside `in_data`. -fn add_offset_to_vec>( - in_data: &[T], - offset: T, -) -> Vec { - in_data.iter().map(|&item| item + offset).collect() -} - #[cfg(test)] mod tests { use crate::assert_contains; diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs new file mode 100644 index 0000000000000..d1cf41ee267ab --- /dev/null +++ b/datafusion/common/src/functional_dependencies.rs @@ -0,0 +1,427 @@ +// 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. + +//! FunctionalDependencies keeps track of functional dependencies +//! inside DFSchema. + +use crate::{DFSchema, JoinType}; +use std::collections::HashSet; + +/// This object defines a constraint on a table. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub enum Constraint { + /// Columns with the given indices form a composite primary key (they are + /// jointly unique and not nullable): + PrimaryKey(Vec), + /// Columns with the given indices form a composite unique key: + Unique(Vec), +} + +/// This object defines a functional dependence in the schema. A functional +/// dependence defines a relationship between determinant keys and dependent +/// columns. A determinant key is a column, or a set of columns, whose value +/// uniquely determines values of some other (dependent) columns. If two rows +/// have the same determinant key, dependent columns in these rows are +/// necessarily the same. If the determinant key is unique, the set of +/// dependent columns is equal to the entire schema and the determinant key can +/// serve as a primary key. Note that a primary key may "downgrade" into a +/// determinant key due to an operation such as a join, and this object is +/// used to track dependence relationships in such cases. For more information +/// on functional dependencies, see: +/// +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct FunctionalDependence { + // Column indices of the (possibly composite) determinant key: + pub source_indices: Vec, + // Column indices of dependent column(s): + pub target_indices: Vec, + /// Flag indicating whether one of the `source_indices` can receive NULL values. + /// For a data source, if the constraint in question is `Constraint::Unique`, + /// this flag is `true`. If the constraint in question is `Constraint::PrimaryKey`, + /// this flag is `false`. + /// Note that as the schema changes between different stages in a plan, + /// such as after LEFT JOIN or RIGHT JOIN operations, this property may + /// change. + pub nullable: bool, + // The functional dependency mode: + pub mode: Dependency, +} + +/// Describes functional dependency mode. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum Dependency { + Single, // A determinant key may occur only once. + Multi, // A determinant key may occur multiple times (in multiple rows). +} + +impl FunctionalDependence { + // Creates a new functional dependence. + pub fn new( + source_indices: Vec, + target_indices: Vec, + nullable: bool, + ) -> Self { + Self { + source_indices, + target_indices, + nullable, + // Start with the least restrictive mode by default: + mode: Dependency::Multi, + } + } + + pub fn with_mode(mut self, mode: Dependency) -> Self { + self.mode = mode; + self + } +} + +/// This object encapsulates all functional dependencies in a given relation. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct FunctionalDependencies { + deps: Vec, +} + +impl FunctionalDependencies { + /// Creates an empty `FunctionalDependencies` object. + pub fn empty() -> Self { + Self { deps: vec![] } + } + + /// Creates a new `FunctionalDependencies` object from a vector of + /// `FunctionalDependence` objects. + pub fn new(dependencies: Vec) -> Self { + Self { deps: dependencies } + } + + /// Creates a new `FunctionalDependencies` object from the given constraints. + pub fn new_from_constraints( + constraints: Option<&[Constraint]>, + n_field: usize, + ) -> Self { + if let Some(constraints) = constraints { + // Construct dependency objects based on each individual constraint: + let dependencies = constraints + .iter() + .map(|constraint| { + // All the field indices are associated with the whole table + // since we are dealing with table level constraints: + let dependency = match constraint { + Constraint::PrimaryKey(indices) => FunctionalDependence::new( + indices.to_vec(), + (0..n_field).collect::>(), + false, + ), + Constraint::Unique(indices) => FunctionalDependence::new( + indices.to_vec(), + (0..n_field).collect::>(), + true, + ), + }; + // As primary keys are guaranteed to be unique, set the + // functional dependency mode to `Dependency::Single`: + dependency.with_mode(Dependency::Single) + }) + .collect::>(); + Self::new(dependencies) + } else { + // There is no constraint, return an empty object: + Self::empty() + } + } + + pub fn with_dependency(mut self, mode: Dependency) -> Self { + self.deps.iter_mut().for_each(|item| item.mode = mode); + self + } + + /// Merges the given functional dependencies with these. + pub fn extend(&mut self, other: FunctionalDependencies) { + self.deps.extend(other.deps); + } + + /// Adds the `offset` value to `source_indices` and `target_indices` for + /// each functional dependency. + pub fn add_offset(&mut self, offset: usize) { + self.deps.iter_mut().for_each( + |FunctionalDependence { + source_indices, + target_indices, + .. + }| { + *source_indices = add_offset_to_vec(source_indices, offset); + *target_indices = add_offset_to_vec(target_indices, offset); + }, + ) + } + + /// Updates `source_indices` and `target_indices` of each functional + /// dependence using the index mapping given in `proj_indices`. + /// + /// Assume that `proj_indices` is \[2, 5, 8\] and we have a functional + /// dependence \[5\] (`source_indices`) -> \[5, 8\] (`target_indices`). + /// In the updated schema, fields at indices \[2, 5, 8\] will transform + /// to \[0, 1, 2\]. Therefore, the resulting functional dependence will + /// be \[1\] -> \[1, 2\]. + pub fn project_functional_dependencies( + &self, + proj_indices: &[usize], + // The argument `n_out` denotes the schema field length, which is needed + // to correctly associate a `Single`-mode dependence with the whole table. + n_out: usize, + ) -> FunctionalDependencies { + let mut projected_func_dependencies = vec![]; + for FunctionalDependence { + source_indices, + target_indices, + nullable, + mode, + } in &self.deps + { + let new_source_indices = + update_elements_with_matching_indices(source_indices, proj_indices); + let new_target_indices = if *mode == Dependency::Single { + // Associate with all of the fields in the schema: + (0..n_out).collect() + } else { + // Update associations according to projection: + update_elements_with_matching_indices(target_indices, proj_indices) + }; + // All of the composite indices should still be valid after projection; + // otherwise, functional dependency cannot be propagated. + if new_source_indices.len() == source_indices.len() { + let new_func_dependence = FunctionalDependence::new( + new_source_indices, + new_target_indices, + *nullable, + ) + .with_mode(*mode); + projected_func_dependencies.push(new_func_dependence); + } + } + FunctionalDependencies::new(projected_func_dependencies) + } + + /// This function joins this set of functional dependencies with the `other` + /// according to the given `join_type`. + pub fn join( + &self, + other: &FunctionalDependencies, + join_type: &JoinType, + left_cols_len: usize, + ) -> FunctionalDependencies { + // Get mutable copies of left and right side dependencies: + let mut right_func_dependencies = other.clone(); + let mut left_func_dependencies = self.clone(); + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right => { + // Add offset to right schema: + right_func_dependencies.add_offset(left_cols_len); + + // Result may have multiple values, update the dependency mode: + left_func_dependencies = + left_func_dependencies.with_dependency(Dependency::Multi); + right_func_dependencies = + right_func_dependencies.with_dependency(Dependency::Multi); + + if *join_type == JoinType::Left { + // Downgrade the right side, since it may have additional NULL values: + right_func_dependencies.downgrade_dependencies(); + } else if *join_type == JoinType::Right { + // Downgrade the left side, since it may have additional NULL values: + left_func_dependencies.downgrade_dependencies(); + } + // Combine left and right functional dependencies: + left_func_dependencies.extend(right_func_dependencies); + left_func_dependencies + } + JoinType::LeftSemi | JoinType::LeftAnti => { + // These joins preserve functional dependencies of the left side: + left_func_dependencies + } + JoinType::RightSemi | JoinType::RightAnti => { + // These joins preserve functional dependencies of the right side: + right_func_dependencies + } + JoinType::Full => { + // All of the functional dependencies are lost in a FULL join: + FunctionalDependencies::empty() + } + } + } + + /// This function downgrades a functional dependency when nullability becomes + /// a possibility: + /// - If the dependency in question is UNIQUE (i.e. nullable), a new null value + /// invalidates the dependency. + /// - If the dependency in question is PRIMARY KEY (i.e. not nullable), a new + /// null value turns it into UNIQUE mode. + fn downgrade_dependencies(&mut self) { + // Delete nullable dependencies, since they are no longer valid: + self.deps.retain(|item| !item.nullable); + self.deps.iter_mut().for_each(|item| item.nullable = true); + } + + /// This function ensures that functional dependencies involving uniquely + /// occuring determinant keys cover their entire table in terms of + /// dependent columns. + pub fn extend_target_indices(&mut self, n_out: usize) { + self.deps.iter_mut().for_each( + |FunctionalDependence { + mode, + target_indices, + .. + }| { + // If unique, cover the whole table: + if *mode == Dependency::Single { + *target_indices = (0..n_out).collect::>(); + } + }, + ) + } +} + +/// Calculates functional dependencies for aggregate output, when there is a GROUP BY expression. +pub fn aggregate_functional_dependencies( + aggr_input_schema: &DFSchema, + group_by_expr_names: &[String], + aggr_schema: &DFSchema, +) -> FunctionalDependencies { + let mut aggregate_func_dependencies = vec![]; + let aggr_input_fields = aggr_input_schema.fields(); + let aggr_fields = aggr_schema.fields(); + // Association covers the whole table: + let target_indices = (0..aggr_schema.fields().len()).collect::>(); + // Get functional dependencies of the schema: + let func_dependencies = aggr_input_schema.functional_dependencies(); + for FunctionalDependence { + source_indices, + nullable, + mode, + .. + } in &func_dependencies.deps + { + // Keep source indices in a `HashSet` to prevent duplicate entries: + let mut new_source_indices = HashSet::new(); + 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); + } + } + // 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(), + target_indices.clone(), + *nullable, + ) + // input uniqueness stays the same when GROUP BY matches with input functional dependence determinants + .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); + } + // Add a new functional dependency associated with the whole table: + aggregate_func_dependencies.push( + // Use nullable property of the group by expression + FunctionalDependence::new( + vec![0], + target_indices, + aggr_fields[0].is_nullable(), + ) + .with_mode(Dependency::Single), + ); + } + FunctionalDependencies::new(aggregate_func_dependencies) +} + +/// Returns target indices, for the determinant keys that are inside +/// group by expressions. +pub fn get_target_functional_dependencies( + schema: &DFSchema, + group_by_expr_names: &[String], +) -> Option> { + let mut combined_target_indices = HashSet::new(); + let dependencies = schema.functional_dependencies(); + let field_names = schema + .fields() + .iter() + .map(|item| item.qualified_name()) + .collect::>(); + for FunctionalDependence { + source_indices, + target_indices, + .. + } in &dependencies.deps + { + let source_key_names = source_indices + .iter() + .map(|id_key_idx| field_names[*id_key_idx].clone()) + .collect::>(); + // If the GROUP BY expression contains a determinant key, we can use + // the associated fields after aggregation even if they are not part + // of the GROUP BY expression. + if source_key_names + .iter() + .all(|source_key_name| group_by_expr_names.contains(source_key_name)) + { + combined_target_indices.extend(target_indices.iter()); + } + } + (!combined_target_indices.is_empty()) + .then_some(combined_target_indices.iter().cloned().collect::>()) +} + +/// Updates entries inside the `entries` vector with their corresponding +/// indices inside the `proj_indices` vector. +fn update_elements_with_matching_indices( + entries: &[usize], + proj_indices: &[usize], +) -> Vec { + entries + .iter() + .filter_map(|val| proj_indices.iter().position(|proj_idx| proj_idx == val)) + .collect() +} + +/// Adds `offset` value to each entry inside `in_data`. +fn add_offset_to_vec>( + in_data: &[T], + offset: T, +) -> Vec { + in_data.iter().map(|&item| item + offset).collect() +} diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 4dbafc91faf2c..bea3069c47462 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -23,6 +23,7 @@ pub mod delta; mod dfschema; pub mod display; mod error; +mod functional_dependencies; mod join_type; pub mod parsers; #[cfg(feature = "pyarrow")] @@ -36,14 +37,15 @@ pub mod tree_node; pub mod utils; pub use column::Column; -pub use dfschema::{ - Constraint, DFField, DFSchema, DFSchemaRef, Dependency, ExprSchema, - FunctionalDependence, FunctionalDependencies, SchemaExt, ToDFSchema, -}; +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 functional_dependencies::{ + aggregate_functional_dependencies, get_target_functional_dependencies, Constraint, + Dependency, FunctionalDependence, FunctionalDependencies, +}; pub use join_type::{JoinConstraint, JoinType}; pub use scalar::{ScalarType, ScalarValue}; pub use schema_reference::{OwnedSchemaReference, SchemaReference}; diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 4d59eeafded67..ff5a300a6d18a 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -41,9 +41,9 @@ use crate::{ }; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ - display::ToStringifiedPlan, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - FunctionalDependencies, OwnedTableReference, Result, ScalarValue, TableReference, - ToDFSchema, + aggregate_functional_dependencies, display::ToStringifiedPlan, Column, DFField, + DFSchema, DFSchemaRef, DataFusionError, FunctionalDependencies, OwnedTableReference, + Result, ScalarValue, TableReference, ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -1234,7 +1234,7 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result { @@ -1268,7 +1268,7 @@ fn contains_grouping_set(group_expr: &[Expr]) -> bool { } /// Calculates functional dependencies for aggregate expressions. -pub(crate) fn aggregate_functional_dependencies( +pub(crate) fn calc_func_dependencies_for_aggregate( // Expressions in the GROUP BY clause: group_expr: &[Expr], // Input plan of the aggregate: @@ -1286,9 +1286,11 @@ pub(crate) fn aggregate_functional_dependencies( .iter() .map(|item| item.display_name()) .collect::>>()?; - let aggregate_func_dependencies = input - .schema() - .aggregate_functional_dependencies(&group_by_expr_names, aggr_schema); + let aggregate_func_dependencies = aggregate_functional_dependencies( + input.schema(), + &group_by_expr_names, + aggr_schema, + ); Ok(aggregate_func_dependencies) } else { Ok(FunctionalDependencies::empty()) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index baef7c1deca28..ff2fb963005b7 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -18,7 +18,7 @@ //! Logical plan types use crate::builder::{ - aggregate_functional_dependencies, project_functional_dependencies, + calc_func_dependencies_for_aggregate, calc_func_dependencies_for_project, }; use crate::expr::{Alias, Exists, InSubquery, Placeholder}; use crate::expr_rewriter::create_col_from_scalar_expr; @@ -1306,7 +1306,7 @@ impl Projection { } // Update functional dependencies of `input` according to projection // expressions: - let id_key_groups = project_functional_dependencies(&expr, &input)?; + 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 { @@ -1668,7 +1668,7 @@ impl Aggregate { } let aggregate_func_dependencies = - aggregate_functional_dependencies(&group_expr, &input, &schema)?; + calc_func_dependencies_for_aggregate(&group_expr, &input, &schema)?; let new_schema = schema.as_ref().clone(); let schema = Arc::new( new_schema.with_functional_dependencies(aggregate_func_dependencies), diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 0f6b5e2451832..95061e38540c7 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -262,7 +262,7 @@ fn rewrite_schema(schema: &DFSchema) -> DFSchemaRef { ) }) .collect::>(); - DFSchemaRef::new( + Arc::new( DFSchema::new_with_metadata(new_fields, schema.metadata().clone()) .unwrap() .with_functional_dependencies(schema.functional_dependencies().clone()), diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 45f7d91f81314..3aed0b6e3020b 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -24,7 +24,7 @@ use crate::utils::{ resolve_columns, resolve_positions_to_exprs, }; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{get_target_functional_dependencies, DataFusionError, Result}; use datafusion_expr::expr::Alias; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, @@ -441,7 +441,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .collect::>>()?; // Get targets that can be used in a select, even if they do not occur in aggregation: if let Some(target_indices) = - schema.get_target_functional_dependencies(&group_by_expr_names) + get_target_functional_dependencies(schema, &group_by_expr_names) { // Calculate dependent fields names with determinant GROUP BY expression: let associated_field_names = target_indices diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index c438998a9861e..dc98f5d96f878 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -164,7 +164,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let constraints = - Self::primary_key_from_constraints(&constraints, plan.schema())?; + Self::constraints_from_source(&constraints, plan.schema())?; Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { @@ -185,7 +185,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let plan = LogicalPlan::EmptyRelation(plan); let constraints = - Self::primary_key_from_constraints(&constraints, plan.schema())?; + Self::constraints_from_source(&constraints, plan.schema())?; Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { name: self.object_name_to_table_reference(name)?, @@ -1163,7 +1163,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .is_ok() } - fn primary_key_from_constraints( + /// Convert each `TableConstraint` to corresponding `Constraint` + fn constraints_from_source( constraints: &[TableConstraint], df_schema: &DFSchemaRef, ) -> Result> { @@ -1175,7 +1176,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { is_primary, .. } => { - // Get primary key indices in the schema: + // Get primary key and/or unique indices in the schema: let indices = columns .iter() .map(|pk| { From c4cd98b8a50882840b8c4837c74c965e2ce86ebc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 27 Jul 2023 10:53:34 +0300 Subject: [PATCH 34/37] Implement Constraints struct --- .../common/src/functional_dependencies.rs | 101 +++++++++++++++++- datafusion/common/src/lib.rs | 2 +- .../src/datasource/default_table_source.rs | 4 +- datafusion/core/src/datasource/memory.rs | 10 +- datafusion/core/src/datasource/provider.rs | 4 +- datafusion/expr/src/logical_plan/ddl.rs | 12 +-- datafusion/expr/src/table_source.rs | 4 +- datafusion/expr/src/utils.rs | 6 +- datafusion/sql/src/query.rs | 4 +- datafusion/sql/src/statement.rs | 71 ++---------- 10 files changed, 128 insertions(+), 90 deletions(-) diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index d1cf41ee267ab..b7a8e768ec78b 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -18,12 +18,14 @@ //! FunctionalDependencies keeps track of functional dependencies //! inside DFSchema. -use crate::{DFSchema, JoinType}; +use crate::{DFSchema, DFSchemaRef, DataFusionError, JoinType, Result}; +use sqlparser::ast::TableConstraint; use std::collections::HashSet; +use std::fmt::{Display, Formatter}; /// This object defines a constraint on a table. #[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum Constraint { +enum Constraint { /// Columns with the given indices form a composite primary key (they are /// jointly unique and not nullable): PrimaryKey(Vec), @@ -31,6 +33,97 @@ pub enum Constraint { Unique(Vec), } +/// This object encapsulates a list of functional constraints: +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct Constraints { + inner: Vec, +} + +impl Constraints { + /// Create empty constraints + pub fn empty() -> Self { + Constraints::new(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 { + Self { inner: constraints } + } + + /// Convert each `TableConstraint` to corresponding `Constraint` + pub fn new_from_table_constraints( + constraints: &[TableConstraint], + df_schema: &DFSchemaRef, + ) -> Result { + let constraints = constraints + .iter() + .map(|c: &TableConstraint| match c { + TableConstraint::Unique { + columns, + is_primary, + .. + } => { + // Get primary key and/or unique indices in the schema: + let indices = columns + .iter() + .map(|pk| { + let idx = df_schema + .fields() + .iter() + .position(|item| { + item.qualified_name() == pk.value.clone() + }) + .ok_or_else(|| { + DataFusionError::Execution( + "Primary key doesn't exist".to_string(), + ) + })?; + Ok(idx) + }) + .collect::>>()?; + Ok(if *is_primary { + Constraint::PrimaryKey(indices) + } else { + 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(), + )), + }) + .collect::>>()?; + Ok(Constraints::new(constraints)) + } + + /// Check whether constraints is empty + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } +} + +impl Display for Constraints { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + let pk: Vec = self.inner.iter().map(|c| format!("{:?}", c)).collect(); + let pk = pk.join(", "); + if !pk.is_empty() { + write!(f, " constraints=[{pk}]") + } else { + write!(f, "") + } + } +} + /// This object defines a functional dependence in the schema. A functional /// dependence defines a relationship between determinant keys and dependent /// columns. A determinant key is a column, or a set of columns, whose value @@ -110,10 +203,10 @@ impl FunctionalDependencies { /// Creates a new `FunctionalDependencies` object from the given constraints. pub fn new_from_constraints( - constraints: Option<&[Constraint]>, + constraints: Option<&Constraints>, n_field: usize, ) -> Self { - if let Some(constraints) = constraints { + if let Some(Constraints { inner: constraints }) = constraints { // Construct dependency objects based on each individual constraint: let dependencies = constraints .iter() diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index bea3069c47462..7a46f28b50736 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -43,7 +43,7 @@ pub use error::{ SharedResult, }; pub use functional_dependencies::{ - aggregate_functional_dependencies, get_target_functional_dependencies, Constraint, + aggregate_functional_dependencies, get_target_functional_dependencies, Constraints, Dependency, FunctionalDependence, FunctionalDependencies, }; pub use join_type::{JoinConstraint, JoinType}; diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 3c658ed6f57b1..bd3e832804cbc 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use crate::datasource::TableProvider; use arrow::datatypes::SchemaRef; -use datafusion_common::{Constraint, DataFusionError}; +use datafusion_common::{Constraints, DataFusionError}; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; /// DataFusion default table source, wrapping TableProvider @@ -55,7 +55,7 @@ impl TableSource for DefaultTableSource { } /// Get a reference to applicable constraints, if any exists. - fn constraints(&self) -> Option<&[Constraint]> { + fn constraints(&self) -> Option<&Constraints> { self.table_provider.constraints() } diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 929fe857cd56a..4b6653c6889f3 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -26,7 +26,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use async_trait::async_trait; -use datafusion_common::{Constraint, SchemaExt}; +use datafusion_common::{Constraints, SchemaExt}; use datafusion_execution::TaskContext; use tokio::sync::RwLock; use tokio::task::JoinSet; @@ -52,7 +52,7 @@ pub type PartitionData = Arc>>; pub struct MemTable { schema: SchemaRef, pub(crate) batches: Vec, - constraints: Option>, + constraints: Option, } impl MemTable { @@ -82,7 +82,7 @@ impl MemTable { } /// Assign constraints - pub fn with_constraints(mut self, constraints: Vec) -> Self { + pub fn with_constraints(mut self, constraints: Constraints) -> Self { if !constraints.is_empty() { self.constraints = Some(constraints); } @@ -163,8 +163,8 @@ impl TableProvider for MemTable { self.schema.clone() } - fn constraints(&self) -> Option<&[Constraint]> { - self.constraints.as_deref() + fn constraints(&self) -> Option<&Constraints> { + self.constraints.as_ref() } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index c4450b4723c44..9c97935105890 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::sync::Arc; use async_trait::async_trait; -use datafusion_common::{Constraint, DataFusionError, Statistics}; +use datafusion_common::{Constraints, DataFusionError, Statistics}; use datafusion_expr::{CreateExternalTable, LogicalPlan}; pub use datafusion_expr::{TableProviderFilterPushDown, TableType}; @@ -42,7 +42,7 @@ pub trait TableProvider: Sync + Send { fn schema(&self) -> SchemaRef; /// Get a reference to the constraints of the table. - fn constraints(&self) -> Option<&[Constraint]> { + 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 fa02ffff4f6e4..b0877a77a4bcd 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -26,7 +26,7 @@ use crate::{Expr, LogicalPlan}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - Constraint, DFSchemaRef, OwnedSchemaReference, OwnedTableReference, + Constraints, DFSchemaRef, OwnedSchemaReference, OwnedTableReference, }; /// Various types of DDL (CREATE / DROP) catalog manipulation @@ -121,13 +121,7 @@ impl DdlStatement { constraints, .. }) => { - let pk: Vec = - constraints.iter().map(|c| format!("{:?}", c)).collect(); - let mut pk = pk.join(", "); - if !pk.is_empty() { - pk = format!(" constraints=[{pk}]"); - } - write!(f, "CreateMemoryTable: {name:?}{pk}") + write!(f, "CreateMemoryTable: {name:?}{constraints:?}") } DdlStatement::CreateView(CreateView { name, .. }) => { write!(f, "CreateView: {name:?}") @@ -224,7 +218,7 @@ pub struct CreateMemoryTable { /// The table name pub name: OwnedTableReference, /// The list of constraints in the schema, such as primary key, unique, etc. - pub constraints: Vec, + pub constraints: Constraints, /// The logical plan pub input: Arc, /// Option to not error if table already exists diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 84181383ceae4..b83ce778133b1 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -20,7 +20,7 @@ use crate::{Expr, LogicalPlan}; use arrow::datatypes::SchemaRef; -use datafusion_common::{Constraint, Result}; +use datafusion_common::{Constraints, Result}; use std::any::Any; @@ -67,7 +67,7 @@ pub trait TableSource: Sync + Send { fn schema(&self) -> SchemaRef; /// Get primary key indices, if one exists. - fn constraints(&self) -> Option<&[Constraint]> { + fn constraints(&self) -> Option<&Constraints> { None } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index f9e90ac575c1e..efaf291398427 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -34,8 +34,8 @@ use datafusion_common::tree_node::{ RewriteRecursion, TreeNode, TreeNodeRewriter, VisitRecursion, }; use datafusion_common::{ - Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, - TableReference, + Column, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, + ScalarValue, TableReference, }; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; use std::cmp::Ordering; @@ -923,7 +923,7 @@ pub fn from_plan( })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { input: Arc::new(inputs[0].clone()), - constraints: vec![], + constraints: Constraints::empty(), name: name.clone(), if_not_exists: *if_not_exists, or_replace: *or_replace, diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 6c4c9095032fa..34b24b0594fa8 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{Constraints, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ CreateMemoryTable, DdlStatement, Expr, LogicalPlan, LogicalPlanBuilder, }; @@ -86,7 +86,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let select_into = select.into.unwrap(); LogicalPlan::Ddl(DdlStatement::CreateMemoryTable(CreateMemoryTable { name: self.object_name_to_table_reference(select_into.name)?, - constraints: Vec::new(), + constraints: Constraints::empty(), input: Arc::new(plan), if_not_exists: false, or_replace: false, diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index dc98f5d96f878..4af32337f77a1 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -27,7 +27,7 @@ use crate::utils::normalize_ident; use arrow_schema::DataType; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - unqualified_field_not_found, Column, Constraint, DFField, DFSchema, DFSchemaRef, + unqualified_field_not_found, Column, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, ExprSchema, OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, }; @@ -48,8 +48,8 @@ use datafusion_expr::{ use sqlparser::ast; use sqlparser::ast::{ Assignment, Expr as SQLExpr, Expr, Ident, ObjectName, ObjectType, Query, SchemaName, - SetExpr, ShowCreateObject, ShowStatementFilter, Statement, TableConstraint, - TableFactor, TableWithJoins, TransactionMode, UnaryOperator, Value, + SetExpr, ShowCreateObject, ShowStatementFilter, Statement, TableFactor, + TableWithJoins, TransactionMode, UnaryOperator, Value, }; use sqlparser::parser::ParserError::ParserError; @@ -163,8 +163,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { plan }; - let constraints = - Self::constraints_from_source(&constraints, plan.schema())?; + let constraints = Constraints::new_from_table_constraints( + &constraints, + plan.schema(), + )?; Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { @@ -184,8 +186,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema, }; let plan = LogicalPlan::EmptyRelation(plan); - let constraints = - Self::constraints_from_source(&constraints, plan.schema())?; + let constraints = Constraints::new_from_table_constraints( + &constraints, + plan.schema(), + )?; Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { name: self.object_name_to_table_reference(name)?, @@ -1162,57 +1166,4 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .get_table_provider(tables_reference) .is_ok() } - - /// Convert each `TableConstraint` to corresponding `Constraint` - fn constraints_from_source( - constraints: &[TableConstraint], - df_schema: &DFSchemaRef, - ) -> Result> { - constraints - .iter() - .map(|c: &TableConstraint| match c { - TableConstraint::Unique { - columns, - is_primary, - .. - } => { - // Get primary key and/or unique indices in the schema: - let indices = columns - .iter() - .map(|pk| { - let idx = df_schema - .fields() - .iter() - .position(|item| { - item.qualified_name() == pk.value.clone() - }) - .ok_or_else(|| { - DataFusionError::Execution( - "Primary key doesn't exist".to_string(), - ) - })?; - Ok(idx) - }) - .collect::>>()?; - Ok(if *is_primary { - Constraint::PrimaryKey(indices) - } else { - 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(), - )), - }) - .collect::>>() - } } From 45ce2494d6cf3672f1f4a3e1a536150730f4ebea Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 27 Jul 2023 11:11:37 +0300 Subject: [PATCH 35/37] Convert some pub functions to private --- datafusion/core/src/execution/context.rs | 2 +- datafusion/expr/src/logical_plan/builder.rs | 71 +------------------- datafusion/expr/src/logical_plan/plan.rs | 72 +++++++++++++++++++-- 3 files changed, 71 insertions(+), 74 deletions(-) diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 9e6848e39a38a..145f34941b908 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -499,7 +499,7 @@ impl SessionContext { let batches: Vec<_> = physical.collect_partitioned().await?; let table = Arc::new( - // pass constrains to the mem table. + // pass constraints to the mem table. MemTable::try_new(schema, batches)?.with_constraints(constraints), ); diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index ff5a300a6d18a..4d985456f9824 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -41,9 +41,9 @@ use crate::{ }; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::{ - aggregate_functional_dependencies, display::ToStringifiedPlan, Column, DFField, - DFSchema, DFSchemaRef, DataFusionError, FunctionalDependencies, OwnedTableReference, - Result, ScalarValue, TableReference, ToDFSchema, + display::ToStringifiedPlan, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, + FunctionalDependencies, OwnedTableReference, Result, ScalarValue, TableReference, + ToDFSchema, }; use std::any::Any; use std::cmp::Ordering; @@ -1232,71 +1232,6 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result { - let input_fields = input.schema().fields(); - // Calculate expression indices (if present) in the input schema. - let proj_indices = exprs - .iter() - .filter_map(|expr| { - let expr_name = match expr { - Expr::Alias(alias) => { - format!("{}", alias.expr) - } - _ => format!("{}", expr), - }; - input_fields - .iter() - .position(|item| item.qualified_name() == expr_name) - }) - .collect::>(); - Ok(input - .schema() - .functional_dependencies() - .project_functional_dependencies(&proj_indices, exprs.len())) -} - -/// Checks whether any expression in `group_expr` contains `Expr::GroupingSet`. -fn contains_grouping_set(group_expr: &[Expr]) -> bool { - group_expr - .iter() - .any(|expr| matches!(expr, Expr::GroupingSet(_))) -} - -/// Calculates functional dependencies for aggregate expressions. -pub(crate) fn calc_func_dependencies_for_aggregate( - // Expressions in the GROUP BY clause: - group_expr: &[Expr], - // Input plan of the aggregate: - input: &LogicalPlan, - // Aggregate schema - aggr_schema: &DFSchema, -) -> Result { - // We can do a case analysis on how to propagate functional dependencies based on - // whether the GROUP BY in question contains a grouping set expression: - // - If so, the functional dependencies will be empty because we cannot guarantee - // that GROUP BY expression results will be unique. - // - Otherwise, it may be possible to propagate functional dependencies. - if !contains_grouping_set(group_expr) { - let group_by_expr_names = group_expr - .iter() - .map(|item| item.display_name()) - .collect::>>()?; - let aggregate_func_dependencies = aggregate_functional_dependencies( - input.schema(), - &group_by_expr_names, - aggr_schema, - ); - Ok(aggregate_func_dependencies) - } else { - Ok(FunctionalDependencies::empty()) - } -} - /// Create Projection /// # Errors /// This function errors under any of the following conditions: diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index ff2fb963005b7..c0af08a366186 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -17,9 +17,6 @@ //! Logical plan types -use crate::builder::{ - calc_func_dependencies_for_aggregate, calc_func_dependencies_for_project, -}; use crate::expr::{Alias, Exists, InSubquery, Placeholder}; use crate::expr_rewriter::create_col_from_scalar_expr; use crate::expr_vec_fmt; @@ -41,8 +38,8 @@ use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeVisitor, VisitRecursion, }; use datafusion_common::{ - plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - OwnedTableReference, Result, ScalarValue, + aggregate_functional_dependencies, plan_err, Column, DFField, DFSchema, DFSchemaRef, + DataFusionError, FunctionalDependencies, OwnedTableReference, Result, ScalarValue, }; // backwards compatibility pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; @@ -1682,6 +1679,71 @@ impl Aggregate { } } +/// Checks whether any expression in `group_expr` contains `Expr::GroupingSet`. +fn contains_grouping_set(group_expr: &[Expr]) -> bool { + group_expr + .iter() + .any(|expr| matches!(expr, Expr::GroupingSet(_))) +} + +/// Calculates functional dependencies for aggregate expressions. +fn calc_func_dependencies_for_aggregate( + // Expressions in the GROUP BY clause: + group_expr: &[Expr], + // Input plan of the aggregate: + input: &LogicalPlan, + // Aggregate schema + aggr_schema: &DFSchema, +) -> Result { + // We can do a case analysis on how to propagate functional dependencies based on + // whether the GROUP BY in question contains a grouping set expression: + // - If so, the functional dependencies will be empty because we cannot guarantee + // that GROUP BY expression results will be unique. + // - Otherwise, it may be possible to propagate functional dependencies. + if !contains_grouping_set(group_expr) { + let group_by_expr_names = group_expr + .iter() + .map(|item| item.display_name()) + .collect::>>()?; + let aggregate_func_dependencies = aggregate_functional_dependencies( + input.schema(), + &group_by_expr_names, + aggr_schema, + ); + Ok(aggregate_func_dependencies) + } else { + Ok(FunctionalDependencies::empty()) + } +} + +/// This function projects functional dependencies of the `input` plan according +/// to projection expressions `exprs`. +fn calc_func_dependencies_for_project( + exprs: &[Expr], + input: &LogicalPlan, +) -> Result { + let input_fields = input.schema().fields(); + // Calculate expression indices (if present) in the input schema. + let proj_indices = exprs + .iter() + .filter_map(|expr| { + let expr_name = match expr { + Expr::Alias(alias) => { + format!("{}", alias.expr) + } + _ => format!("{}", expr), + }; + input_fields + .iter() + .position(|item| item.qualified_name() == expr_name) + }) + .collect::>(); + Ok(input + .schema() + .functional_dependencies() + .project_functional_dependencies(&proj_indices, exprs.len())) +} + /// Sorts its input according to a list of sort expressions. #[derive(Clone, PartialEq, Eq, Hash)] pub struct Sort { From d219bfe928954d08ac1f18f0e8ba729118eff7c6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 27 Jul 2023 13:08:09 +0300 Subject: [PATCH 36/37] Minor changes --- datafusion/sql/src/select.rs | 73 +++++++++++++++++++++--------------- 1 file changed, 42 insertions(+), 31 deletions(-) diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 3aed0b6e3020b..daf79e969e1f2 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -24,7 +24,9 @@ use crate::utils::{ resolve_columns, resolve_positions_to_exprs, }; -use datafusion_common::{get_target_functional_dependencies, DataFusionError, Result}; +use datafusion_common::{ + get_target_functional_dependencies, DFSchemaRef, DataFusionError, Result, +}; use datafusion_expr::expr::Alias; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, @@ -432,36 +434,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { group_by_exprs: Vec, aggr_exprs: Vec, ) -> Result<(LogicalPlan, Vec, Option)> { - let schema = input.schema(); - let mut new_group_by_exprs = group_by_exprs.clone(); - 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()); - } - } - } - - let group_by_exprs = new_group_by_exprs; + 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()) @@ -587,3 +561,40 @@ fn match_window_definitions( } Ok(()) } + +/// Update group by exprs, according to functioanl dependencies +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) +} From c989acd29864646195f124543beb378f183f3444 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 27 Jul 2023 13:40:23 +0300 Subject: [PATCH 37/37] Minor changes --- datafusion/expr/src/logical_plan/ddl.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index b0877a77a4bcd..dc247da3642c0 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -121,7 +121,7 @@ impl DdlStatement { constraints, .. }) => { - write!(f, "CreateMemoryTable: {name:?}{constraints:?}") + write!(f, "CreateMemoryTable: {name:?}{constraints}") } DdlStatement::CreateView(CreateView { name, .. }) => { write!(f, "CreateView: {name:?}")