From 5427f4b75b900529e96ca2009d3b065972bb5ebc Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Fri, 28 Jun 2024 11:54:33 +0200 Subject: [PATCH 01/10] draft: add logical types and support extension types --- .../examples/dataframe_subquery.rs | 6 +- datafusion-examples/examples/expr_api.rs | 21 +- .../examples/function_factory.rs | 6 +- datafusion-examples/examples/rewrite_expr.rs | 3 +- datafusion/common/src/column.rs | 15 +- datafusion/common/src/dfschema.rs | 238 +++++----- datafusion/common/src/lib.rs | 1 + .../common/src/logical_type/extension.rs | 289 ++++++++++++ datafusion/common/src/logical_type/field.rs | 163 +++++++ datafusion/common/src/logical_type/fields.rs | 94 ++++ datafusion/common/src/logical_type/mod.rs | 156 +++++++ .../common/src/logical_type/registry.rs | 51 +++ datafusion/common/src/logical_type/schema.rs | 159 +++++++ .../common/src/logical_type/type_signature.rs | 42 ++ datafusion/common/src/param_value.rs | 6 +- datafusion/common/src/scalar/mod.rs | 115 ++++- datafusion/core/src/dataframe/mod.rs | 15 +- .../core/src/datasource/listing/helpers.rs | 3 +- .../core/src/datasource/listing/table.rs | 5 +- datafusion/core/src/datasource/memory.rs | 2 +- .../physical_plan/file_scan_config.rs | 1 - .../physical_plan/parquet/row_filter.rs | 7 +- .../physical_plan/parquet/row_groups.rs | 9 +- .../core/src/execution/session_state.rs | 9 +- .../core/src/physical_optimizer/pruning.rs | 47 +- datafusion/core/src/physical_planner.rs | 11 +- datafusion/core/src/test/mod.rs | 2 +- datafusion/core/src/test/variable.rs | 12 +- datafusion/core/src/test_util/parquet.rs | 3 +- .../tests/dataframe/dataframe_functions.rs | 7 +- datafusion/core/tests/dataframe/mod.rs | 13 +- .../core/tests/expr_api/parse_sql_expr.rs | 12 +- .../core/tests/expr_api/simplification.rs | 41 +- .../core/tests/optimizer_integration.rs | 6 +- datafusion/core/tests/parquet/page_pruning.rs | 3 +- .../user_defined_scalar_functions.rs | 23 +- .../expr/src/conditional_expressions.rs | 10 +- datafusion/expr/src/expr.rs | 36 +- datafusion/expr/src/expr_fn.rs | 7 +- datafusion/expr/src/expr_rewriter/mod.rs | 12 +- datafusion/expr/src/expr_rewriter/order_by.rs | 10 +- datafusion/expr/src/expr_schema.rs | 138 +++--- datafusion/expr/src/logical_plan/builder.rs | 103 +++-- datafusion/expr/src/logical_plan/ddl.rs | 6 +- datafusion/expr/src/logical_plan/dml.rs | 9 +- datafusion/expr/src/logical_plan/extension.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 33 +- datafusion/expr/src/simplify.rs | 9 +- datafusion/expr/src/type_coercion/binary.rs | 2 + .../expr/src/type_coercion/functions.rs | 2 +- datafusion/expr/src/type_coercion/mod.rs | 47 +- datafusion/expr/src/utils.rs | 65 ++- datafusion/expr/src/var_provider.rs | 4 +- datafusion/functions/src/core/arrow_cast.rs | 4 +- datafusion/functions/src/core/getfield.rs | 17 +- datafusion/functions/src/core/named_struct.rs | 3 +- datafusion/functions/src/math/log.rs | 10 +- datafusion/functions/src/math/power.rs | 7 +- .../src/analyzer/count_wildcard_rule.rs | 5 +- .../src/analyzer/function_rewrite.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 236 +++++----- .../optimizer/src/common_subexpr_eliminate.rs | 18 +- .../src/decorrelate_predicate_subquery.rs | 78 ++-- .../optimizer/src/eliminate_one_union.rs | 5 +- .../optimizer/src/eliminate_outer_join.rs | 6 +- .../src/extract_equijoin_predicate.rs | 6 +- .../optimizer/src/optimize_projections/mod.rs | 24 +- datafusion/optimizer/src/optimizer.rs | 10 +- .../optimizer/src/propagate_empty_relation.rs | 5 +- datafusion/optimizer/src/push_down_filter.rs | 8 +- .../optimizer/src/scalar_subquery_to_join.rs | 34 +- .../simplify_expressions/expr_simplifier.rs | 51 ++- .../simplify_expressions/simplify_exprs.rs | 10 +- datafusion/optimizer/src/test/mod.rs | 2 +- .../src/unwrap_cast_in_comparison.rs | 426 +++++++----------- .../optimizer/tests/optimizer_integration.rs | 3 +- datafusion/physical-expr-common/src/utils.rs | 3 +- .../physical-expr/src/expressions/in_list.rs | 2 +- .../physical-expr/src/expressions/negative.rs | 2 +- datafusion/physical-expr/src/planner.rs | 14 +- .../physical-expr/src/utils/guarantee.rs | 5 +- datafusion/proto-common/src/from_proto/mod.rs | 24 +- datafusion/proto-common/src/to_proto/mod.rs | 21 +- datafusion/proto/src/logical_plan/mod.rs | 12 +- .../tests/cases/roundtrip_logical_plan.rs | 16 +- datafusion/sql/examples/sql.rs | 3 +- datafusion/sql/src/expr/function.rs | 12 +- datafusion/sql/src/expr/identifier.rs | 4 +- datafusion/sql/src/expr/mod.rs | 22 +- datafusion/sql/src/expr/value.rs | 6 +- datafusion/sql/src/planner.rs | 66 +-- datafusion/sql/src/relation/join.rs | 2 +- datafusion/sql/src/statement.rs | 14 +- datafusion/sql/src/unparser/expr.rs | 107 ++--- datafusion/sql/src/utils.rs | 11 +- datafusion/sql/tests/cases/plan_to_sql.rs | 1 - datafusion/sql/tests/common/mod.rs | 3 +- datafusion/sql/tests/sql_integration.rs | 27 +- .../engines/datafusion_engine/normalize.rs | 45 +- .../substrait/src/logical_plan/consumer.rs | 119 ++--- .../substrait/src/logical_plan/producer.rs | 107 ++--- 101 files changed, 2382 insertions(+), 1327 deletions(-) create mode 100644 datafusion/common/src/logical_type/extension.rs create mode 100644 datafusion/common/src/logical_type/field.rs create mode 100644 datafusion/common/src/logical_type/fields.rs create mode 100644 datafusion/common/src/logical_type/mod.rs create mode 100644 datafusion/common/src/logical_type/registry.rs create mode 100644 datafusion/common/src/logical_type/schema.rs create mode 100644 datafusion/common/src/logical_type/type_signature.rs diff --git a/datafusion-examples/examples/dataframe_subquery.rs b/datafusion-examples/examples/dataframe_subquery.rs index e798751b3353..f3cdd048968b 100644 --- a/datafusion-examples/examples/dataframe_subquery.rs +++ b/datafusion-examples/examples/dataframe_subquery.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::DataType; +use datafusion_common::logical_type::LogicalType; use std::sync::Arc; use datafusion::error::Result; @@ -48,7 +48,7 @@ async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { scalar_subquery(Arc::new( ctx.table("t2") .await? - .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(LogicalType::Utf8, "t1.c1").eq(col("t2.c1")))? .aggregate(vec![], vec![avg(col("t2.c2"))])? .select(vec![avg(col("t2.c2"))])? .into_unoptimized_plan(), @@ -91,7 +91,7 @@ async fn where_exist_subquery(ctx: &SessionContext) -> Result<()> { .filter(exists(Arc::new( ctx.table("t2") .await? - .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(LogicalType::Utf8, "t1.c1").eq(col("t2.c1")))? .select(vec![col("t2.c2")])? .into_unoptimized_plan(), )))? diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 36ce3badcb5e..d53a1feff085 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -28,6 +28,9 @@ use datafusion::functions_aggregate::first_last::first_value_udaf; use datafusion::optimizer::simplify_expressions::ExprSimplifier; use datafusion::physical_expr::{analyze, AnalysisContext, ExprBoundaries}; use datafusion::prelude::*; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::LogicalType; use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr::BinaryExpr; @@ -156,7 +159,7 @@ fn simplify_demo() -> Result<()> { // However, DataFusion's simplification logic can do this for you // you need to tell DataFusion the type of column "ts": - let schema = Schema::new(vec![make_ts_field("ts")]).to_dfschema_ref()?; + let schema = LogicalSchema::from(Schema::new(vec![make_ts_field("ts")])).to_dfschema_ref()?; // And then build a simplifier // the ExecutionProps carries information needed to simplify @@ -177,10 +180,10 @@ fn simplify_demo() -> Result<()> { ); // here are some other examples of what DataFusion is capable of - let schema = Schema::new(vec![ + let schema = LogicalSchema::from(Schema::new(vec![ make_field("i", DataType::Int64), make_field("b", DataType::Boolean), - ]) + ])) .to_dfschema_ref()?; let context = SimplifyContext::new(&props).with_schema(schema.clone()); let simplifier = ExprSimplifier::new(context); @@ -211,7 +214,7 @@ fn simplify_demo() -> Result<()> { // String --> Date simplification // `cast('2020-09-01' as date)` --> 18500 assert_eq!( - simplifier.simplify(lit("2020-09-01").cast_to(&DataType::Date32, &schema)?)?, + simplifier.simplify(lit("2020-09-01").cast_to(&LogicalType::Date32, &schema)?)?, lit(ScalarValue::Date32(Some(18506))) ); @@ -258,7 +261,7 @@ fn range_analysis_demo() -> Result<()> { let analysis_result = analyze( &physical_expr, AnalysisContext::new(boundaries), - df_schema.as_ref(), + &df_schema.into(), )?; // The results of the analysis is an range, encoded as an `Interval`, for @@ -293,14 +296,14 @@ fn expression_type_demo() -> Result<()> { // a schema. In this case we create a schema where the column `c` is of // type Utf8 (a String / VARCHAR) let schema = DFSchema::from_unqualified_fields( - vec![Field::new("c", DataType::Utf8, true)].into(), + vec![LogicalField::new("c", LogicalType::Utf8, true)].into(), HashMap::new(), )?; assert_eq!("Utf8", format!("{}", expr.get_type(&schema).unwrap())); // Using a schema where the column `foo` is of type Int32 let schema = DFSchema::from_unqualified_fields( - vec![Field::new("c", DataType::Int32, true)].into(), + vec![LogicalField::new("c", LogicalType::Int32, true)].into(), HashMap::new(), )?; assert_eq!("Int32", format!("{}", expr.get_type(&schema).unwrap())); @@ -310,8 +313,8 @@ fn expression_type_demo() -> Result<()> { let expr = col("c1") + col("c2"); let schema = DFSchema::from_unqualified_fields( vec![ - Field::new("c1", DataType::Int32, true), - Field::new("c2", DataType::Float32, true), + LogicalField::new("c1", LogicalType::Int32, true), + LogicalField::new("c2", LogicalType::Float32, true), ] .into(), HashMap::new(), diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index f57b3bf60404..bf2a44e7063e 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -22,6 +22,7 @@ use datafusion::error::Result; use datafusion::execution::context::{ FunctionFactory, RegisterFunction, SessionContext, SessionState, }; +use datafusion_common::logical_type::extension::ExtensionType; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{exec_err, internal_err, DataFusionError}; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; @@ -216,13 +217,14 @@ impl TryFrom for ScalarFunctionWrapper { .expect("Expression has to be defined!"), return_type: definition .return_type - .expect("Return type has to be defined!"), + .expect("Return type has to be defined!") + .physical_type(), signature: Signature::exact( definition .args .unwrap_or_default() .into_iter() - .map(|a| a.data_type) + .map(|a| a.data_type.physical_type()) .collect(), definition .params diff --git a/datafusion-examples/examples/rewrite_expr.rs b/datafusion-examples/examples/rewrite_expr.rs index 06286d5d66ed..ed748288ff3d 100644 --- a/datafusion-examples/examples/rewrite_expr.rs +++ b/datafusion-examples/examples/rewrite_expr.rs @@ -17,6 +17,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; +use datafusion_common::logical_type::LogicalType; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ @@ -211,7 +212,7 @@ impl ContextProvider for MyContextProvider { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index e36a4f890644..ac54e87fd617 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -17,8 +17,6 @@ //! Column -use arrow_schema::{Field, FieldRef}; - use crate::error::_schema_err; use crate::utils::{parse_identifiers_normalized, quote_identifier}; use crate::{DFSchema, DataFusionError, Result, SchemaError, TableReference}; @@ -27,6 +25,7 @@ use std::convert::Infallible; use std::fmt; use std::str::FromStr; use std::sync::Arc; +use crate::logical_type::field::{LogicalField, LogicalFieldRef}; /// A named reference to a qualified field in a schema. #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] @@ -349,15 +348,15 @@ impl From for Column { } /// Create a column, use qualifier and field name -impl From<(Option<&TableReference>, &Field)> for Column { - fn from((relation, field): (Option<&TableReference>, &Field)) -> Self { +impl From<(Option<&TableReference>, &LogicalField)> for Column { + fn from((relation, field): (Option<&TableReference>, &LogicalField)) -> Self { Self::new(relation.cloned(), field.name()) } } /// Create a column, use qualifier and field name -impl From<(Option<&TableReference>, &FieldRef)> for Column { - fn from((relation, field): (Option<&TableReference>, &FieldRef)) -> Self { +impl From<(Option<&TableReference>, &LogicalFieldRef)> for Column { + fn from((relation, field): (Option<&TableReference>, &LogicalFieldRef)) -> Self { Self::new(relation.cloned(), field.name()) } } @@ -380,7 +379,7 @@ impl fmt::Display for Column { mod tests { use super::*; use arrow::datatypes::DataType; - use arrow_schema::SchemaBuilder; + use arrow_schema::{Field, SchemaBuilder}; fn create_qualified_schema(qualifier: &str, names: Vec<&str>) -> Result { let mut schema_builder = SchemaBuilder::new(); @@ -389,7 +388,7 @@ mod tests { .iter() .map(|f| Field::new(*f, DataType::Boolean, true)), ); - let schema = Arc::new(schema_builder.finish()); + let schema = Arc::new(schema_builder.finish().into()); DFSchema::try_from_qualified_schema(qualifier, &schema) } diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 3c2cc89fc014..58c43dab5675 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -30,8 +30,12 @@ use crate::{ }; use arrow::compute::can_cast_types; -use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; -use arrow_schema::SchemaBuilder; +use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use crate::logical_type::extension::ExtensionType; +use crate::logical_type::field::{LogicalField, LogicalFieldRef}; +use crate::logical_type::fields::LogicalFields; +use crate::logical_type::LogicalType; +use crate::logical_type::schema::{LogicalSchema, LogicalSchemaBuilder, LogicalSchemaRef}; /// A reference-counted reference to a [DFSchema]. pub type DFSchemaRef = Arc; @@ -62,7 +66,7 @@ pub type DFSchemaRef = Arc; /// Field::new("c1", DataType::Int32, false), /// ]); /// -/// let df_schema = DFSchema::try_from_qualified_schema("t1", &arrow_schema).unwrap(); +/// let df_schema = DFSchema::try_from_qualified_schema("t1", &arrow_schema.into()).unwrap(); /// let column = Column::from_qualified_name("t1.c1"); /// assert!(df_schema.has_column(&column)); /// @@ -107,9 +111,9 @@ pub type DFSchemaRef = Arc; #[derive(Debug, Clone, PartialEq, Eq)] pub struct DFSchema { /// Inner Arrow schema reference. - inner: SchemaRef, + inner: LogicalSchemaRef, /// Optional qualifiers for each column in this schema. In the same order as - /// the `self.inner.fields()` + /// the `self.inner.fields` field_qualifiers: Vec>, /// Stores functional dependencies in the schema. functional_dependencies: FunctionalDependencies, @@ -119,7 +123,7 @@ impl DFSchema { /// Creates an empty `DFSchema` pub fn empty() -> Self { Self { - inner: Arc::new(Schema::new([])), + inner: Arc::new(LogicalSchema::new([])), field_qualifiers: vec![], functional_dependencies: FunctionalDependencies::empty(), } @@ -128,26 +132,26 @@ impl DFSchema { /// Return a reference to the inner Arrow [`Schema`] /// /// Note this does not have the qualifier information - pub fn as_arrow(&self) -> &Schema { + pub fn as_arrow(&self) -> &LogicalSchema { self.inner.as_ref() } /// Return a reference to the inner Arrow [`SchemaRef`] /// /// Note this does not have the qualifier information - pub fn inner(&self) -> &SchemaRef { + pub fn inner(&self) -> &LogicalSchemaRef { &self.inner } /// Create a `DFSchema` from an Arrow schema where all the fields have a given qualifier pub fn new_with_metadata( - qualified_fields: Vec<(Option, Arc)>, + qualified_fields: Vec<(Option, Arc)>, metadata: HashMap, ) -> Result { - let (qualifiers, fields): (Vec>, Vec>) = + let (qualifiers, fields): (Vec>, Vec>) = qualified_fields.into_iter().unzip(); - let schema = Arc::new(Schema::new_with_metadata(fields, metadata)); + let schema = Arc::new(LogicalSchema::new_with_metadata(fields, metadata)); let dfschema = Self { inner: schema, @@ -173,11 +177,11 @@ impl DFSchema { note = "Please use `from_unqualified_fields` instead (this one's name is a typo). This method is subject to be removed soon" )] pub fn from_unqualifed_fields( - fields: Fields, + fields: LogicalFields, metadata: HashMap, ) -> Result { let field_count = fields.len(); - let schema = Arc::new(Schema::new_with_metadata(fields, metadata)); + let schema = Arc::new(LogicalSchema::new_with_metadata(fields, metadata)); let dfschema = Self { inner: schema, field_qualifiers: vec![None; field_count], @@ -193,7 +197,7 @@ impl DFSchema { /// `DFSchema::try_from`. pub fn try_from_qualified_schema( qualifier: impl Into, - schema: &Schema, + schema: &LogicalSchema, ) -> Result { let qualifier = qualifier.into(); let schema = DFSchema { @@ -208,7 +212,7 @@ impl DFSchema { /// Create a `DFSchema` from an Arrow schema where all the fields have a given qualifier pub fn from_field_specific_qualified_schema( qualifiers: Vec>, - schema: &SchemaRef, + schema: &LogicalSchemaRef, ) -> Result { let dfschema = Self { inner: schema.clone(), @@ -224,7 +228,7 @@ impl DFSchema { let mut qualified_names = BTreeSet::new(); let mut unqualified_names = BTreeSet::new(); - for (field, qualifier) in self.inner.fields().iter().zip(&self.field_qualifiers) { + for (field, qualifier) in self.inner.fields.iter().zip(&self.field_qualifiers) { if let Some(qualifier) = qualifier { qualified_names.insert((qualifier, field.name())); } else if !unqualified_names.insert(field.name()) { @@ -263,8 +267,8 @@ impl DFSchema { /// 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 { - let mut schema_builder = SchemaBuilder::new(); - schema_builder.extend(self.inner.fields().iter().cloned()); + let mut schema_builder = LogicalSchemaBuilder::new(); + schema_builder.extend(self.inner.fields.iter().cloned()); schema_builder.extend(schema.fields().iter().cloned()); let new_schema = schema_builder.finish(); @@ -291,23 +295,23 @@ impl DFSchema { return; } - let self_fields: HashSet<(Option<&TableReference>, &FieldRef)> = + let self_fields: HashSet<(Option<&TableReference>, &LogicalFieldRef)> = self.iter().collect(); let self_unqualified_names: HashSet<&str> = self .inner .fields .iter() - .map(|field| field.name().as_str()) + .map(|field| field.name()) .collect(); - let mut schema_builder = SchemaBuilder::from(self.inner.fields.clone()); + let mut schema_builder = LogicalSchemaBuilder::from(self.inner.fields.clone()); let mut qualifiers = Vec::new(); for (qualifier, field) in other_schema.iter() { // skip duplicate columns let duplicated_field = match qualifier { Some(q) => self_fields.contains(&(Some(q), field)), // for unqualified columns, check as unqualified name - None => self_unqualified_names.contains(field.name().as_str()), + None => self_unqualified_names.contains(field.name()), }; if !duplicated_field { // self.inner.fields.push(field.clone()); @@ -325,19 +329,19 @@ impl DFSchema { } /// Get a list of fields - pub fn fields(&self) -> &Fields { + pub fn fields(&self) -> &LogicalFields { &self.inner.fields } /// Returns an immutable reference of a specific `Field` instance selected using an /// offset within the internal `fields` vector - pub fn field(&self, i: usize) -> &Field { + pub fn field(&self, i: usize) -> &LogicalField { &self.inner.fields[i] } /// Returns an immutable reference of a specific `Field` instance selected using an /// offset within the internal `fields` vector and its qualifier - pub fn qualified_field(&self, i: usize) -> (Option<&TableReference>, &Field) { + pub fn qualified_field(&self, i: usize) -> (Option<&TableReference>, &LogicalField) { (self.field_qualifiers[i].as_ref(), self.field(i)) } @@ -404,7 +408,7 @@ impl DFSchema { &self, qualifier: Option<&TableReference>, name: &str, - ) -> Result<&Field> { + ) -> Result<&LogicalField> { if let Some(qualifier) = qualifier { self.field_with_qualified_name(qualifier, name) } else { @@ -417,7 +421,7 @@ impl DFSchema { &self, qualifier: Option<&TableReference>, name: &str, - ) -> Result<(Option<&TableReference>, &Field)> { + ) -> Result<(Option<&TableReference>, &LogicalField)> { if let Some(qualifier) = qualifier { let idx = self .index_of_column_by_name(Some(qualifier), name) @@ -429,7 +433,7 @@ impl DFSchema { } /// Find all fields having the given qualifier - pub fn fields_with_qualified(&self, qualifier: &TableReference) -> Vec<&Field> { + pub fn fields_with_qualified(&self, qualifier: &TableReference) -> Vec<&LogicalField> { self.iter() .filter(|(q, _)| q.map(|q| q.eq(qualifier)).unwrap_or(false)) .map(|(_, f)| f.as_ref()) @@ -448,7 +452,7 @@ impl DFSchema { } /// Find all fields that match the given name - pub fn fields_with_unqualified_name(&self, name: &str) -> Vec<&Field> { + pub fn fields_with_unqualified_name(&self, name: &str) -> Vec<&LogicalField> { self.fields() .iter() .filter(|field| field.name() == name) @@ -460,7 +464,7 @@ impl DFSchema { pub fn qualified_fields_with_unqualified_name( &self, name: &str, - ) -> Vec<(Option<&TableReference>, &Field)> { + ) -> Vec<(Option<&TableReference>, &LogicalField)> { self.iter() .filter(|(_, field)| field.name() == name) .map(|(qualifier, field)| (qualifier, field.as_ref())) @@ -479,7 +483,7 @@ impl DFSchema { pub fn columns(&self) -> Vec { self.iter() .map(|(qualifier, field)| { - Column::new(qualifier.cloned(), field.name().clone()) + Column::new(qualifier.cloned(), field.name()) }) .collect() } @@ -488,7 +492,7 @@ impl DFSchema { pub fn qualified_field_with_unqualified_name( &self, name: &str, - ) -> Result<(Option<&TableReference>, &Field)> { + ) -> Result<(Option<&TableReference>, &LogicalField)> { let matches = self.qualified_fields_with_unqualified_name(name); match matches.len() { 0 => Err(unqualified_field_not_found(name, self)), @@ -520,7 +524,7 @@ impl DFSchema { } /// Find the field with the given name - pub fn field_with_unqualified_name(&self, name: &str) -> Result<&Field> { + pub fn field_with_unqualified_name(&self, name: &str) -> Result<&LogicalField> { let matches = self.qualified_fields_with_unqualified_name(name); match matches.len() { 0 => Err(unqualified_field_not_found(name, self)), @@ -556,7 +560,7 @@ impl DFSchema { &self, qualifier: &TableReference, name: &str, - ) -> Result<&Field> { + ) -> Result<&LogicalField> { let idx = self .index_of_column_by_name(Some(qualifier), name) .ok_or_else(|| field_not_found(Some(qualifier.clone()), name, self))?; @@ -565,7 +569,7 @@ impl DFSchema { } /// Find the field with the given qualified column - pub fn field_from_column(&self, column: &Column) -> Result<&Field> { + pub fn field_from_column(&self, column: &Column) -> Result<&LogicalField> { match &column.relation { Some(r) => self.field_with_qualified_name(r, &column.name), None => self.field_with_unqualified_name(&column.name), @@ -576,7 +580,7 @@ impl DFSchema { pub fn qualified_field_from_column( &self, column: &Column, - ) -> Result<(Option<&TableReference>, &Field)> { + ) -> Result<(Option<&TableReference>, &LogicalField)> { self.qualified_field_with_name(column.relation.as_ref(), &column.name) } @@ -671,7 +675,7 @@ impl DFSchema { self_fields.zip(other_fields).all(|((q1, f1), (q2, f2))| { q1 == q2 && f1.name() == f2.name() - && Self::datatype_is_semantically_equal(f1.data_type(), f2.data_type()) + && Self::datatype_is_semantically_equal(&f1.data_type().physical_type(), &f2.data_type().physical_type()) }) } @@ -679,40 +683,8 @@ impl DFSchema { /// than datatype_is_semantically_equal in that a Dictionary type is logically /// equal to a plain V type, but not semantically equal. Dictionary is also /// logically equal to Dictionary. - pub fn datatype_is_logically_equal(dt1: &DataType, dt2: &DataType) -> bool { - // check nested fields - match (dt1, dt2) { - (DataType::Dictionary(_, v1), DataType::Dictionary(_, v2)) => { - v1.as_ref() == v2.as_ref() - } - (DataType::Dictionary(_, v1), othertype) => v1.as_ref() == othertype, - (othertype, DataType::Dictionary(_, v1)) => v1.as_ref() == othertype, - (DataType::List(f1), DataType::List(f2)) - | (DataType::LargeList(f1), DataType::LargeList(f2)) - | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) - | (DataType::Map(f1, _), DataType::Map(f2, _)) => { - Self::field_is_logically_equal(f1, f2) - } - (DataType::Struct(fields1), DataType::Struct(fields2)) => { - let iter1 = fields1.iter(); - let iter2 = fields2.iter(); - fields1.len() == fields2.len() && - // all fields have to be the same - iter1 - .zip(iter2) - .all(|(f1, f2)| Self::field_is_logically_equal(f1, f2)) - } - (DataType::Union(fields1, _), DataType::Union(fields2, _)) => { - let iter1 = fields1.iter(); - let iter2 = fields2.iter(); - fields1.len() == fields2.len() && - // all fields have to be the same - iter1 - .zip(iter2) - .all(|((t1, f1), (t2, f2))| t1 == t2 && Self::field_is_logically_equal(f1, f2)) - } - _ => dt1 == dt2, - } + pub fn datatype_is_logically_equal(dt1: &LogicalType, dt2: &LogicalType) -> bool { + dt1 == dt2 } /// Returns true of two [`DataType`]s are semantically equal (same @@ -762,11 +734,6 @@ impl DFSchema { } } - fn field_is_logically_equal(f1: &Field, f2: &Field) -> bool { - f1.name() == f2.name() - && Self::datatype_is_logically_equal(f1.data_type(), f2.data_type()) - } - fn field_is_semantically_equal(f1: &Field, f2: &Field) -> bool { f1.name() == f2.name() && Self::datatype_is_semantically_equal(f1.data_type(), f2.data_type()) @@ -809,10 +776,10 @@ impl DFSchema { } /// Iterate over the qualifiers and fields in the DFSchema - pub fn iter(&self) -> impl Iterator, &FieldRef)> { + pub fn iter(&self) -> impl Iterator, &LogicalFieldRef)> { self.field_qualifiers .iter() - .zip(self.inner.fields().iter()) + .zip(self.inner.fields.iter()) .map(|(qualifier, field)| (qualifier.as_ref(), field)) } } @@ -820,7 +787,7 @@ impl DFSchema { impl From for Schema { /// Convert DFSchema into a Schema fn from(df_schema: DFSchema) -> Self { - let fields: Fields = df_schema.inner.fields.clone(); + let fields: Fields = df_schema.inner.fields.clone().into(); Schema::new_with_metadata(fields, df_schema.inner.metadata.clone()) } } @@ -828,23 +795,29 @@ impl From for Schema { impl From<&DFSchema> for Schema { /// Convert DFSchema reference into a Schema fn from(df_schema: &DFSchema) -> Self { - let fields: Fields = df_schema.inner.fields.clone(); + let fields: Fields = df_schema.inner.fields.clone().into(); Schema::new_with_metadata(fields, df_schema.inner.metadata.clone()) } } -/// Allow DFSchema to be converted into an Arrow `&Schema` -impl AsRef for DFSchema { - fn as_ref(&self) -> &Schema { - self.as_arrow() +/// Create a `DFSchema` from an Arrow schema +impl TryFrom for DFSchema { + type Error = DataFusionError; + fn try_from(schema: LogicalSchema) -> Result { + Self::try_from(Arc::new(schema)) } } -/// Allow DFSchema to be converted into an Arrow `&SchemaRef` (to clone, for -/// example) -impl AsRef for DFSchema { - fn as_ref(&self) -> &SchemaRef { - self.inner() +impl TryFrom for DFSchema { + type Error = DataFusionError; + fn try_from(schema: LogicalSchemaRef) -> Result { + let field_count = schema.fields.len(); + let dfschema = Self { + inner: schema, + field_qualifiers: vec![None; field_count], + functional_dependencies: FunctionalDependencies::empty(), + }; + Ok(dfschema) } } @@ -852,20 +825,14 @@ impl AsRef for DFSchema { impl TryFrom for DFSchema { type Error = DataFusionError; fn try_from(schema: Schema) -> Result { - Self::try_from(Arc::new(schema)) + Self::try_from(LogicalSchema::from(schema)) } } impl TryFrom for DFSchema { type Error = DataFusionError; fn try_from(schema: SchemaRef) -> Result { - let field_count = schema.fields.len(); - let dfschema = Self { - inner: schema, - field_qualifiers: vec![None; field_count], - functional_dependencies: FunctionalDependencies::empty(), - }; - Ok(dfschema) + Self::try_from(schema.as_ref().clone()) } } @@ -897,22 +864,22 @@ where } } -impl ToDFSchema for Schema { +impl ToDFSchema for LogicalSchema { fn to_dfschema(self) -> Result { DFSchema::try_from(self) } } -impl ToDFSchema for SchemaRef { +impl ToDFSchema for LogicalSchemaRef { fn to_dfschema(self) -> Result { DFSchema::try_from(self) } } -impl ToDFSchema for Vec { +impl ToDFSchema for Vec { fn to_dfschema(self) -> Result { let field_count = self.len(); - let schema = Schema { + let schema = LogicalSchema { fields: self.into(), metadata: HashMap::new(), }; @@ -949,13 +916,13 @@ pub trait ExprSchema: std::fmt::Debug { fn nullable(&self, col: &Column) -> Result; /// What is the datatype of this column? - fn data_type(&self, col: &Column) -> Result<&DataType>; + fn data_type(&self, col: &Column) -> Result<&LogicalType>; /// Returns the column's optional metadata. fn metadata(&self, col: &Column) -> Result<&HashMap>; /// Return the coulmn's datatype and nullability - fn data_type_and_nullable(&self, col: &Column) -> Result<(&DataType, bool)>; + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)>; } // Implement `ExprSchema` for `Arc` @@ -964,7 +931,7 @@ impl + std::fmt::Debug> ExprSchema for P { self.as_ref().nullable(col) } - fn data_type(&self, col: &Column) -> Result<&DataType> { + fn data_type(&self, col: &Column) -> Result<&LogicalType> { self.as_ref().data_type(col) } @@ -972,7 +939,7 @@ impl + std::fmt::Debug> ExprSchema for P { ExprSchema::metadata(self.as_ref(), col) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&DataType, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)> { self.as_ref().data_type_and_nullable(col) } } @@ -982,7 +949,7 @@ impl ExprSchema for DFSchema { Ok(self.field_from_column(col)?.is_nullable()) } - fn data_type(&self, col: &Column) -> Result<&DataType> { + fn data_type(&self, col: &Column) -> Result<&LogicalType> { Ok(self.field_from_column(col)?.data_type()) } @@ -990,7 +957,7 @@ impl ExprSchema for DFSchema { Ok(self.field_from_column(col)?.metadata()) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&DataType, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)> { let field = self.field_from_column(col)?; Ok((field.data_type(), field.is_nullable())) } @@ -1041,8 +1008,8 @@ impl SchemaExt for Schema { .all(|(f1, f2)| { f1.name() == f2.name() && DFSchema::datatype_is_logically_equal( - f1.data_type(), - f2.data_type(), + &f1.data_type().into(), + &f2.data_type().into(), ) }) } @@ -1082,7 +1049,7 @@ mod tests { &Schema::new(vec![ Field::new("CapitalColumn", DataType::Boolean, true), Field::new("field.with.period", DataType::Boolean, true), - ]), + ]).into(), )?; // lookup with unqualified name "t1.c0" @@ -1112,9 +1079,9 @@ mod tests { fn test_from_field_specific_qualified_schema() -> Result<()> { let schema = DFSchema::from_field_specific_qualified_schema( vec![Some("t1".into()), None], - &Arc::new(Schema::new(vec![ - Field::new("c0", DataType::Boolean, true), - Field::new("c1", DataType::Boolean, true), + &Arc::new(LogicalSchema::new(vec![ + LogicalField::new("c0", LogicalType::Boolean, true), + LogicalField::new("c1", LogicalType::Boolean, true), ])), )?; assert_eq!("fields:[t1.c0, c1], metadata:{}", schema.to_string()); @@ -1127,9 +1094,9 @@ mod tests { vec![ ( Some("t0".into()), - Arc::new(Field::new("c0", DataType::Boolean, true)), + Arc::new(Field::new("c0", DataType::Boolean, true).into()), ), - (None, Arc::new(Field::new("c1", DataType::Boolean, true))), + (None, Arc::new(Field::new("c1", DataType::Boolean, true).into())), ], HashMap::new(), )?; @@ -1273,41 +1240,42 @@ mod tests { vec![Field::new("c0", DataType::Int64, true)], metadata.clone(), ); - let arrow_schema_ref = Arc::new(arrow_schema.clone()); + let logical_schema = LogicalSchema::from(arrow_schema); + let logical_schema_ref = Arc::new(logical_schema.clone()); let df_schema = DFSchema { - inner: arrow_schema_ref.clone(), - field_qualifiers: vec![None; arrow_schema_ref.fields.len()], + inner: logical_schema_ref.clone(), + field_qualifiers: vec![None; logical_schema_ref.fields.len()], functional_dependencies: FunctionalDependencies::empty(), }; let df_schema_ref = Arc::new(df_schema.clone()); { - let arrow_schema = arrow_schema.clone(); - let arrow_schema_ref = arrow_schema_ref.clone(); + let logical_schema = logical_schema.clone(); + let logical_schema_ref = logical_schema_ref.clone(); - assert_eq!(df_schema, arrow_schema.to_dfschema().unwrap()); - assert_eq!(df_schema, arrow_schema_ref.to_dfschema().unwrap()); + assert_eq!(df_schema, logical_schema.to_dfschema().unwrap()); + assert_eq!(df_schema, logical_schema_ref.to_dfschema().unwrap()); } { - let arrow_schema = arrow_schema.clone(); - let arrow_schema_ref = arrow_schema_ref.clone(); + let logical_schema = logical_schema.clone(); + let logical_schema_ref = logical_schema_ref.clone(); - assert_eq!(df_schema_ref, arrow_schema.to_dfschema_ref().unwrap()); - assert_eq!(df_schema_ref, arrow_schema_ref.to_dfschema_ref().unwrap()); + assert_eq!(df_schema_ref, logical_schema.to_dfschema_ref().unwrap()); + assert_eq!(df_schema_ref, logical_schema_ref.to_dfschema_ref().unwrap()); } // Now, consume the refs - assert_eq!(df_schema_ref, arrow_schema.to_dfschema_ref().unwrap()); - assert_eq!(df_schema_ref, arrow_schema_ref.to_dfschema_ref().unwrap()); + assert_eq!(df_schema_ref, logical_schema.to_dfschema_ref().unwrap()); + assert_eq!(df_schema_ref, logical_schema_ref.to_dfschema_ref().unwrap()); } - fn test_schema_1() -> Schema { + fn test_schema_1() -> LogicalSchema { Schema::new(vec![ Field::new("c0", DataType::Boolean, true), Field::new("c1", DataType::Boolean, true), - ]) + ]).into() } #[test] fn test_dfschema_to_schema_convertion() { @@ -1319,7 +1287,7 @@ mod tests { b_metadata.insert("key".to_string(), "value".to_string()); let b_field = Field::new("b", DataType::Int64, false).with_metadata(b_metadata); - let schema = Arc::new(Schema::new(vec![a_field, b_field])); + let schema = LogicalSchemaRef::new(Schema::new(vec![a_field, b_field]).into()); let df_schema = DFSchema { inner: schema.clone(), @@ -1363,10 +1331,10 @@ mod tests { Ok(()) } - fn test_schema_2() -> Schema { - Schema::new(vec![ - Field::new("c100", DataType::Boolean, true), - Field::new("c101", DataType::Boolean, true), + fn test_schema_2() -> LogicalSchema { + LogicalSchema::new(vec![ + LogicalField::new("c100", LogicalType::Boolean, true), + LogicalField::new("c101", LogicalType::Boolean, true), ]) } diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index c275152642f0..7585f1a2c12a 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -42,6 +42,7 @@ pub mod stats; pub mod test_util; pub mod tree_node; pub mod utils; +pub mod logical_type; /// Reexport arrow crate pub use arrow; diff --git a/datafusion/common/src/logical_type/extension.rs b/datafusion/common/src/logical_type/extension.rs new file mode 100644 index 000000000000..0332de49f19e --- /dev/null +++ b/datafusion/common/src/logical_type/extension.rs @@ -0,0 +1,289 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow_schema::{DataType, FieldRef, IntervalUnit, TimeUnit}; + +use crate::logical_type::type_signature::TypeSignature; +use crate::logical_type::LogicalType; + +pub type ExtensionTypeRef = Arc; + +pub trait ExtensionType: std::fmt::Debug { + fn display_name(&self) -> &str; + fn type_signature(&self) -> TypeSignature; + fn physical_type(&self) -> DataType; + + fn is_comparable(&self) -> bool; + fn is_orderable(&self) -> bool; + fn is_numeric(&self) -> bool; + fn is_floating(&self) -> bool; +} + +impl ExtensionType for LogicalType { + fn display_name(&self) -> &str { + use crate::logical_type::LogicalType::*; + match self { + Null => "Null", + Boolean => "Boolean", + Int8 => "Int8", + Int16 => "Int16", + Int32 => "Int32", + Int64 => "Int64", + UInt8 => "Uint8", + UInt16 => "Uint16", + UInt32 => "Uint32", + UInt64 => "Uint64", + Float16 => "Float16", + Float32 => "Float16", + Float64 => "Float64", + Date32 => "Date32", + Date64 => "Date64", + Time32(_) => "Time32", + Time64(_) => "Time64", + Timestamp(_, _) => "Timestamp", + Duration(_) => "Duration", + Interval(_) => "Interval", + Binary => "Binary", + FixedSizeBinary(_) => "FixedSizeBinary", + LargeBinary => "LargeBinary", + Utf8 => "Utf8", + LargeUtf8 => "LargeUtf8", + List(_) => "List", + FixedSizeList(_, _) => "FixedSizeList", + LargeList(_) => "LargeList", + Struct(_) => "Struct", + Map(_, _) => "Map", + Decimal128(_, _) => "Decimal128", + Decimal256(_, _) => "Decimal256", + Extension(ext) => ext.display_name(), + } + } + + fn type_signature(&self) -> TypeSignature { + use crate::logical_type::LogicalType::*; + fn time_unit_to_param(tu: &TimeUnit) -> &'static str { + match tu { + TimeUnit::Second => "second", + TimeUnit::Millisecond => "millisecond", + TimeUnit::Microsecond => "microsecond", + TimeUnit::Nanosecond => "nanosecond", + } + } + + match self { + Boolean => TypeSignature::new("boolean"), + Int32 => TypeSignature::new("int32"), + Int64 => TypeSignature::new("int64"), + UInt64 => TypeSignature::new("uint64"), + Float32 => TypeSignature::new("float32"), + Float64 => TypeSignature::new("float64"), + Timestamp(tu, zone) => { + let params = if let Some(zone) = zone { + vec![time_unit_to_param(tu).into(), zone.as_ref().into()] + } else { + vec![time_unit_to_param(tu).into()] + }; + + TypeSignature::new_with_params("timestamp", params) + } + Binary => TypeSignature::new("binary"), + Utf8 => TypeSignature::new("string"), + Struct(fields) => { + let params = fields.iter().map(|f| f.name().into()).collect(); + TypeSignature::new_with_params("struct", params) + } + Extension(ext) => ext.type_signature(), + Null => TypeSignature::new("null"), + Int8 => TypeSignature::new("int8"), + Int16 => TypeSignature::new("int16"), + UInt8 => TypeSignature::new("uint8"), + UInt16 => TypeSignature::new("uint16"), + UInt32 => TypeSignature::new("uint32"), + Float16 => TypeSignature::new("float16"), + Date32 => TypeSignature::new("date_32"), + Date64 => TypeSignature::new("date_64"), + Time32(tu) => TypeSignature::new_with_params( + "time_32", + vec![time_unit_to_param(tu).into()], + ), + Time64(tu) => TypeSignature::new_with_params( + "time_64", + vec![time_unit_to_param(tu).into()], + ), + Duration(tu) => TypeSignature::new_with_params( + "duration", + vec![time_unit_to_param(tu).into()], + ), + Interval(iu) => { + let iu = match iu { + IntervalUnit::YearMonth => "year_month", + IntervalUnit::DayTime => "day_time", + IntervalUnit::MonthDayNano => "month_day_nano", + }; + TypeSignature::new_with_params("interval", vec![iu.into()]) + } + FixedSizeBinary(size) => TypeSignature::new_with_params( + "fixed_size_binary", + vec![size.to_string().into()], + ), + LargeBinary => TypeSignature::new("large_binary"), + LargeUtf8 => TypeSignature::new("large_utf_8"), + List(f) => TypeSignature::new_with_params( + "list", + vec![f.data_type().display_name().into()], + ), + FixedSizeList(f, size) => TypeSignature::new_with_params( + "fixed_size_list", + vec![f.data_type().display_name().into(), size.to_string().into()], + ), + LargeList(f) => TypeSignature::new_with_params( + "large_list", + vec![f.data_type().display_name().into()], + ), + Map(f, b) => TypeSignature::new_with_params( + "map", + vec![f.data_type().display_name().into(), b.to_string().into()], + ), + Decimal128(a, b) => TypeSignature::new_with_params( + "decimal_128", + vec![a.to_string().into(), b.to_string().into()], + ), + Decimal256(a, b) => TypeSignature::new_with_params( + "decimal_256", + vec![a.to_string().into(), b.to_string().into()], + ), + } + } + + fn physical_type(&self) -> DataType { + use crate::logical_type::LogicalType::*; + match self { + Boolean => DataType::Boolean, + Int32 => DataType::Int32, + Int64 => DataType::Int64, + UInt64 => DataType::UInt64, + Float32 => DataType::Float32, + Float64 => DataType::Float64, + Timestamp(tu, zone) => DataType::Timestamp(tu.clone(), zone.clone()), + Binary => DataType::Binary, + Utf8 => DataType::Utf8, + Struct(fields) => { + let fields = fields + .iter() + .map(|f| FieldRef::new(f.as_ref().clone().into())) + .collect::>(); + DataType::Struct(fields.into()) + } + Extension(ext) => ext.physical_type(), + Null => DataType::Null, + Int8 => DataType::Int8, + Int16 => DataType::Int16, + UInt8 => DataType::UInt8, + UInt16 => DataType::UInt16, + UInt32 => DataType::UInt32, + Float16 => DataType::Float16, + Date32 => DataType::Date32, + Date64 => DataType::Date64, + Time32(tu) => DataType::Time32(tu.to_owned()), + Time64(tu) => DataType::Time64(tu.to_owned()), + Duration(tu) => DataType::Duration(tu.to_owned()), + Interval(iu) => DataType::Interval(iu.to_owned()), + FixedSizeBinary(size) => DataType::FixedSizeBinary(size.to_owned()), + LargeBinary => DataType::LargeBinary, + LargeUtf8 => DataType::LargeUtf8, + List(f) => DataType::List(FieldRef::new(f.as_ref().clone().into())), + FixedSizeList(f, size) => DataType::FixedSizeList(FieldRef::new(f.as_ref().clone().into()), size.to_owned()), + LargeList(f) => DataType::LargeList(FieldRef::new(f.as_ref().clone().into())), + Map(f, b) => DataType::Map(FieldRef::new(f.as_ref().clone().into()), b.to_owned()), + Decimal128(a, b) => DataType::Decimal128(a.to_owned(), b.to_owned()), + Decimal256(a, b) => DataType::Decimal256(a.to_owned(), b.to_owned()), + } + } + + fn is_comparable(&self) -> bool { + use crate::logical_type::LogicalType::*; + match self { + Null + | Boolean + | Int8 + | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float16 + | Float32 + | Float64 + | Date32 + | Date64 + | Time32(_) + | Time64(_) + | Timestamp(_, _) + | Duration(_) + | Interval(_) + | Binary + | FixedSizeBinary(_) + | LargeBinary + | Utf8 + | LargeUtf8 + | Decimal128(_, _) + | Decimal256(_, _) => true, + Extension(ext) => ext.is_comparable(), + _ => false, + } + } + + fn is_orderable(&self) -> bool { + todo!() + } + + #[inline] + fn is_numeric(&self) -> bool { + use crate::logical_type::LogicalType::*; + match self { + UInt8 + | UInt16 + | UInt32 + | UInt64 + | Int8 + | Int16 + | Int32 + | Int64 + | Float16 + | Float32 + | Float64 + | Decimal128(_, _) + | Decimal256(_, _) => true, + Extension(t) => t.is_numeric(), + _ => false, + } + } + + #[inline] + fn is_floating(&self) -> bool { + use crate::logical_type::LogicalType::*; + match self { + Float16 | Float32 | Float64 => true, + Extension(t) => t.is_floating(), + _ => false, + } + } +} diff --git a/datafusion/common/src/logical_type/field.rs b/datafusion/common/src/logical_type/field.rs new file mode 100644 index 000000000000..3cd6f73fe374 --- /dev/null +++ b/datafusion/common/src/logical_type/field.rs @@ -0,0 +1,163 @@ +use std::collections::HashMap; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; +use arrow_schema::{DataType, Field}; +use crate::logical_type::extension::ExtensionType; +use crate::logical_type::fields::LogicalFields; +use crate::logical_type::LogicalType; +use crate::logical_type::type_signature::TypeSignature; + +pub type LogicalFieldRef = Arc; + +#[derive(Debug, Clone)] +pub struct LogicalField { + name: String, + data_type: LogicalType, + nullable: bool, + metadata: HashMap, +} + +impl From<&Field> for LogicalField { + fn from(value: &Field) -> Self { + Self { + name: value.name().clone(), + data_type: value.data_type().clone().into(), + nullable: value.is_nullable(), + metadata: value.metadata().clone() + } + } +} + +impl From for LogicalField { + fn from(value: Field) -> Self { + Self::from(&value) + } +} + +impl Into for LogicalField { + fn into(self) -> Field { + Field::new(self.name, self.data_type.physical_type(), self.nullable).with_metadata(self.metadata) + } +} + +impl PartialEq for LogicalField { + fn eq(&self, other: &Self) -> bool { + self.name == other.name + && self.data_type == other.data_type + && self.nullable == other.nullable + && self.metadata == other.metadata + } +} + +impl Eq for LogicalField {} + +impl Hash for LogicalField { + fn hash(&self, state: &mut H) { + self.name.hash(state); + self.data_type.hash(state); + self.nullable.hash(state); + + // ensure deterministic key order + let mut keys: Vec<&String> = self.metadata.keys().collect(); + keys.sort(); + for k in keys { + k.hash(state); + self.metadata.get(k).expect("key valid").hash(state); + } + } +} + +impl ExtensionType for LogicalField { + fn display_name(&self) -> &str { + &self.name + } + + fn type_signature(&self) -> TypeSignature { + TypeSignature::new(self.name()) + } + + fn physical_type(&self) -> DataType { + self.data_type.physical_type() + } + + fn is_comparable(&self) -> bool { + self.data_type.is_comparable() + } + + fn is_orderable(&self) -> bool { + self.data_type.is_orderable() + } + + fn is_numeric(&self) -> bool { + self.data_type.is_numeric() + } + + fn is_floating(&self) -> bool { + self.data_type.is_floating() + } +} + +impl LogicalField { + pub fn new(name: impl Into, data_type: LogicalType, nullable: bool) -> Self { + LogicalField { + name: name.into(), + data_type, + nullable, + metadata: HashMap::default(), + } + } + + pub fn new_list_field(data_type: LogicalType, nullable: bool) -> Self { + Self::new("item", data_type, nullable) + } + + pub fn new_struct(name: impl Into, fields: impl Into, nullable: bool) -> Self { + Self::new(name, LogicalType::Struct(fields.into()), nullable) + } + + pub fn name(&self) -> &str { + &self.name + } + + pub fn data_type(&self) -> &LogicalType { + &self.data_type + } + + pub fn is_nullable(&self) -> bool { + self.nullable + } + + pub fn metadata(&self) -> &HashMap { + &self.metadata + } + + #[inline] + pub fn with_name(mut self, name: impl Into) -> Self { + self.name = name.into(); + self + } + + #[inline] + pub fn with_nullable(mut self, nullable: bool) -> Self { + self.nullable = nullable; + self + } + + #[inline] + pub fn with_metadata(mut self, metadata: HashMap) -> Self { + self.metadata = metadata; + self + } + + #[inline] + pub fn with_data_type(mut self, data_type: LogicalType) -> Self { + self.data_type = data_type; + self + } +} + +impl std::fmt::Display for LogicalField { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{self:?}") + } +} diff --git a/datafusion/common/src/logical_type/fields.rs b/datafusion/common/src/logical_type/fields.rs new file mode 100644 index 000000000000..456835bc8d9b --- /dev/null +++ b/datafusion/common/src/logical_type/fields.rs @@ -0,0 +1,94 @@ +use std::ops::Deref; +use std::sync::Arc; +use arrow_schema::{Field, Fields}; +use crate::logical_type::field::{LogicalField, LogicalFieldRef}; + +#[derive(Clone, Eq, PartialEq, Hash)] +pub struct LogicalFields(Arc<[LogicalFieldRef]>); + +impl std::fmt::Debug for LogicalFields { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.0.as_ref().fmt(f) + } +} + +impl From for LogicalFields { + fn from(value: Fields) -> Self { + Self(value.into_iter().map(|v| LogicalFieldRef::new(LogicalField::from(v.as_ref()))).collect()) + } +} + +impl Into for LogicalFields { + fn into(self) -> Fields { + Fields::from( + self.iter() + .map(|f| f.as_ref().clone().into()) + .collect::>() + ) + } +} + +impl Default for LogicalFields { + fn default() -> Self { + Self::empty() + } +} + +impl FromIterator for LogicalFields { + fn from_iter>(iter: T) -> Self { + iter.into_iter().map(Arc::new).collect() + } +} + +impl FromIterator for LogicalFields { + fn from_iter>(iter: T) -> Self { + Self(iter.into_iter().collect()) + } +} + +impl From> for LogicalFields { + fn from(value: Vec) -> Self { + value.into_iter().collect() + } +} + +impl From> for LogicalFields { + fn from(value: Vec) -> Self { + Self(value.into()) + } +} + +impl From<&[LogicalFieldRef]> for LogicalFields { + fn from(value: &[LogicalFieldRef]) -> Self { + Self(value.into()) + } +} + +impl From<[LogicalFieldRef; N]> for LogicalFields { + fn from(value: [LogicalFieldRef; N]) -> Self { + Self(Arc::new(value)) + } +} + +impl Deref for LogicalFields { + type Target = [LogicalFieldRef]; + + fn deref(&self) -> &Self::Target { + self.0.as_ref() + } +} + +impl<'a> IntoIterator for &'a LogicalFields { + type Item = &'a LogicalFieldRef; + type IntoIter = std::slice::Iter<'a, LogicalFieldRef>; + + fn into_iter(self) -> Self::IntoIter { + self.0.iter() + } +} + +impl LogicalFields { + pub fn empty() -> Self { + Self(Arc::new([])) + } +} diff --git a/datafusion/common/src/logical_type/mod.rs b/datafusion/common/src/logical_type/mod.rs new file mode 100644 index 000000000000..fd464411a4ef --- /dev/null +++ b/datafusion/common/src/logical_type/mod.rs @@ -0,0 +1,156 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{fmt::Display, sync::Arc}; + +use arrow_schema::{DataType, IntervalUnit, TimeUnit}; + +use crate::logical_type::extension::{ExtensionType, ExtensionTypeRef}; +use crate::logical_type::field::{LogicalField, LogicalFieldRef}; +use crate::logical_type::fields::LogicalFields; + +pub mod type_signature; +pub mod extension; +pub mod registry; +pub mod schema; +pub mod field; +pub mod fields; + +#[derive(Clone, Debug)] +pub enum LogicalType { + Null, + Boolean, + Int8, + Int16, + Int32, + Int64, + UInt8, + UInt16, + UInt32, + UInt64, + Float16, + Float32, + Float64, + Date32, + Date64, + Time32(TimeUnit), + Time64(TimeUnit), + Timestamp(TimeUnit, Option>), + Duration(TimeUnit), + Interval(IntervalUnit), + Binary, + FixedSizeBinary(i32), + LargeBinary, + Utf8, + LargeUtf8, + List(LogicalFieldRef), + FixedSizeList(LogicalFieldRef, i32), + LargeList(LogicalFieldRef), + Struct(LogicalFields), + Map(LogicalFieldRef, bool), + Decimal128(u8, i8), + Decimal256(u8, i8), + Extension(ExtensionTypeRef), + // TODO: tbd union +} + +impl PartialEq for LogicalType { + fn eq(&self, other: &Self) -> bool { + self.type_signature() == other.type_signature() + } +} + +impl Eq for LogicalType {} + +impl std::hash::Hash for LogicalType { + fn hash(&self, state: &mut H) { + self.type_signature().hash(state) + } +} + +impl Display for LogicalType { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{self:?}") + } +} + +impl From<&DataType> for LogicalType { + fn from(value: &DataType) -> Self { + value.clone().into() + } +} + +impl From for LogicalType { + fn from(value: DataType) -> Self { + match value { + DataType::Null => LogicalType::Null, + DataType::Boolean => LogicalType::Boolean, + DataType::Int8 => LogicalType::Int8, + DataType::Int16 => LogicalType::Int16, + DataType::Int32 => LogicalType::Int32, + DataType::Int64 => LogicalType::Int64, + DataType::UInt8 => LogicalType::UInt8, + DataType::UInt16 => LogicalType::UInt16, + DataType::UInt32 => LogicalType::UInt32, + DataType::UInt64 => LogicalType::UInt64, + DataType::Float16 => LogicalType::Float16, + DataType::Float32 => LogicalType::Float32, + DataType::Float64 => LogicalType::Float64, + DataType::Timestamp(tu, z) => LogicalType::Timestamp(tu, z), + DataType::Date32 => LogicalType::Date32, + DataType::Date64 => LogicalType::Date64, + DataType::Time32(tu) => LogicalType::Time32(tu), + DataType::Time64(tu) => LogicalType::Time64(tu), + DataType::Duration(tu) => LogicalType::Duration(tu), + DataType::Interval(iu) => LogicalType::Interval(iu), + DataType::Binary | DataType::BinaryView => LogicalType::Binary, + DataType::FixedSizeBinary(len) => LogicalType::FixedSizeBinary(len), + DataType::LargeBinary => LogicalType::LargeBinary, + DataType::Utf8 | DataType::Utf8View => LogicalType::Utf8, + DataType::LargeUtf8 => LogicalType::LargeUtf8, + DataType::List(f) | DataType::ListView(f) => LogicalType::List(LogicalFieldRef::new(f.as_ref().into())), + DataType::FixedSizeList(f, len) => LogicalType::FixedSizeList(LogicalFieldRef::new(f.as_ref().into()), len), + DataType::LargeList(f) | DataType::LargeListView(f) => LogicalType::LargeList(LogicalFieldRef::new(f.as_ref().into())), + DataType::Struct(fields) => LogicalType::Struct(fields.into()), + DataType::Dictionary(_, dt) => dt.as_ref().into(), + DataType::Decimal128(p, s) => LogicalType::Decimal128(p, s), + DataType::Decimal256(p, s) => LogicalType::Decimal256(p, s), + DataType::Map(f, sorted) => LogicalType::Map(LogicalFieldRef::new(f.as_ref().into()), sorted), + DataType::RunEndEncoded(_, f) => f.data_type().into(), + DataType::Union(_, _) => unimplemented!(), // TODO: tbd union + } + } +} + +impl LogicalType { + + pub fn new_list(data_type: LogicalType, nullable: bool) -> Self { + LogicalType::List(Arc::new(LogicalField::new_list_field(data_type, nullable))) + } + + pub fn new_large_list(data_type: LogicalType, nullable: bool) -> Self { + LogicalType::LargeList(Arc::new(LogicalField::new_list_field(data_type, nullable))) + } + + pub fn new_fixed_size_list(data_type: LogicalType, size: i32, nullable: bool) -> Self { + LogicalType::FixedSizeList(Arc::new(LogicalField::new_list_field(data_type, nullable)), size) + } + + pub fn is_floating(&self) -> bool { + matches!(self, Self::Float16 | Self::Float32 | Self::Float64) + } +} diff --git a/datafusion/common/src/logical_type/registry.rs b/datafusion/common/src/logical_type/registry.rs new file mode 100644 index 000000000000..63d25bb23e91 --- /dev/null +++ b/datafusion/common/src/logical_type/registry.rs @@ -0,0 +1,51 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashMap; + +use crate::error::_plan_datafusion_err; +use crate::logical_type::extension::ExtensionTypeRef; +use crate::logical_type::type_signature::TypeSignature; + +pub trait TypeRegistry { + fn register_data_type( + &mut self, + extension_type: ExtensionTypeRef, + ) -> crate::Result>; + + fn data_type(&self, signature: &TypeSignature) -> crate::Result; +} + + +#[derive(Default, Debug)] +pub struct MemoryTypeRegistry { + types: HashMap, +} + +impl TypeRegistry for MemoryTypeRegistry { + fn register_data_type(&mut self, extension_type: ExtensionTypeRef) -> crate::Result> { + Ok(self.types.insert(extension_type.type_signature(), extension_type)) + } + + fn data_type(&self, signature: &TypeSignature) -> crate::Result { + self.types + .get(signature) + .cloned() + .ok_or_else(|| _plan_datafusion_err!("Type with signature {signature:?} not found")) + } +} + diff --git a/datafusion/common/src/logical_type/schema.rs b/datafusion/common/src/logical_type/schema.rs new file mode 100644 index 000000000000..f0c2854b4e64 --- /dev/null +++ b/datafusion/common/src/logical_type/schema.rs @@ -0,0 +1,159 @@ +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_schema::Schema; + +use crate::logical_type::field::{LogicalField, LogicalFieldRef}; +use crate::logical_type::fields::LogicalFields; + +#[derive(Debug, Default)] +pub struct LogicalSchemaBuilder { + fields: Vec, + metadata: HashMap, +} + +impl LogicalSchemaBuilder { + pub fn new() -> Self { + Self::default() + } + + pub fn with_capacity(capacity: usize) -> Self { + Self { + fields: Vec::with_capacity(capacity), + metadata: Default::default(), + } + } + + pub fn push(&mut self, field: impl Into) { + self.fields.push(field.into()) + } + + pub fn remove(&mut self, idx: usize) -> LogicalFieldRef { + self.fields.remove(idx) + } + + pub fn field(&mut self, idx: usize) -> &LogicalFieldRef { + &mut self.fields[idx] + } + + pub fn field_mut(&mut self, idx: usize) -> &mut LogicalFieldRef { + &mut self.fields[idx] + } + + pub fn metadata(&mut self) -> &HashMap { + &self.metadata + } + + pub fn metadata_mut(&mut self) -> &mut HashMap { + &mut self.metadata + } + + pub fn reverse(&mut self) { + self.fields.reverse(); + } + + pub fn finish(self) -> LogicalSchema { + LogicalSchema { + fields: self.fields.into(), + metadata: self.metadata, + } + } +} + +impl From<&LogicalFields> for LogicalSchemaBuilder { + fn from(value: &LogicalFields) -> Self { + Self { + fields: value.to_vec(), + metadata: Default::default(), + } + } +} + +impl From for LogicalSchemaBuilder { + fn from(value: LogicalFields) -> Self { + Self { + fields: value.to_vec(), + metadata: Default::default(), + } + } +} + +impl From<&LogicalSchema> for LogicalSchemaBuilder { + fn from(value: &LogicalSchema) -> Self { + Self::from(value.clone()) + } +} + +impl From for LogicalSchemaBuilder { + fn from(value: LogicalSchema) -> Self { + Self { + fields: value.fields.to_vec(), + metadata: value.metadata, + } + } +} + +impl Extend for LogicalSchemaBuilder { + fn extend>(&mut self, iter: T) { + let iter = iter.into_iter(); + self.fields.reserve(iter.size_hint().0); + for f in iter { + self.push(f) + } + } +} + +impl Extend for LogicalSchemaBuilder { + fn extend>(&mut self, iter: T) { + let iter = iter.into_iter(); + self.fields.reserve(iter.size_hint().0); + for f in iter { + self.push(f) + } + } +} + +pub type LogicalSchemaRef = Arc; + +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct LogicalSchema { + pub fields: LogicalFields, + pub metadata: HashMap, +} + +impl From for LogicalSchema { + fn from(value: Schema) -> Self { + Self { + fields: value.fields.into(), + metadata: value.metadata, + } + } +} + +impl LogicalSchema { + pub fn new(fields: impl Into) -> Self { + Self::new_with_metadata(fields, HashMap::new()) + } + + #[inline] + pub fn new_with_metadata(fields: impl Into, metadata: HashMap) -> Self { + Self { + fields: fields.into(), + metadata, + } + } + + #[inline] + pub fn with_metadata(mut self, metadata: HashMap) -> Self { + self.metadata = metadata; + self + } + + pub fn metadata(&self) -> &HashMap { + &self.metadata + } + + pub fn field(&self, i: usize) -> &LogicalFieldRef { + &self.fields[i] + } +} \ No newline at end of file diff --git a/datafusion/common/src/logical_type/type_signature.rs b/datafusion/common/src/logical_type/type_signature.rs new file mode 100644 index 000000000000..4b46d040e5c4 --- /dev/null +++ b/datafusion/common/src/logical_type/type_signature.rs @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct TypeSignature { + // **func_name**(p1, p2) + name: Arc, + // func_name(**p1**, **p2**) + params: Vec>, +} + +impl TypeSignature { + pub fn new(name: impl Into>) -> Self { + Self::new_with_params(name, vec![]) + } + + pub fn new_with_params( + name: impl Into>, + params: Vec>, + ) -> Self { + Self { + name: name.into(), + params, + } + } +} diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index 8d61bad97b9f..e389d520d834 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -17,8 +17,8 @@ use crate::error::{_plan_datafusion_err, _plan_err}; use crate::{Result, ScalarValue}; -use arrow_schema::DataType; use std::collections::HashMap; +use crate::logical_type::LogicalType; /// The parameter value corresponding to the placeholder #[derive(Debug, Clone)] @@ -31,7 +31,7 @@ pub enum ParamValues { impl ParamValues { /// Verify parameter list length and type - pub fn verify(&self, expect: &[DataType]) -> Result<()> { + pub fn verify(&self, expect: &[LogicalType]) -> Result<()> { match self { ParamValues::List(list) => { // Verify if the number of params matches the number of values @@ -46,7 +46,7 @@ impl ParamValues { // Verify if the types of the params matches the types of the values let iter = expect.iter().zip(list.iter()); for (i, (param_type, value)) in iter.enumerate() { - if *param_type != value.data_type() { + if *param_type != value.data_type().into() { return _plan_err!( "Expected parameter of type {:?}, got {:?} at index {}", param_type, diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 5b9c4a223de6..67e959984b3d 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -54,10 +54,11 @@ use arrow::{ }, }; use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, ScalarBuffer}; -use arrow_schema::{UnionFields, UnionMode}; +use arrow_schema::{FieldRef, UnionFields, UnionMode}; use half::f16; pub use struct_builder::ScalarStructBuilder; +use crate::logical_type::LogicalType; /// A dynamically typed, nullable single value. /// @@ -3270,6 +3271,118 @@ impl TryFrom<&DataType> for ScalarValue { } } + +impl TryFrom for ScalarValue { + type Error = DataFusionError; + + /// Create a Null instance of ScalarValue for this datatype + fn try_from(datatype: LogicalType) -> Result { + (&datatype).try_into() + } +} + +impl TryFrom<&LogicalType> for ScalarValue { + type Error = DataFusionError; + + /// Create a Null instance of ScalarValue for this datatype + fn try_from(data_type: &LogicalType) -> Result { + Ok(match data_type { + LogicalType::Boolean => ScalarValue::Boolean(None), + LogicalType::Float16 => ScalarValue::Float16(None), + LogicalType::Float64 => ScalarValue::Float64(None), + LogicalType::Float32 => ScalarValue::Float32(None), + LogicalType::Int8 => ScalarValue::Int8(None), + LogicalType::Int16 => ScalarValue::Int16(None), + LogicalType::Int32 => ScalarValue::Int32(None), + LogicalType::Int64 => ScalarValue::Int64(None), + LogicalType::UInt8 => ScalarValue::UInt8(None), + LogicalType::UInt16 => ScalarValue::UInt16(None), + LogicalType::UInt32 => ScalarValue::UInt32(None), + LogicalType::UInt64 => ScalarValue::UInt64(None), + LogicalType::Decimal128(precision, scale) => { + ScalarValue::Decimal128(None, *precision, *scale) + } + LogicalType::Decimal256(precision, scale) => { + ScalarValue::Decimal256(None, *precision, *scale) + } + LogicalType::Utf8 => ScalarValue::Utf8(None), + LogicalType::LargeUtf8 => ScalarValue::LargeUtf8(None), + LogicalType::Binary => ScalarValue::Binary(None), + LogicalType::FixedSizeBinary(len) => ScalarValue::FixedSizeBinary(*len, None), + LogicalType::LargeBinary => ScalarValue::LargeBinary(None), + LogicalType::Date32 => ScalarValue::Date32(None), + LogicalType::Date64 => ScalarValue::Date64(None), + LogicalType::Time32(TimeUnit::Second) => ScalarValue::Time32Second(None), + LogicalType::Time32(TimeUnit::Millisecond) => { + ScalarValue::Time32Millisecond(None) + } + LogicalType::Time64(TimeUnit::Microsecond) => { + ScalarValue::Time64Microsecond(None) + } + LogicalType::Time64(TimeUnit::Nanosecond) => ScalarValue::Time64Nanosecond(None), + LogicalType::Timestamp(TimeUnit::Second, tz_opt) => { + ScalarValue::TimestampSecond(None, tz_opt.clone()) + } + LogicalType::Timestamp(TimeUnit::Millisecond, tz_opt) => { + ScalarValue::TimestampMillisecond(None, tz_opt.clone()) + } + LogicalType::Timestamp(TimeUnit::Microsecond, tz_opt) => { + ScalarValue::TimestampMicrosecond(None, tz_opt.clone()) + } + LogicalType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { + ScalarValue::TimestampNanosecond(None, tz_opt.clone()) + } + LogicalType::Interval(IntervalUnit::YearMonth) => { + ScalarValue::IntervalYearMonth(None) + } + LogicalType::Interval(IntervalUnit::DayTime) => { + ScalarValue::IntervalDayTime(None) + } + LogicalType::Interval(IntervalUnit::MonthDayNano) => { + ScalarValue::IntervalMonthDayNano(None) + } + LogicalType::Duration(TimeUnit::Second) => ScalarValue::DurationSecond(None), + LogicalType::Duration(TimeUnit::Millisecond) => { + ScalarValue::DurationMillisecond(None) + } + LogicalType::Duration(TimeUnit::Microsecond) => { + ScalarValue::DurationMicrosecond(None) + } + LogicalType::Duration(TimeUnit::Nanosecond) => { + ScalarValue::DurationNanosecond(None) + } + // `ScalaValue::List` contains single element `ListArray`. + LogicalType::List(field_ref) => ScalarValue::List(Arc::new( + GenericListArray::new_null(FieldRef::new(field_ref.as_ref().clone().into()), 1), + )), + // `ScalarValue::LargeList` contains single element `LargeListArray`. + LogicalType::LargeList(field_ref) => ScalarValue::LargeList(Arc::new( + GenericListArray::new_null(FieldRef::new(field_ref.as_ref().clone().into()), 1), + )), + // `ScalaValue::FixedSizeList` contains single element `FixedSizeList`. + LogicalType::FixedSizeList(field_ref, fixed_length) => { + ScalarValue::FixedSizeList(Arc::new(FixedSizeListArray::new_null( + FieldRef::new(field_ref.as_ref().clone().into()), + *fixed_length, + 1, + ))) + } + LogicalType::Struct(fields) => ScalarValue::Struct( + new_null_array(&DataType::Struct(fields.clone().into()), 1) + .as_struct() + .to_owned() + .into(), + ), + LogicalType::Null => ScalarValue::Null, + _ => { + return _not_impl_err!( + "Can't create a scalar from data_type \"{data_type:?}\"" + ); + } + }) + } +} + macro_rules! format_option { ($F:expr, $EXPR:expr) => {{ match $EXPR { diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 8e55da8c3ad0..afcfe6615c0a 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -58,6 +58,8 @@ use datafusion_expr::{ use datafusion_functions_aggregate::expr_fn::{avg, count, median, stddev, sum}; use async_trait::async_trait; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; /// Contains options that control how data is /// written out from a DataFrame @@ -666,7 +668,7 @@ impl DataFrame { original_schema_fields .clone() .filter(|f| { - !matches!(f.data_type(), DataType::Binary | DataType::Boolean) + !matches!(f.data_type(), LogicalType::Binary | LogicalType::Boolean) }) .map(|f| min(col(f.name())).alias(f.name())) .collect::>(), @@ -677,7 +679,7 @@ impl DataFrame { original_schema_fields .clone() .filter(|f| { - !matches!(f.data_type(), DataType::Binary | DataType::Boolean) + !matches!(f.data_type(), LogicalType::Binary | LogicalType::Boolean) }) .map(|f| max(col(f.name())).alias(f.name())) .collect::>(), @@ -1285,7 +1287,7 @@ impl DataFrame { let plan = LogicalPlanBuilder::insert_into( self.plan, table_name.to_owned(), - &arrow_schema, + &arrow_schema.into(), write_options.overwrite, )? .build()?; @@ -1695,6 +1697,7 @@ mod tests { use arrow::array::{self, Int32Array}; use datafusion_common::{Constraint, Constraints}; + use datafusion_common::logical_type::LogicalType; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::{ array_agg, cast, create_udf, expr, lit, BuiltInWindowFunction, @@ -2362,7 +2365,7 @@ mod tests { let field = df.schema().field(0); // There are two columns named 'c', one from the input of the aggregate and the other from the output. // Select should return the column from the output of the aggregate, which is a list. - assert!(matches!(field.data_type(), DataType::List(_))); + assert!(matches!(field.data_type(), LogicalType::List(_))); Ok(()) } @@ -3169,7 +3172,7 @@ mod tests { .await? .select_columns(&["c2", "c3"])? .limit(0, Some(1))? - .with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?; + .with_column("sum", cast(col("c2") + col("c3"), LogicalType::Int64))?; let df_results = df.clone().collect().await?; df.clone().show().await?; @@ -3271,7 +3274,7 @@ mod tests { .await? .select_columns(&["c2", "c3"])? .limit(0, Some(1))? - .with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?; + .with_column("sum", cast(col("c2") + col("c3"), LogicalType::Int64))?; let cached_df = df.clone().cache().await?; diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index c1ce4cc5b6c5..9e3d0f03ee9e 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -44,6 +44,7 @@ use datafusion_expr::{Expr, Volatility}; use datafusion_physical_expr::create_physical_expr; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; +use datafusion_common::logical_type::field::LogicalField; /// Check whether the given expression can be resolved using only the columns `col_names`. /// This means that if this function returns true: @@ -264,7 +265,7 @@ async fn prune_partitions( let df_schema = DFSchema::from_unqualified_fields( partition_cols .iter() - .map(|(n, d)| Field::new(n, d.clone(), true)) + .map(|(n, d)| LogicalField::new(n, d.clone().into(), true)) .collect(), Default::default(), )?; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ea4d396a14cb..ec0ee09e9785 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -55,6 +55,7 @@ use async_trait::async_trait; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; +use datafusion_common::logical_type::schema::LogicalSchema; /// Configuration for creating a [`ListingTable`] #[derive(Debug, Clone)] @@ -787,7 +788,7 @@ impl TableProvider for ListingTable { let filters = if let Some(expr) = conjunction(filters.to_vec()) { // NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns. - let table_df_schema = self.table_schema.as_ref().clone().to_dfschema()?; + let table_df_schema = LogicalSchema::from(self.table_schema.as_ref().clone()).to_dfschema()?; let filters = create_physical_expr(&expr, &table_df_schema, state.execution_props())?; Some(filters) @@ -1879,7 +1880,7 @@ mod tests { // Therefore, we will have 8 partitions in the final plan. // Create an insert plan to insert the source data into the initial table let insert_into_table = - LogicalPlanBuilder::insert_into(scan_plan, "t", &schema, false)?.build()?; + LogicalPlanBuilder::insert_into(scan_plan, "t", &schema.as_ref().clone().into(), false)?.build()?; // Create a physical plan from the insert plan let plan = session_ctx .state() diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index aab42285a0b2..3126e8dee5de 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -624,7 +624,7 @@ mod tests { let scan_plan = LogicalPlanBuilder::scan("source", source, None)?.build()?; // Create an insert plan to insert the source data into the initial table let insert_into_table = - LogicalPlanBuilder::insert_into(scan_plan, "t", &schema, false)?.build()?; + LogicalPlanBuilder::insert_into(scan_plan, "t", &schema.as_ref().clone().into(), false)?.build()?; // Create a physical plan from the insert plan let plan = session_ctx .state() diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index f5d3c7a6410d..f09e77a9a8be 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -626,7 +626,6 @@ fn create_output_array( #[cfg(test)] mod tests { use arrow_array::Int32Array; - use super::*; use crate::{test::columns, test_util::aggr_test_schema}; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index f9cce5f783ff..b73bbd8dad66 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -425,6 +425,9 @@ mod test { use parquet::file::reader::{FileReader, SerializedFileReader}; use rand::prelude::*; + use datafusion_common::logical_type::LogicalType; + + // We should ignore predicate that read non-primitive columns #[test] fn test_filter_candidate_builder_ignore_complex_types() { @@ -471,10 +474,10 @@ mod test { ]); // The parquet file with `file_schema` just has `bigint_col` and `float_col` column, and don't have the `int_col` - let expr = col("bigint_col").eq(cast(col("int_col"), DataType::Int64)); + let expr = col("bigint_col").eq(cast(col("int_col"), LogicalType::Int64)); let expr = logical2physical(&expr, &table_schema); let expected_candidate_expr = - col("bigint_col").eq(cast(lit(ScalarValue::Int32(None)), DataType::Int64)); + col("bigint_col").eq(cast(lit(ScalarValue::Int32(None)), LogicalType::Int64)); let expected_candidate_expr = logical2physical(&expected_candidate_expr, &table_schema); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 9bc79805746f..87d64c4a43ba 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -411,9 +411,8 @@ mod tests { use crate::datasource::physical_plan::parquet::reader::ParquetFileReader; use crate::physical_plan::metrics::ExecutionPlanMetricsSet; - use arrow::datatypes::DataType::Decimal128; use arrow::datatypes::{DataType, Field}; - use datafusion_common::Result; + use datafusion_common::{Result, logical_type::LogicalType::*}; use datafusion_expr::{cast, col, lit, Expr}; use datafusion_physical_expr::planner::logical2physical; @@ -819,7 +818,7 @@ mod tests { .with_scale(0) .with_precision(9); let schema_descr = get_test_schema_descr(vec![field]); - let expr = cast(col("c1"), DataType::Decimal128(11, 2)).gt(cast( + let expr = cast(col("c1"), Decimal128(11, 2)).gt(cast( lit(ScalarValue::Decimal128(Some(500), 5, 2)), Decimal128(11, 2), )); @@ -936,7 +935,7 @@ mod tests { .with_byte_len(16); let schema_descr = get_test_schema_descr(vec![field]); // cast the type of c1 to decimal(28,3) - let left = cast(col("c1"), DataType::Decimal128(28, 3)); + let left = cast(col("c1"), Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); @@ -1010,7 +1009,7 @@ mod tests { .with_byte_len(16); let schema_descr = get_test_schema_descr(vec![field]); // cast the type of c1 to decimal(28,3) - let left = cast(col("c1"), DataType::Decimal128(28, 3)); + let left = cast(col("c1"), Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 0b880ddbf81b..793570d4c707 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -42,7 +42,7 @@ use crate::physical_optimizer::optimizer::PhysicalOptimizer; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; use crate::{functions, functions_aggregate}; -use arrow_schema::{DataType, SchemaRef}; +use arrow_schema::{SchemaRef}; use async_trait::async_trait; use chrono::{DateTime, Utc}; use datafusion_common::alias::AliasGenerator; @@ -84,6 +84,7 @@ use std::fmt::Debug; use std::sync::Arc; use url::Url; use uuid::Uuid; +use datafusion_common::logical_type::LogicalType; /// Execution context for registering data sources and executing queries. /// See [`SessionContext`] for a higher level API. @@ -1002,7 +1003,7 @@ impl<'a> ContextProvider for SessionContextProvider<'a> { self.state.window_functions().get(name).cloned() } - fn get_variable_type(&self, variable_names: &[String]) -> Option { + fn get_variable_type(&self, variable_names: &[String]) -> Option { if variable_names.is_empty() { return None; } @@ -1219,7 +1220,7 @@ impl<'a> SessionSimplifyProvider<'a> { impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> { fn is_boolean_type(&self, expr: &Expr) -> datafusion_common::Result { - Ok(expr.get_type(self.df_schema)? == DataType::Boolean) + Ok(expr.get_type(self.df_schema)? == LogicalType::Boolean) } fn nullable(&self, expr: &Expr) -> datafusion_common::Result { @@ -1230,7 +1231,7 @@ impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> { self.state.execution_props() } - fn get_data_type(&self, expr: &Expr) -> datafusion_common::Result { + fn get_data_type(&self, expr: &Expr) -> datafusion_common::Result { expr.get_type(self.df_schema) } } diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index e8f2f34abda0..8624a80bfce3 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -1571,6 +1571,7 @@ mod tests { use datafusion_expr::expr::InList; use datafusion_expr::{cast, is_null, try_cast, Expr}; use datafusion_physical_expr::planner::logical2physical; + use datafusion_common::logical_type::LogicalType; #[derive(Debug, Default)] /// Mock statistic provider for tests @@ -2607,13 +2608,13 @@ mod tests { // test cast(c1 as int64) = 1 // test column on the left - let expr = cast(col("c1"), DataType::Int64).eq(lit(ScalarValue::Int64(Some(1)))); + let expr = cast(col("c1"), LogicalType::Int64).eq(lit(ScalarValue::Int64(Some(1)))); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right - let expr = lit(ScalarValue::Int64(Some(1))).eq(cast(col("c1"), DataType::Int64)); + let expr = lit(ScalarValue::Int64(Some(1))).eq(cast(col("c1"), LogicalType::Int64)); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); @@ -2625,14 +2626,14 @@ mod tests { // test column on the left let expr = - try_cast(col("c1"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(1)))); + try_cast(col("c1"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(1)))); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = - lit(ScalarValue::Int64(Some(1))).lt(try_cast(col("c1"), DataType::Int64)); + lit(ScalarValue::Int64(Some(1))).lt(try_cast(col("c1"), LogicalType::Int64)); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); @@ -2645,7 +2646,7 @@ mod tests { let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); // test cast(c1 as int64) in int64(1, 2, 3) let expr = Expr::InList(InList::new( - Box::new(cast(col("c1"), DataType::Int64)), + Box::new(cast(col("c1"), LogicalType::Int64)), vec![ lit(ScalarValue::Int64(Some(1))), lit(ScalarValue::Int64(Some(2))), @@ -2670,7 +2671,7 @@ mod tests { assert_eq!(predicate_expr.to_string(), expected_expr); let expr = Expr::InList(InList::new( - Box::new(cast(col("c1"), DataType::Int64)), + Box::new(cast(col("c1"), LogicalType::Int64)), vec![ lit(ScalarValue::Int64(Some(1))), lit(ScalarValue::Int64(Some(2))), @@ -2724,7 +2725,7 @@ mod tests { prune_with_expr( // with cast column to other type - cast(col("s1"), DataType::Decimal128(14, 3)) + cast(col("s1"), LogicalType::Decimal128(14, 3)) .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))), &schema, &TestStatistics::new().with( @@ -2739,7 +2740,7 @@ mod tests { prune_with_expr( // with try cast column to other type - try_cast(col("s1"), DataType::Decimal128(14, 3)) + try_cast(col("s1"), LogicalType::Decimal128(14, 3)) .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))), &schema, &TestStatistics::new().with( @@ -2826,7 +2827,7 @@ mod tests { prune_with_expr( // filter with cast - cast(col("s2"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(5)))), + cast(col("s2"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(5)))), &schema, &statistics, &[false, true, true, true], @@ -3054,7 +3055,7 @@ mod tests { prune_with_expr( // cast(i as utf8) <= 0 - cast(col("i"), DataType::Utf8).lt_eq(lit("0")), + cast(col("i"), LogicalType::Utf8).lt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3062,7 +3063,7 @@ mod tests { prune_with_expr( // try_cast(i as utf8) <= 0 - try_cast(col("i"), DataType::Utf8).lt_eq(lit("0")), + try_cast(col("i"), LogicalType::Utf8).lt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3070,7 +3071,7 @@ mod tests { prune_with_expr( // cast(-i as utf8) >= 0 - cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")), + cast(Expr::Negative(Box::new(col("i"))), LogicalType::Utf8).gt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3078,7 +3079,7 @@ mod tests { prune_with_expr( // try_cast(-i as utf8) >= 0 - try_cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")), + try_cast(Expr::Negative(Box::new(col("i"))), LogicalType::Utf8).gt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3119,14 +3120,14 @@ mod tests { let expected_ret = &[true, false, false, true, false]; prune_with_expr( - cast(col("i"), DataType::Int64).eq(lit(0i64)), + cast(col("i"), LogicalType::Int64).eq(lit(0i64)), &schema, &statistics, expected_ret, ); prune_with_expr( - try_cast(col("i"), DataType::Int64).eq(lit(0i64)), + try_cast(col("i"), LogicalType::Int64).eq(lit(0i64)), &schema, &statistics, expected_ret, @@ -3149,7 +3150,7 @@ mod tests { let expected_ret = &[true, true, true, true, true]; prune_with_expr( - cast(col("i"), DataType::Utf8).eq(lit("0")), + cast(col("i"), LogicalType::Utf8).eq(lit("0")), &schema, &statistics, expected_ret, @@ -3304,7 +3305,7 @@ mod tests { prune_with_expr( // i > int64(0) - col("i").gt(cast(lit(ScalarValue::Int64(Some(0))), DataType::Int32)), + col("i").gt(cast(lit(ScalarValue::Int64(Some(0))), LogicalType::Int32)), &schema, &statistics, expected_ret, @@ -3312,7 +3313,7 @@ mod tests { prune_with_expr( // cast(i as int64) > int64(0) - cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), + cast(col("i"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), &schema, &statistics, expected_ret, @@ -3320,7 +3321,7 @@ mod tests { prune_with_expr( // try_cast(i as int64) > int64(0) - try_cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), + try_cast(col("i"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), &schema, &statistics, expected_ret, @@ -3328,7 +3329,7 @@ mod tests { prune_with_expr( // `-cast(i as int64) < 0` convert to `cast(i as int64) > -0` - Expr::Negative(Box::new(cast(col("i"), DataType::Int64))) + Expr::Negative(Box::new(cast(col("i"), LogicalType::Int64))) .lt(lit(ScalarValue::Int64(Some(0)))), &schema, &statistics, @@ -3357,7 +3358,7 @@ mod tests { assert_eq!(result_right.to_string(), right_input.to_string()); // cast op lit - let left_input = cast(col("a"), DataType::Decimal128(20, 3)); + let left_input = cast(col("a"), LogicalType::Decimal128(20, 3)); let left_input = logical2physical(&left_input, &schema); let right_input = lit(ScalarValue::Decimal128(Some(12), 20, 3)); let right_input = logical2physical(&right_input, &schema); @@ -3372,7 +3373,7 @@ mod tests { assert_eq!(result_right.to_string(), right_input.to_string()); // try_cast op lit - let left_input = try_cast(col("a"), DataType::Int64); + let left_input = try_cast(col("a"), LogicalType::Int64); let left_input = logical2physical(&left_input, &schema); let right_input = lit(ScalarValue::Int64(Some(12))); let right_input = logical2physical(&right_input, &schema); @@ -3391,7 +3392,7 @@ mod tests { // this cast is not supported let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); let df_schema = DFSchema::try_from(schema.clone()).unwrap(); - let left_input = cast(col("a"), DataType::Int64); + let left_input = cast(col("a"), LogicalType::Int64); let left_input = logical2physical(&left_input, &schema); let right_input = lit(ScalarValue::Int64(Some(12))); let right_input = logical2physical(&right_input, &schema); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index a0390cf16c17..164fa2ed088a 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2243,6 +2243,8 @@ mod tests { use arrow::array::{ArrayRef, DictionaryArray, Int32Array}; use arrow::datatypes::{DataType, Field, Int32Type}; use datafusion_common::{assert_contains, DFSchemaRef, TableReference}; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::LogicalType; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{col, lit, LogicalPlanBuilder, UserDefinedLogicalNodeCore}; @@ -2485,12 +2487,11 @@ mod tests { let expected_error: &str = "Error during planning: \ Extension planner for NoOp created an ExecutionPlan with mismatched schema. \ LogicalPlan schema: \ - DFSchema { inner: Schema { fields: \ - [Field { name: \"a\", \ + DFSchema { inner: LogicalSchema { fields: \ + [LogicalField { name: \"a\", \ data_type: Int32, \ nullable: false, \ - dict_id: 0, \ - dict_is_ordered: false, metadata: {} }], \ + metadata: {} }], \ metadata: {} }, field_qualifiers: [None], \ functional_dependencies: FunctionalDependencies { deps: [] } }, \ ExecutionPlan schema: Schema { fields: \ @@ -2764,7 +2765,7 @@ mod tests { Self { schema: DFSchemaRef::new( DFSchema::from_unqualified_fields( - vec![Field::new("a", DataType::Int32, false)].into(), + vec![LogicalField::new("a", LogicalType::Int32, false)].into(), HashMap::new(), ) .unwrap(), diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index e8550a79cb0e..1212b572df3e 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -206,7 +206,7 @@ pub fn assert_fields_eq(plan: &LogicalPlan, expected: Vec<&str>) { .schema() .fields() .iter() - .map(|f| f.name().clone()) + .map(|f| f.name().to_string()) .collect(); assert_eq!(actual, expected); } diff --git a/datafusion/core/src/test/variable.rs b/datafusion/core/src/test/variable.rs index 38207b42cb7b..7cf5a7f3ec2b 100644 --- a/datafusion/core/src/test/variable.rs +++ b/datafusion/core/src/test/variable.rs @@ -20,7 +20,7 @@ use crate::error::Result; use crate::scalar::ScalarValue; use crate::variable::VarProvider; -use arrow::datatypes::DataType; +use datafusion_common::logical_type::LogicalType; /// System variable #[derive(Default, Debug)] @@ -40,8 +40,8 @@ impl VarProvider for SystemVar { Ok(ScalarValue::from(s)) } - fn get_type(&self, _: &[String]) -> Option { - Some(DataType::Utf8) + fn get_type(&self, _: &[String]) -> Option { + Some(LogicalType::Utf8) } } @@ -67,11 +67,11 @@ impl VarProvider for UserDefinedVar { } } - fn get_type(&self, var_names: &[String]) -> Option { + fn get_type(&self, var_names: &[String]) -> Option { if var_names[0] != "@integer" { - Some(DataType::Utf8) + Some(LogicalType::Utf8) } else { - Some(DataType::Int32) + Some(LogicalType::Int32) } } } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 9f06ad9308ab..e8e33badc0af 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -42,6 +42,7 @@ use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; +use datafusion_common::logical_type::schema::LogicalSchema; /// a ParquetFile that has been created for testing. pub struct TestParquetFile { @@ -153,7 +154,7 @@ impl TestParquetFile { extensions: None, }); - let df_schema = self.schema.clone().to_dfschema_ref()?; + let df_schema = LogicalSchema::from(self.schema.as_ref().clone()).to_dfschema_ref()?; // run coercion on the filters to coerce types etc. let props = ExecutionProps::new(); diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index 1c55c48fea40..b1d615405cac 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -31,6 +31,7 @@ use datafusion::prelude::*; use datafusion::assert_batches_eq; use datafusion_common::{DFSchema, ScalarValue}; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::Alias; use datafusion_expr::ExprSchemable; use datafusion_functions_aggregate::expr_fn::{approx_median, approx_percentile_cont}; @@ -376,7 +377,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { // the arg2 parameter is a complex expr, but it can be evaluated to the literal value let alias_expr = Expr::Alias(Alias::new( - cast(lit(0.5), DataType::Float32), + cast(lit(0.5), LogicalType::Float32), None::<&str>, "arg_2".to_string(), )); @@ -949,7 +950,7 @@ async fn test_fn_substr() -> Result<()> { #[tokio::test] async fn test_cast() -> Result<()> { - let expr = cast(col("b"), DataType::Float64); + let expr = cast(col("b"), LogicalType::Float64); let expected = [ "+--------+", "| test.b |", @@ -1052,7 +1053,7 @@ async fn test_fn_decode() -> Result<()> { let expr = decode(encode(col("a"), lit("hex")), lit("hex")) // need to cast to utf8 otherwise the default display of binary array is hex // so it looks like nothing is done - .cast_to(&DataType::Utf8, &df_schema)?; + .cast_to(&LogicalType::Utf8, &df_schema)?; let expected = [ "+------------------------------------------------+", diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c3bc2fcca2b5..d7228dcf3689 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -47,6 +47,7 @@ use datafusion::prelude::{CsvReadOptions, ParquetReadOptions}; use datafusion::test_util::{parquet_test_data, populate_csv_partitions}; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; use datafusion_common::{assert_contains, DataFusionError, ScalarValue, UnnestOptions}; +use datafusion_common::logical_type::LogicalType; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_expr::expr::{GroupingSet, Sort}; @@ -256,7 +257,7 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { scalar_subquery(Arc::new( ctx.table("t2") .await? - .filter(out_ref_col(DataType::UInt32, "t1.a").eq(col("t2.a")))? + .filter(out_ref_col(LogicalType::UInt32, "t1.a").eq(col("t2.a")))? .aggregate(vec![], vec![count(wildcard())])? .select(vec![col(count(wildcard()).to_string())])? .into_unoptimized_plan(), @@ -746,8 +747,8 @@ async fn join_with_alias_filter() -> Result<()> { // filter: t1.a + CAST(Int64(1), UInt32) = t2.a + CAST(Int64(2), UInt32) as t1.a + 1 = t2.a + 2 let filter = Expr::eq( - col("t1.a") + lit(3i64).cast_to(&DataType::UInt32, &t1_schema)?, - col("t2.a") + lit(1i32).cast_to(&DataType::UInt32, &t2_schema)?, + col("t1.a") + lit(3i64).cast_to(&LogicalType::UInt32, &t1_schema)?, + col("t2.a") + lit(1i32).cast_to(&LogicalType::UInt32, &t2_schema)?, ) .alias("t1.b + 1 = t2.a + 2"); @@ -1622,7 +1623,7 @@ async fn consecutive_projection_same_schema() -> Result<()> { // Add `t` column full of nulls let df = df - .with_column("t", cast(Expr::Literal(ScalarValue::Null), DataType::Int32)) + .with_column("t", cast(Expr::Literal(ScalarValue::Null), LogicalType::Int32)) .unwrap(); df.clone().show().await.unwrap(); @@ -1925,8 +1926,8 @@ impl VarProvider for HardcodedIntProvider { Ok(ScalarValue::Int64(Some(1234))) } - fn get_type(&self, _: &[String]) -> Option { - Some(DataType::Int64) + fn get_type(&self, _: &[String]) -> Option { + Some(LogicalType::Int64) } } diff --git a/datafusion/core/tests/expr_api/parse_sql_expr.rs b/datafusion/core/tests/expr_api/parse_sql_expr.rs index 991579b5a350..e082aaed322b 100644 --- a/datafusion/core/tests/expr_api/parse_sql_expr.rs +++ b/datafusion/core/tests/expr_api/parse_sql_expr.rs @@ -15,9 +15,11 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::{DataType, Field, Schema}; use datafusion::prelude::{CsvReadOptions, SessionContext}; use datafusion_common::{DFSchemaRef, Result, ToDFSchema}; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::Expr; use datafusion_sql::unparser::Unparser; @@ -27,10 +29,10 @@ use datafusion_sql::unparser::Unparser; /// b: Int32 /// s: Float32 fn schema() -> DFSchemaRef { - Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, false), - Field::new("c", DataType::Float32, false), + LogicalSchema::new(vec![ + LogicalField::new("a", LogicalType::Int32, true), + LogicalField::new("b", LogicalType::Int32, false), + LogicalField::new("c", LogicalType::Float32, false), ]) .to_dfschema_ref() .unwrap() diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 9ce47153ba4a..38e1e8cd1551 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -37,6 +37,9 @@ use datafusion_optimizer::optimizer::Optimizer; use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyExpressions}; use datafusion_optimizer::{OptimizerContext, OptimizerRule}; use std::sync::Arc; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; /// In order to simplify expressions, DataFusion must have information /// about the expressions. @@ -56,7 +59,7 @@ impl SimplifyInfo for MyInfo { fn is_boolean_type(&self, expr: &Expr) -> Result { Ok(matches!( expr.get_type(self.schema.as_ref())?, - DataType::Boolean + LogicalType::Boolean )) } @@ -68,7 +71,7 @@ impl SimplifyInfo for MyInfo { &self.execution_props } - fn get_data_type(&self, expr: &Expr) -> Result { + fn get_data_type(&self, expr: &Expr) -> Result { expr.get_type(self.schema.as_ref()) } } @@ -88,10 +91,10 @@ impl From for MyInfo { /// b: Int32 /// s: Utf8 fn schema() -> DFSchemaRef { - Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, false), - Field::new("s", DataType::Utf8, false), + LogicalSchema::new(vec![ + LogicalField::new("a", LogicalType::Int32, true), + LogicalField::new("b", LogicalType::Int32, false), + LogicalField::new("s", LogicalType::Utf8, false), ]) .to_dfschema_ref() .unwrap() @@ -190,7 +193,7 @@ fn make_udf_add(volatility: Volatility) -> Arc { } fn cast_to_int64_expr(expr: Expr) -> Expr { - Expr::Cast(Cast::new(expr.into(), DataType::Int64)) + Expr::Cast(Cast::new(expr.into(), LogicalType::Int64)) } fn to_timestamp_expr(arg: impl Into) -> Expr { @@ -281,7 +284,7 @@ fn select_date_plus_interval() -> Result<()> { let schema = table_scan.schema(); let date_plus_interval_expr = to_timestamp_expr(ts_string) - .cast_to(&DataType::Date32, schema)? + .cast_to(&LogicalType::Date32, schema)? + Expr::Literal(ScalarValue::IntervalDayTime(Some(IntervalDayTime { days: 123, milliseconds: 0, @@ -483,15 +486,15 @@ fn multiple_now() -> Result<()> { // ------------------------------ fn expr_test_schema() -> DFSchemaRef { - Schema::new(vec![ - Field::new("c1", DataType::Utf8, true), - Field::new("c2", DataType::Boolean, true), - Field::new("c3", DataType::Int64, true), - Field::new("c4", DataType::UInt32, true), - Field::new("c1_non_null", DataType::Utf8, false), - Field::new("c2_non_null", DataType::Boolean, false), - Field::new("c3_non_null", DataType::Int64, false), - Field::new("c4_non_null", DataType::UInt32, false), + LogicalSchema::new(vec![ + LogicalField::new("c1", LogicalType::Utf8, true), + LogicalField::new("c2", LogicalType::Boolean, true), + LogicalField::new("c3", LogicalType::Int64, true), + LogicalField::new("c4", LogicalType::UInt32, true), + LogicalField::new("c1_non_null", LogicalType::Utf8, false), + LogicalField::new("c2_non_null", LogicalType::Boolean, false), + LogicalField::new("c3_non_null", LogicalType::Int64, false), + LogicalField::new("c4_non_null", LogicalType::UInt32, false), ]) .to_dfschema_ref() .unwrap() @@ -688,8 +691,8 @@ fn test_simplify_concat() { #[test] fn test_simplify_cycles() { // cast(now() as int64) < cast(to_timestamp(0) as int64) + i64::MAX - let expr = cast(now(), DataType::Int64) - .lt(cast(to_timestamp(vec![lit(0)]), DataType::Int64) + lit(i64::MAX)); + let expr = cast(now(), LogicalType::Int64) + .lt(cast(to_timestamp(vec![lit(0)]), LogicalType::Int64) + lit(i64::MAX)); let expected = lit(true); test_simplify_with_cycle_count(expr, expected, 3); } diff --git a/datafusion/core/tests/optimizer_integration.rs b/datafusion/core/tests/optimizer_integration.rs index 39f745cd3309..40c91c4bd4ff 100644 --- a/datafusion/core/tests/optimizer_integration.rs +++ b/datafusion/core/tests/optimizer_integration.rs @@ -25,6 +25,8 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use arrow_schema::{Fields, SchemaBuilder}; use datafusion_common::config::ConfigOptions; +use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::LogicalType; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{plan_err, DFSchema, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::{Interval, NullableInterval}; @@ -203,7 +205,7 @@ impl ContextProvider for MyContextProvider { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -259,7 +261,7 @@ fn test_nested_schema_nullability() { let dfschema = DFSchema::from_field_specific_qualified_schema( vec![Some("table_name".into()), None], - &Arc::new(schema), + &LogicalSchemaRef::new(schema.into()), ) .unwrap(); diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 15efd4bcd9dd..cfe74b12e25a 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -35,6 +35,7 @@ use datafusion_physical_expr::create_physical_expr; use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; +use datafusion_common::logical_type::schema::LogicalSchema; async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { let object_store_url = ObjectStoreUrl::local_filesystem(); @@ -66,7 +67,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { extensions: None, }; - let df_schema = schema.clone().to_dfschema().unwrap(); + let df_schema = LogicalSchema::from(schema.as_ref().clone()).to_dfschema().unwrap(); let execution_props = ExecutionProps::new(); let predicate = create_physical_expr(&filter, &df_schema, &execution_props).unwrap(); diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 5e3c44c039ab..2603290442b6 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -39,6 +39,8 @@ use datafusion_expr::{ use datafusion_functions_array::range::range_udf; use parking_lot::Mutex; use sqlparser::ast::Ident; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; /// test that casting happens on udfs. /// c11 is f32, but `custom_sqrt` requires f64. Casting happens but the logical plan and @@ -518,14 +520,14 @@ impl ScalarUDFImpl for CastToI64UDF { // SimplifyInfo so we have to replicate some of the casting logic here. let source_type = info.get_data_type(&arg)?; - let new_expr = if source_type == DataType::Int64 { + let new_expr = if source_type == LogicalType::Int64 { // the argument's data type is already the correct type arg } else { // need to use an actual cast to get the correct type Expr::Cast(datafusion_expr::Cast { expr: Box::new(arg), - data_type: DataType::Int64, + data_type: LogicalType::Int64, }) }; // return the newly written argument to DataFusion @@ -645,7 +647,7 @@ impl ScalarUDFImpl for TakeUDF { ); }; - arg_exprs.get(take_idx).unwrap().get_type(schema) + arg_exprs.get(take_idx).unwrap().get_type(schema).map(|t| t.physical_type()) } // The actual implementation @@ -687,8 +689,8 @@ async fn verify_udf_return_type() -> Result<()> { // The output schema should be // * type of column smallint_col (int32) // * type of column double_col (float64) - assert_eq!(schema.field(0).data_type(), &DataType::Int32); - assert_eq!(schema.field(1).data_type(), &DataType::Float64); + assert_eq!(schema.field(0).data_type(), &LogicalType::Int32); + assert_eq!(schema.field(1).data_type(), &LogicalType::Float64); let expected = [ "+-------+-------+", @@ -835,13 +837,16 @@ impl TryFrom for ScalarFunctionWrapper { .expect("Expression has to be defined!"), return_type: definition .return_type - .expect("Return type has to be defined!"), + .expect("Return type has to be defined!") + .physical_type(), + // TODO(@notfilippo): avoid conversion to physical type signature: Signature::exact( definition .args .unwrap_or_default() .into_iter() - .map(|a| a.data_type) + // TODO(@notfilippo): avoid conversion to physical type + .map(|a| a.data_type.physical_type()) .collect(), definition .params @@ -990,10 +995,10 @@ async fn create_scalar_function_from_sql_statement_postgres_syntax() -> Result<( value: "name".into(), quote_style: None, }), - data_type: DataType::Utf8, + data_type: LogicalType::Utf8, default_expr: None, }]), - return_type: Some(DataType::Int32), + return_type: Some(LogicalType::Int32), params: CreateFunctionBody { language: Some(Ident { value: "plrust".into(), diff --git a/datafusion/expr/src/conditional_expressions.rs b/datafusion/expr/src/conditional_expressions.rs index 7a2bf4b6c44a..176a2002501a 100644 --- a/datafusion/expr/src/conditional_expressions.rs +++ b/datafusion/expr/src/conditional_expressions.rs @@ -18,9 +18,9 @@ //! Conditional expressions use crate::expr::Case; use crate::{expr_schema::ExprSchemable, Expr}; -use arrow::datatypes::DataType; use datafusion_common::{plan_err, DFSchema, Result}; use std::collections::HashSet; +use datafusion_common::logical_type::LogicalType; /// Helper struct for building [Expr::Case] pub struct CaseBuilder { @@ -70,18 +70,18 @@ impl CaseBuilder { then_expr.push(e.as_ref().to_owned()); } - let then_types: Vec = then_expr + let then_types: Vec = then_expr .iter() .map(|e| match e { Expr::Literal(_) => e.get_type(&DFSchema::empty()), - _ => Ok(DataType::Null), + _ => Ok(LogicalType::Null), }) .collect::>>()?; - if then_types.contains(&DataType::Null) { + if then_types.contains(&LogicalType::Null) { // cannot verify types until execution type } else { - let unique_types: HashSet<&DataType> = then_types.iter().collect(); + let unique_types: HashSet<&LogicalType> = then_types.iter().collect(); if unique_types.len() != 1 { return plan_err!( "CASE expression 'then' values had multiple data types: {unique_types:?}" diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 846b627b2242..0c084c8cadb7 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -33,7 +33,7 @@ use crate::{ }; use crate::{window_frame, Volatility}; -use arrow::datatypes::{DataType, FieldRef}; +use arrow::datatypes::DataType; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -41,6 +41,8 @@ use datafusion_common::{ internal_err, plan_err, Column, DFSchema, Result, ScalarValue, TableReference, }; use sqlparser::ast::NullTreatment; +use datafusion_common::logical_type::field::LogicalFieldRef; +use datafusion_common::logical_type::LogicalType; /// Represents logical expressions such as `A + 1`, or `CAST(c1 AS int)`. /// @@ -153,7 +155,7 @@ use sqlparser::ast::NullTreatment; /// Field::new("c2", DataType::Float64, false), /// ]); /// // DFSchema is a an Arrow schema with optional relation name -/// let df_schema = DFSchema::try_from_qualified_schema("t1", &arrow_schema) +/// let df_schema = DFSchema::try_from_qualified_schema("t1", &arrow_schema.into()) /// .unwrap(); /// /// // Form Vec with an expression for each column in the schema @@ -223,7 +225,7 @@ pub enum Expr { /// A named reference to a qualified filed in a schema. Column(Column), /// A named reference to a variable in a registry. - ScalarVariable(DataType, Vec), + ScalarVariable(LogicalType, Vec), /// A constant value. Literal(ScalarValue), /// A binary expression such as "age > 21" @@ -317,7 +319,7 @@ pub enum Expr { Placeholder(Placeholder), /// A place holder which hold a reference to a qualified field /// in the outer query, used for correlated sub queries. - OuterReferenceColumn(DataType, Column), + OuterReferenceColumn(LogicalType, Column), /// Unnest expression Unnest(Unnest), } @@ -339,8 +341,8 @@ impl From for Expr { /// useful for creating [`Expr`] from a [`DFSchema`]. /// /// See example on [`Expr`] -impl<'a> From<(Option<&'a TableReference>, &'a FieldRef)> for Expr { - fn from(value: (Option<&'a TableReference>, &'a FieldRef)) -> Self { +impl<'a> From<(Option<&'a TableReference>, &'a LogicalFieldRef)> for Expr { + fn from(value: (Option<&'a TableReference>, &'a LogicalFieldRef)) -> Self { Expr::from(Column::from(value)) } } @@ -558,13 +560,13 @@ pub enum GetFieldAccess { pub struct Cast { /// The expression being cast pub expr: Box, - /// The `DataType` the expression will yield - pub data_type: DataType, + /// The `LogicalType` the expression will yield + pub data_type: LogicalType, } impl Cast { /// Create a new Cast expression - pub fn new(expr: Box, data_type: DataType) -> Self { + pub fn new(expr: Box, data_type: LogicalType) -> Self { Self { expr, data_type } } } @@ -574,13 +576,13 @@ impl Cast { pub struct TryCast { /// The expression being cast pub expr: Box, - /// The `DataType` the expression will yield - pub data_type: DataType, + /// The `LogicalType` the expression will yield + pub data_type: LogicalType, } impl TryCast { /// Create a new TryCast expression - pub fn new(expr: Box, data_type: DataType) -> Self { + pub fn new(expr: Box, data_type: LogicalType) -> Self { Self { expr, data_type } } } @@ -926,12 +928,12 @@ pub struct Placeholder { /// The identifier of the parameter, including the leading `$` (e.g, `"$1"` or `"$foo"`) pub id: String, /// The type the parameter will be filled in with - pub data_type: Option, + pub data_type: Option, } impl Placeholder { /// Create a new Placeholder expression - pub fn new(id: String, data_type: Option) -> Self { + pub fn new(id: String, data_type: Option) -> Self { Self { id, data_type } } } @@ -2060,7 +2062,7 @@ fn write_name(w: &mut W, e: &Expr) -> Result<()> { Expr::InSubquery(InSubquery { negated: true, .. }) => w.write_str("NOT IN")?, Expr::InSubquery(InSubquery { negated: false, .. }) => w.write_str("IN")?, Expr::ScalarSubquery(subquery) => { - w.write_str(subquery.subquery.schema().field(0).name().as_str())?; + w.write_str(subquery.subquery.schema().field(0).name())?; } Expr::Unnest(Unnest { expr }) => { w.write_str("unnest(")?; @@ -2222,7 +2224,7 @@ mod test { fn format_cast() -> Result<()> { let expr = Expr::Cast(Cast { expr: Box::new(Expr::Literal(ScalarValue::Float32(Some(1.23)))), - data_type: DataType::Utf8, + data_type: LogicalType::Utf8, }); let expected_canonical = "CAST(Float32(1.23) AS Utf8)"; assert_eq!(expected_canonical, expr.canonical_name()); @@ -2255,7 +2257,7 @@ mod test { fn test_collect_expr() -> Result<()> { // single column { - let expr = &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)); + let expr = &Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Float64)); let columns = expr.column_refs(); assert_eq!(1, columns.len()); assert!(columns.contains(&Column::from_name("a"))); diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 8b0213fd52fd..2db73e5c413b 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -40,6 +40,7 @@ use std::any::Any; use std::fmt::Debug; use std::ops::Not; use std::sync::Arc; +use datafusion_common::logical_type::LogicalType; /// Create a column expression based on a qualified or unqualified column name. Will /// normalize unquoted identifiers according to SQL rules (identifiers will become lowercase). @@ -62,7 +63,7 @@ pub fn col(ident: impl Into) -> Expr { /// Create an out reference column which hold a reference that has been resolved to a field /// outside of the current plan. -pub fn out_ref_col(dt: DataType, ident: impl Into) -> Expr { +pub fn out_ref_col(dt: LogicalType, ident: impl Into) -> Expr { Expr::OuterReferenceColumn(dt, ident.into()) } @@ -308,12 +309,12 @@ pub fn rollup(exprs: Vec) -> Expr { } /// Create a cast expression -pub fn cast(expr: Expr, data_type: DataType) -> Expr { +pub fn cast(expr: Expr, data_type: LogicalType) -> Expr { Expr::Cast(Cast::new(Box::new(expr), data_type)) } /// Create a try cast expression -pub fn try_cast(expr: Expr, data_type: DataType) -> Expr { +pub fn try_cast(expr: Expr, data_type: LogicalType) -> Expr { Expr::TryCast(TryCast::new(Box::new(expr), data_type)) } diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 1441374bdba3..ea3d6f353aa9 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -286,7 +286,9 @@ mod test { use super::*; use crate::expr::Sort; use crate::{col, lit, Cast}; - use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::LogicalType; + use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_common::ScalarValue; #[derive(Default)] @@ -407,10 +409,10 @@ mod test { ) -> DFSchema { let fields = fields .iter() - .map(|f| Arc::new(Field::new(f.to_string(), DataType::Int8, false))) + .map(|f| Arc::new(LogicalField::new(f.to_string(), LogicalType::Int8, false))) .collect::>(); - let schema = Arc::new(Schema::new(fields)); - DFSchema::from_field_specific_qualified_schema(qualifiers, &schema).unwrap() + let schema = Arc::new(LogicalSchema::new(fields)); + DFSchema::from_field_specific_qualified_schema(qualifiers, &schema.into()).unwrap() } #[test] @@ -440,7 +442,7 @@ mod test { // cast data types test_rewrite( col("a"), - Expr::Cast(Cast::new(Box::new(col("a")), DataType::Int32)), + Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Int32)), ); // change literal type from i32 to i64 diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index 4b56ca3d1c2e..8084995bdf63 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -154,7 +154,7 @@ mod test { use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; - + use datafusion_common::logical_type::LogicalType; use crate::{ cast, col, lit, logical_plan::builder::LogicalTableSource, min, test::function_stub::avg, try_cast, LogicalPlanBuilder, @@ -270,13 +270,13 @@ mod test { let cases = vec![ TestCase { desc: "Cast is preserved by rewrite_sort_cols_by_aggs", - input: sort(cast(col("c2"), DataType::Int64)), - expected: sort(cast(col("c2").alias("c2"), DataType::Int64)), + input: sort(cast(col("c2"), LogicalType::Int64)), + expected: sort(cast(col("c2").alias("c2"), LogicalType::Int64)), }, TestCase { desc: "TryCast is preserved by rewrite_sort_cols_by_aggs", - input: sort(try_cast(col("c2"), DataType::Int64)), - expected: sort(try_cast(col("c2").alias("c2"), DataType::Int64)), + input: sort(try_cast(col("c2"), LogicalType::Int64)), + expected: sort(try_cast(col("c2").alias("c2"), LogicalType::Int64)), }, ]; diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 8bb655eda575..90204ac0dcd0 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -26,18 +26,20 @@ use crate::type_coercion::functions::{ }; use crate::{utils, LogicalPlan, Projection, Subquery, WindowFunctionDefinition}; use arrow::compute::can_cast_types; -use arrow::datatypes::{DataType, Field}; use datafusion_common::{ internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, ExprSchema, Result, TableReference, }; use std::collections::HashMap; use std::sync::Arc; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; /// trait to allow expr to typable with respect to a schema pub trait ExprSchemable { /// given a schema, return the type of the expr - fn get_type(&self, schema: &dyn ExprSchema) -> Result; + fn get_type(&self, schema: &dyn ExprSchema) -> Result; /// given a schema, return the nullability of the expr fn nullable(&self, input_schema: &dyn ExprSchema) -> Result; @@ -49,14 +51,14 @@ pub trait ExprSchemable { fn to_field( &self, input_schema: &dyn ExprSchema, - ) -> Result<(Option, Arc)>; + ) -> Result<(Option, Arc)>; /// cast to a type with respect to a schema - fn cast_to(self, cast_to_type: &DataType, schema: &dyn ExprSchema) -> Result; + fn cast_to(self, cast_to_type: &LogicalType, schema: &dyn ExprSchema) -> Result; /// given a schema, return the type and nullability of the expr fn data_type_and_nullable(&self, schema: &dyn ExprSchema) - -> Result<(DataType, bool)>; + -> Result<(LogicalType, bool)>; } impl ExprSchemable for Expr { @@ -98,7 +100,7 @@ impl ExprSchemable for Expr { /// expression refers to a column that does not exist in the /// schema, or when the expression is incorrectly typed /// (e.g. `[utf8] + [bool]`). - fn get_type(&self, schema: &dyn ExprSchema) -> Result { + fn get_type(&self, schema: &dyn ExprSchema) -> Result { match self { Expr::Alias(Alias { expr, name, .. }) => match &**expr { Expr::Placeholder(Placeholder { data_type, .. }) => match &data_type { @@ -111,7 +113,7 @@ impl ExprSchemable for Expr { Expr::Column(c) => Ok(schema.data_type(c)?.clone()), Expr::OuterReferenceColumn(ty, _) => Ok(ty.clone()), Expr::ScalarVariable(ty, _) => Ok(ty.clone()), - Expr::Literal(l) => Ok(l.data_type()), + Expr::Literal(l) => Ok(l.data_type().into()), Expr::Case(case) => case.when_then_expr[0].1.get_type(schema), Expr::Cast(Cast { data_type, .. }) | Expr::TryCast(TryCast { data_type, .. }) => Ok(data_type.clone()), @@ -119,13 +121,13 @@ impl ExprSchemable for Expr { let arg_data_type = expr.get_type(schema)?; // Unnest's output type is the inner type of the list match arg_data_type{ - DataType::List(field) | DataType::LargeList(field) | DataType::FixedSizeList(field, _) =>{ + LogicalType::List(field) | LogicalType::LargeList(field) | LogicalType::FixedSizeList(field, _) =>{ Ok(field.data_type().clone()) } - DataType::Struct(_) => { + LogicalType::Struct(_) => { Ok(arg_data_type) } - DataType::Null => { + LogicalType::Null => { not_impl_err!("unnest() does not support null yet") } _ => { @@ -138,28 +140,40 @@ impl ExprSchemable for Expr { .iter() .map(|e| e.get_type(schema)) .collect::>>()?; - // verify that function is invoked with correct number and type of arguments as defined in `TypeSignature` - data_types_with_scalar_udf(&arg_data_types, func).map_err(|err| { - plan_datafusion_err!( - "{} {}", - err, - utils::generate_signature_error_msg( - func.name(), - func.signature().clone(), - &arg_data_types, - ) - ) - })?; - // perform additional function arguments validation (due to limited - // expressiveness of `TypeSignature`), then infer return type - Ok(func.return_type_from_exprs(args, schema, &arg_data_types)?) + // TODO(@notfilippo): not convert to DataType + let arg_data_types = arg_data_types + .into_iter() + .map(|e| e.physical_type()) + .collect::>(); + + // verify that function is invoked with correct number and type of arguments as defined in `TypeSignature` + data_types_with_scalar_udf(&arg_data_types, func).map_err(|err| { + plan_datafusion_err!( + "{} {}", + err, + utils::generate_signature_error_msg( + func.name(), + func.signature().clone(), + &arg_data_types, + ) + ) + })?; + + // perform additional function arguments validation (due to limited + // expressiveness of `TypeSignature`), then infer return type + Ok(func.return_type_from_exprs(args, schema, &arg_data_types)?.into()) } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { let data_types = args .iter() .map(|e| e.get_type(schema)) .collect::>>()?; + // TODO(@notfilippo): not convert to DataType + let data_types = data_types + .into_iter() + .map(|e| e.physical_type()) + .collect::>(); let nullability = args .iter() .map(|e| e.nullable(schema)) @@ -177,10 +191,10 @@ impl ExprSchemable for Expr { ) ) })?; - Ok(fun.return_type(&new_types, &nullability)?) + Ok(fun.return_type(&new_types, &nullability)?.into()) } _ => { - fun.return_type(&data_types, &nullability) + Ok(fun.return_type(&data_types, &nullability)?.into()) } } } @@ -193,9 +207,14 @@ impl ExprSchemable for Expr { .iter() .map(|e| e.nullable(schema)) .collect::>>()?; + // TODO(@notfilippo): not convert to DataType + let data_types = data_types + .into_iter() + .map(|e| e.physical_type()) + .collect::>(); match func_def { AggregateFunctionDefinition::BuiltIn(fun) => { - fun.return_type(&data_types, &nullability) + Ok(fun.return_type(&data_types, &nullability)?.into()) } AggregateFunctionDefinition::UDF(fun) => { let new_types = data_types_with_aggregate_udf(&data_types, fun).map_err(|err| { @@ -209,7 +228,7 @@ impl ExprSchemable for Expr { ) ) })?; - Ok(fun.return_type(&new_types)?) + Ok(fun.return_type(&new_types)?.into()) } } } @@ -225,7 +244,7 @@ impl ExprSchemable for Expr { | Expr::IsUnknown(_) | Expr::IsNotTrue(_) | Expr::IsNotFalse(_) - | Expr::IsNotUnknown(_) => Ok(DataType::Boolean), + | Expr::IsNotUnknown(_) => Ok(LogicalType::Boolean), Expr::ScalarSubquery(subquery) => { Ok(subquery.subquery.schema().field(0).data_type().clone()) } @@ -233,8 +252,9 @@ impl ExprSchemable for Expr { ref left, ref right, ref op, - }) => get_result_type(&left.get_type(schema)?, op, &right.get_type(schema)?), - Expr::Like { .. } | Expr::SimilarTo { .. } => Ok(DataType::Boolean), + // TODO(@notfilippo): do not convert to physical type + }) => Ok(get_result_type(&left.get_type(schema)?.physical_type(), op, &right.get_type(schema)?.physical_type())?.into()), + Expr::Like { .. } | Expr::SimilarTo { .. } => Ok(LogicalType::Boolean), Expr::Placeholder(Placeholder { data_type, .. }) => { data_type.clone().ok_or_else(|| { plan_datafusion_err!("Placeholder type could not be resolved. Make sure that the placeholder is bound to a concrete type, e.g. by providing parameter values.") @@ -244,12 +264,12 @@ impl ExprSchemable for Expr { // Wildcard do not really have a type and do not appear in projections match qualifier { Some(_) => internal_err!("QualifiedWildcard expressions are not valid in a logical query plan"), - None => Ok(DataType::Null) + None => Ok(LogicalType::Null) } } Expr::GroupingSet(_) => { // grouping sets do not really have a type and do not appear in projections - Ok(DataType::Null) + Ok(LogicalType::Null) } } } @@ -392,7 +412,7 @@ impl ExprSchemable for Expr { fn data_type_and_nullable( &self, schema: &dyn ExprSchema, - ) -> Result<(DataType, bool)> { + ) -> Result<(LogicalType, bool)> { match self { Expr::Alias(Alias { expr, name, .. }) => match &**expr { Expr::Placeholder(Placeholder { data_type, .. }) => match &data_type { @@ -411,7 +431,7 @@ impl ExprSchemable for Expr { .map(|(d, n)| (d.clone(), n)), Expr::OuterReferenceColumn(ty, _) => Ok((ty.clone(), true)), Expr::ScalarVariable(ty, _) => Ok((ty.clone(), true)), - Expr::Literal(l) => Ok((l.data_type(), l.is_null())), + Expr::Literal(l) => Ok((l.data_type().into(), l.is_null())), Expr::IsNull(_) | Expr::IsNotNull(_) | Expr::IsTrue(_) @@ -420,7 +440,7 @@ impl ExprSchemable for Expr { | Expr::IsNotTrue(_) | Expr::IsNotFalse(_) | Expr::IsNotUnknown(_) - | Expr::Exists { .. } => Ok((DataType::Boolean, false)), + | Expr::Exists { .. } => Ok((LogicalType::Boolean, false)), Expr::ScalarSubquery(subquery) => Ok(( subquery.subquery.schema().field(0).data_type().clone(), subquery.subquery.schema().field(0).is_nullable(), @@ -432,7 +452,8 @@ impl ExprSchemable for Expr { }) => { let left = left.data_type_and_nullable(schema)?; let right = right.data_type_and_nullable(schema)?; - Ok((get_result_type(&left.0, op, &right.0)?, left.1 || right.1)) + // TODO(@notfilippo): do not convert to physical type + Ok((get_result_type(&left.0.physical_type(), op, &right.0.physical_type())?.into(), left.1 || right.1)) } _ => Ok((self.get_type(schema)?, self.nullable(schema)?)), } @@ -445,13 +466,13 @@ impl ExprSchemable for Expr { fn to_field( &self, input_schema: &dyn ExprSchema, - ) -> Result<(Option, Arc)> { + ) -> Result<(Option, Arc)> { match self { Expr::Column(c) => { let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; Ok(( c.relation.clone(), - Field::new(&c.name, data_type, nullable) + LogicalField::new(&c.name, data_type, nullable) .with_metadata(self.metadata(input_schema)?) .into(), )) @@ -460,7 +481,7 @@ impl ExprSchemable for Expr { let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; Ok(( relation.clone(), - Field::new(name, data_type, nullable) + LogicalField::new(name, data_type, nullable) .with_metadata(self.metadata(input_schema)?) .into(), )) @@ -469,7 +490,7 @@ impl ExprSchemable for Expr { let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; Ok(( None, - Field::new(self.display_name()?, data_type, nullable) + LogicalField::new(self.display_name()?, data_type, nullable) .with_metadata(self.metadata(input_schema)?) .into(), )) @@ -483,7 +504,7 @@ impl ExprSchemable for Expr { /// /// This function errors when it is impossible to cast the /// expression to the target [arrow::datatypes::DataType]. - fn cast_to(self, cast_to_type: &DataType, schema: &dyn ExprSchema) -> Result { + fn cast_to(self, cast_to_type: &LogicalType, schema: &dyn ExprSchema) -> Result { let this_type = self.get_type(schema)?; if this_type == *cast_to_type { return Ok(self); @@ -493,7 +514,8 @@ impl ExprSchemable for Expr { // like all of the binary expressions below. Perhaps Expr should track the // type of the expression? - if can_cast_types(&this_type, cast_to_type) { + // TODO(@notfilippo): The basis for whether cast can be executed should be the logical type + if can_cast_types(&this_type.physical_type(), &cast_to_type.physical_type()) { match self { Expr::ScalarSubquery(subquery) => { Ok(Expr::ScalarSubquery(cast_subquery(subquery, cast_to_type)?)) @@ -507,7 +529,7 @@ impl ExprSchemable for Expr { } /// cast subquery in InSubquery/ScalarSubquery to a given type. -pub fn cast_subquery(subquery: Subquery, cast_to_type: &DataType) -> Result { +pub fn cast_subquery(subquery: Subquery, cast_to_type: &LogicalType) -> Result { if subquery.subquery.schema().field(0).data_type() == cast_to_type { return Ok(subquery); } @@ -574,7 +596,7 @@ mod tests { fn test_between_nullability() { let get_schema = |nullable| { MockExprSchema::new() - .with_data_type(DataType::Int32) + .with_data_type(LogicalType::Int32) .with_nullable(nullable) }; @@ -598,7 +620,7 @@ mod tests { fn test_inlist_nullability() { let get_schema = |nullable| { MockExprSchema::new() - .with_data_type(DataType::Int32) + .with_data_type(LogicalType::Int32) .with_nullable(nullable) }; @@ -623,7 +645,7 @@ mod tests { fn test_like_nullability() { let get_schema = |nullable| { MockExprSchema::new() - .with_data_type(DataType::Utf8) + .with_data_type(LogicalType::Utf8) .with_nullable(nullable) }; @@ -639,8 +661,8 @@ mod tests { fn expr_schema_data_type() { let expr = col("foo"); assert_eq!( - DataType::Utf8, - expr.get_type(&MockExprSchema::new().with_data_type(DataType::Utf8)) + LogicalType::Utf8, + expr.get_type(&MockExprSchema::new().with_data_type(LogicalType::Utf8)) .unwrap() ); } @@ -651,7 +673,7 @@ mod tests { meta.insert("bar".to_string(), "buzz".to_string()); let expr = col("foo"); let schema = MockExprSchema::new() - .with_data_type(DataType::Int32) + .with_data_type(LogicalType::Int32) .with_metadata(meta.clone()); // col and alias should be metadata-preserving @@ -662,14 +684,14 @@ mod tests { assert_eq!( HashMap::new(), expr.clone() - .cast_to(&DataType::Int64, &schema) + .cast_to(&LogicalType::Int64, &schema) .unwrap() .metadata(&schema) .unwrap() ); let schema = DFSchema::from_unqualified_fields( - vec![Field::new("foo", DataType::Int32, true).with_metadata(meta.clone())] + vec![LogicalField::new("foo", LogicalType::Int32, true).with_metadata(meta.clone())] .into(), HashMap::new(), ) @@ -682,7 +704,7 @@ mod tests { #[derive(Debug)] struct MockExprSchema { nullable: bool, - data_type: DataType, + data_type: LogicalType, error_on_nullable: bool, metadata: HashMap, } @@ -691,7 +713,7 @@ mod tests { fn new() -> Self { Self { nullable: false, - data_type: DataType::Null, + data_type: LogicalType::Null, error_on_nullable: false, metadata: HashMap::new(), } @@ -702,7 +724,7 @@ mod tests { self } - fn with_data_type(mut self, data_type: DataType) -> Self { + fn with_data_type(mut self, data_type: LogicalType) -> Self { self.data_type = data_type; self } @@ -727,7 +749,7 @@ mod tests { } } - fn data_type(&self, _col: &Column) -> Result<&DataType> { + fn data_type(&self, _col: &Column) -> Result<&LogicalType> { Ok(&self.data_type) } @@ -735,7 +757,7 @@ mod tests { Ok(&self.metadata) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&DataType, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)> { Ok((self.data_type(col)?, self.nullable(col)?)) } } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 1f1175088227..91f72a1386d8 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -48,14 +48,15 @@ use crate::{ WriteOp, }; -use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use arrow::datatypes::{Schema, SchemaRef}; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::{ - get_target_functional_dependencies, internal_err, not_impl_err, plan_datafusion_err, - plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, - TableReference, ToDFSchema, UnnestOptions, -}; +use datafusion_common::{get_target_functional_dependencies, internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, TableReference, UnnestOptions, ToDFSchema}; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; /// Default table name for unnamed table pub const UNNAMED_TABLE: &str = "?table?"; @@ -182,26 +183,26 @@ impl LogicalPlanBuilder { } let empty_schema = DFSchema::empty(); - let mut field_types: Vec = Vec::with_capacity(n_cols); + let mut field_types: Vec = Vec::with_capacity(n_cols); for j in 0..n_cols { - let mut common_type: Option = None; + let mut common_type: Option = None; for (i, row) in values.iter().enumerate() { let value = &row[j]; let data_type = value.get_type(&empty_schema)?; - if data_type == DataType::Null { + if data_type == LogicalType::Null { continue; } if let Some(prev_type) = common_type { // get common type of each column values. - let Some(new_type) = values_coercion(&data_type, &prev_type) else { + let Some(new_type) = values_coercion(&data_type.physical_type(), &prev_type.physical_type()) else { return plan_err!("Inconsistent data type across values list at row {i} column {j}. Was {prev_type} but found {data_type}"); }; - common_type = Some(new_type); + common_type = Some(new_type.into()); } else { common_type = Some(data_type.clone()); } } - field_types.push(common_type.unwrap_or(DataType::Utf8)); + field_types.push(common_type.unwrap_or(LogicalType::Utf8)); } // wrap cast if data type is not same as common type. for row in &mut values { @@ -220,7 +221,7 @@ impl LogicalPlanBuilder { .map(|(j, data_type)| { // naming is following convention https://www.postgresql.org/docs/current/queries-values.html let name = &format!("column{}", j + 1); - Field::new(name, data_type.clone(), true) + LogicalField::new(name, data_type.clone(), true) }) .collect::>(); let dfschema = DFSchema::from_unqualified_fields(fields.into(), HashMap::new())?; @@ -289,7 +290,7 @@ impl LogicalPlanBuilder { pub fn insert_into( input: LogicalPlan, table_name: impl Into, - table_schema: &Schema, + table_schema: &LogicalSchema, overwrite: bool, ) -> Result { let table_schema = table_schema.clone().to_dfschema_ref()?; @@ -383,7 +384,7 @@ impl LogicalPlanBuilder { } /// Make a builder for a prepare logical plan from the builder's plan - pub fn prepare(self, name: String, data_types: Vec) -> Result { + pub fn prepare(self, name: String, data_types: Vec) -> Result { Ok(Self::from(LogicalPlan::Prepare(Prepare { name, data_types, @@ -1181,7 +1182,7 @@ impl From> for LogicalPlanBuilder { } } -pub fn change_redundant_column(fields: &Fields) -> Vec { +pub fn change_redundant_column(fields: &LogicalFields) -> Vec { let mut name_map = HashMap::new(); fields .into_iter() @@ -1190,7 +1191,7 @@ pub fn change_redundant_column(fields: &Fields) -> Vec { *counter += 1; if *counter > 1 { let new_name = format!("{}:{}", field.name(), *counter - 1); - Field::new(new_name, field.data_type().clone(), field.is_nullable()) + LogicalField::new(new_name, field.data_type().clone(), field.is_nullable()) } else { field.as_ref().clone() } @@ -1205,8 +1206,8 @@ pub fn build_join_schema( join_type: &JoinType, ) -> Result { fn nullify_fields<'a>( - fields: impl Iterator, &'a Arc)>, - ) -> Vec<(Option, Arc)> { + fields: impl Iterator, &'a Arc)>, + ) -> Vec<(Option, Arc)> { fields .map(|(q, f)| { // TODO: find a good way to do that @@ -1219,7 +1220,7 @@ pub fn build_join_schema( let right_fields = right.iter(); let left_fields = left.iter(); - let qualified_fields: Vec<(Option, Arc)> = match join_type { + let qualified_fields: Vec<(Option, Arc)> = match join_type { JoinType::Inner => { // left then right let left_fields = left_fields @@ -1377,9 +1378,10 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result) -> Result { // - Struct(field1, field2) returns ["a.field1","a.field2"] pub fn get_unnested_columns( col_name: &String, - data_type: &DataType, -) -> Result)>> { + data_type: &LogicalType, +) -> Result)>> { let mut qualified_columns = Vec::with_capacity(1); match data_type { - DataType::List(field) - | DataType::FixedSizeList(field, _) - | DataType::LargeList(field) => { - let new_field = Arc::new(Field::new( + LogicalType::List(field) + | LogicalType::FixedSizeList(field, _) + | LogicalType::LargeList(field) => { + let new_field = Arc::new(LogicalField::new( col_name.clone(), field.data_type().clone(), // Unnesting may produce NULLs even if the list is not null. @@ -1621,7 +1623,7 @@ pub fn get_unnested_columns( // let column = Column::from((None, &new_field)); qualified_columns.push((column, new_field)); } - DataType::Struct(fields) => { + LogicalType::Struct(fields) => { qualified_columns.extend(fields.iter().map(|f| { let new_name = format!("{}.{}", col_name, f.name()); let column = Column::from_name(&new_name); @@ -1670,10 +1672,10 @@ pub fn unnest_with_options( original_field.data_type(), )?; match original_field.data_type() { - DataType::List(_) - | DataType::FixedSizeList(_, _) - | DataType::LargeList(_) => list_columns.push(index), - DataType::Struct(_) => struct_columns.push(index), + LogicalType::List(_) + | LogicalType::FixedSizeList(_, _) + | LogicalType::LargeList(_) => list_columns.push(index), + LogicalType::Struct(_) => struct_columns.push(index), _ => { panic!( "not reachable, should be caught by get_unnested_columns" @@ -1685,7 +1687,7 @@ pub fn unnest_with_options( .extend(std::iter::repeat(index).take(flatten_columns.len())); Ok(flatten_columns .iter() - .map(|col: &(Column, Arc)| { + .map(|col: &(Column, Arc)| { (col.0.relation.to_owned(), col.1.to_owned()) }) .collect()) @@ -1720,6 +1722,7 @@ pub fn unnest_with_options( #[cfg(test)] mod tests { + use arrow::datatypes::{DataType, Field, Fields}; use super::*; use crate::logical_plan::StringifiedPlan; use crate::{col, expr, expr_fn::exists, in_subquery, lit, scalar_subquery}; @@ -1752,7 +1755,7 @@ mod tests { .unwrap(); let expected = DFSchema::try_from_qualified_schema( TableReference::bare("employee_csv"), - &schema, + &schema.clone().into(), ) .unwrap(); assert_eq!(&expected, plan.schema().as_ref()); @@ -2109,7 +2112,7 @@ mod tests { // Check unnested field is a scalar let field = plan.schema().field_with_name(None, "strings").unwrap(); - assert_eq!(&DataType::Utf8, field.data_type()); + assert_eq!(&LogicalType::Utf8, field.data_type()); // Unnesting the singular struct column result into 2 new columns for each subfield let plan = nested_table_scan("test_table")? @@ -2127,7 +2130,7 @@ mod tests { .schema() .field_with_name(None, &format!("struct_singular.{}", field_name)) .unwrap(); - assert_eq!(&DataType::UInt32, field.data_type()); + assert_eq!(&LogicalType::UInt32, field.data_type()); } // Unnesting multiple fields in separate plans @@ -2146,7 +2149,7 @@ mod tests { // Check unnested struct list field should be a struct. let field = plan.schema().field_with_name(None, "structs").unwrap(); - assert!(matches!(field.data_type(), DataType::Struct(_))); + assert!(matches!(field.data_type(), LogicalType::Struct(_))); // Unnesting multiple fields at the same time let cols = vec!["strings", "structs", "struct_singular"] @@ -2220,23 +2223,23 @@ mod tests { #[test] fn test_change_redundant_column() -> Result<()> { - let t1_field_1 = Field::new("a", DataType::Int32, false); - let t2_field_1 = Field::new("a", DataType::Int32, false); - let t2_field_3 = Field::new("a", DataType::Int32, false); - let t1_field_2 = Field::new("b", DataType::Int32, false); - let t2_field_2 = Field::new("b", DataType::Int32, false); + let t1_field_1 = LogicalField::new("a", LogicalType::Int32, false); + let t2_field_1 = LogicalField::new("a", LogicalType::Int32, false); + let t2_field_3 = LogicalField::new("a", LogicalType::Int32, false); + let t1_field_2 = LogicalField::new("b", LogicalType::Int32, false); + let t2_field_2 = LogicalField::new("b", LogicalType::Int32, false); let field_vec = vec![t1_field_1, t2_field_1, t1_field_2, t2_field_2, t2_field_3]; - let remove_redundant = change_redundant_column(&Fields::from(field_vec)); + let remove_redundant = change_redundant_column(&LogicalFields::from(field_vec)); assert_eq!( remove_redundant, vec![ - Field::new("a", DataType::Int32, false), - Field::new("a:1", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - Field::new("b:1", DataType::Int32, false), - Field::new("a:2", DataType::Int32, false), + LogicalField::new("a", LogicalType::Int32, false), + LogicalField::new("a:1", LogicalType::Int32, false), + LogicalField::new("b", LogicalType::Int32, false), + LogicalField::new("b:1", LogicalType::Int32, false), + LogicalField::new("a:2", LogicalType::Int32, false), ] ); Ok(()) diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 45ddbafecfd7..8b938a808b96 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -24,9 +24,9 @@ use std::{ use crate::{Expr, LogicalPlan, Volatility}; -use arrow::datatypes::DataType; use datafusion_common::{Constraints, DFSchemaRef, SchemaReference, TableReference}; use sqlparser::ast::Ident; +use datafusion_common::logical_type::LogicalType; /// Various types of DDL (CREATE / DROP) catalog manipulation #[derive(Clone, PartialEq, Eq, Hash)] @@ -322,7 +322,7 @@ pub struct CreateFunction { pub temporary: bool, pub name: String, pub args: Option>, - pub return_type: Option, + pub return_type: Option, pub params: CreateFunctionBody, /// Dummy schema pub schema: DFSchemaRef, @@ -332,7 +332,7 @@ pub struct OperateFunctionArg { // TODO: figure out how to support mode // pub mode: Option, pub name: Option, - pub data_type: DataType, + pub data_type: LogicalType, pub default_expr: Option, } #[derive(Clone, PartialEq, Eq, Hash, Debug)] diff --git a/datafusion/expr/src/logical_plan/dml.rs b/datafusion/expr/src/logical_plan/dml.rs index c9eef9bd34cc..04124984af3f 100644 --- a/datafusion/expr/src/logical_plan/dml.rs +++ b/datafusion/expr/src/logical_plan/dml.rs @@ -20,10 +20,11 @@ use std::fmt::{self, Display}; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{DFSchemaRef, TableReference}; - +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; use crate::LogicalPlan; /// Operator that copies the contents of a database to file(s) @@ -130,8 +131,6 @@ impl Display for WriteOp { fn make_count_schema() -> DFSchemaRef { Arc::new( - Schema::new(vec![Field::new("count", DataType::UInt64, false)]) - .try_into() - .unwrap(), + LogicalSchema::new(vec![LogicalField::new("count", LogicalType::UInt64, false)]).try_into().unwrap() ) } diff --git a/datafusion/expr/src/logical_plan/extension.rs b/datafusion/expr/src/logical_plan/extension.rs index 2f581c1928f4..182018cbaf65 100644 --- a/datafusion/expr/src/logical_plan/extension.rs +++ b/datafusion/expr/src/logical_plan/extension.rs @@ -349,5 +349,5 @@ impl UserDefinedLogicalNode for T { } fn get_all_columns_from_schema(schema: &DFSchema) -> HashSet { - schema.fields().iter().map(|f| f.name().clone()).collect() + schema.fields().iter().map(|f| f.name().to_string()).collect() } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index aef11853c81c..0da3b156967a 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -40,7 +40,7 @@ use crate::{ TableProviderFilterPushDown, TableSource, WindowFunctionDefinition, }; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -55,6 +55,9 @@ use crate::display::PgJsonVisitor; use crate::logical_plan::tree_node::unwrap_arc; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; /// A `LogicalPlan` is a node in a tree of relational operators (such as /// Projection or Filter). @@ -351,20 +354,20 @@ impl LogicalPlan { } /// Returns the (fixed) output schema for explain plans - pub fn explain_schema() -> SchemaRef { - SchemaRef::new(Schema::new(vec![ + pub fn explain_schema() -> LogicalSchemaRef { + LogicalSchemaRef::new(Schema::new(vec![ Field::new("plan_type", DataType::Utf8, false), Field::new("plan", DataType::Utf8, false), - ])) + ]).into()) } /// Returns the (fixed) output schema for `DESCRIBE` plans - pub fn describe_schema() -> Schema { + pub fn describe_schema() -> LogicalSchema { Schema::new(vec![ Field::new("column_name", DataType::Utf8, false), Field::new("data_type", DataType::Utf8, false), Field::new("is_nullable", DataType::Utf8, false), - ]) + ]).into() } /// Returns all expressions (non-recursively) evaluated by the current @@ -1388,8 +1391,8 @@ impl LogicalPlan { /// Walk the logical plan, find any `Placeholder` tokens, and return a map of their IDs and DataTypes pub fn get_parameter_types( &self, - ) -> Result>, DataFusionError> { - let mut param_types: HashMap> = HashMap::new(); + ) -> Result>, DataFusionError> { + let mut param_types: HashMap> = HashMap::new(); self.apply_with_subqueries(|plan| { plan.apply_expressions(|expr| { @@ -2085,7 +2088,7 @@ impl SubqueryAlias { // functional dependencies: let func_dependencies = plan.schema().functional_dependencies().clone(); let schema = DFSchemaRef::new( - DFSchema::try_from_qualified_schema(alias.clone(), &schema)? + DFSchema::try_from_qualified_schema(alias.clone(), &schema.into())? .with_functional_dependencies(func_dependencies)?, ); Ok(SubqueryAlias { @@ -2124,7 +2127,7 @@ impl Filter { // construction (such as with correlated subqueries) so we make a best effort here and // ignore errors resolving the expression against the schema. if let Ok(predicate_type) = predicate.get_type(input.schema()) { - if predicate_type != DataType::Boolean { + if predicate_type != LogicalType::Boolean { return plan_err!( "Cannot create filter with non-boolean predicate '{predicate}' returning {predicate_type}" ); @@ -2257,7 +2260,7 @@ pub struct Window { impl Window { /// Create a new window operator. pub fn try_new(window_expr: Vec, input: Arc) -> Result { - let fields: Vec<(Option, Arc)> = input + let fields: Vec<(Option, Arc)> = input .schema() .iter() .map(|(q, f)| (q.cloned(), f.clone())) @@ -2398,7 +2401,7 @@ impl TableScan { if table_name.table().is_empty() { return plan_err!("table_name cannot be empty"); } - let schema = table_source.schema(); + let schema: LogicalSchema = table_source.schema().as_ref().clone().into(); let func_dependencies = FunctionalDependencies::new_from_constraints( table_source.constraints(), schema.fields.len(), @@ -2412,7 +2415,7 @@ impl TableScan { let df_schema = DFSchema::new_with_metadata( p.iter() .map(|i| { - (Some(table_name.clone()), Arc::new(schema.field(*i).clone())) + (Some(table_name.clone()), schema.field(*i).clone()) }) .collect(), schema.metadata.clone(), @@ -2473,7 +2476,7 @@ pub struct Prepare { /// The name of the statement pub name: String, /// Data types of the parameters ([`Expr::Placeholder`]) - pub data_types: Vec, + pub data_types: Vec, /// The logical plan of the statements pub input: Arc, } @@ -3494,7 +3497,7 @@ digraph { let schema = Arc::new( DFSchema::try_from_qualified_schema( TableReference::bare("tab"), - &source.schema(), + &source.schema().as_ref().clone().into(), ) .unwrap(), ); diff --git a/datafusion/expr/src/simplify.rs b/datafusion/expr/src/simplify.rs index ccf45ff0d048..8c647f16bd75 100644 --- a/datafusion/expr/src/simplify.rs +++ b/datafusion/expr/src/simplify.rs @@ -17,9 +17,8 @@ //! Structs and traits to provide the information needed for expression simplification. -use arrow::datatypes::DataType; use datafusion_common::{DFSchemaRef, DataFusionError, Result}; - +use datafusion_common::logical_type::LogicalType; use crate::{execution_props::ExecutionProps, Expr, ExprSchemable}; /// Provides the information necessary to apply algebraic simplification to an @@ -39,7 +38,7 @@ pub trait SimplifyInfo { fn execution_props(&self) -> &ExecutionProps; /// Returns data type of this expr needed for determining optimized int type of a value - fn get_data_type(&self, expr: &Expr) -> Result; + fn get_data_type(&self, expr: &Expr) -> Result; } /// Provides simplification information based on DFSchema and @@ -75,7 +74,7 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { /// returns true if this Expr has boolean type fn is_boolean_type(&self, expr: &Expr) -> Result { for schema in &self.schema { - if let Ok(DataType::Boolean) = expr.get_type(schema) { + if let Ok(LogicalType::Boolean) = expr.get_type(schema) { return Ok(true); } } @@ -94,7 +93,7 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { } /// Returns data type of this expr needed for determining optimized int type of a value - fn get_data_type(&self, expr: &Expr) -> Result { + fn get_data_type(&self, expr: &Expr) -> Result { let schema = self.schema.as_ref().ok_or_else(|| { DataFusionError::Internal( "attempt to get data type without schema".to_string(), diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 5645a2a4dede..0f144c6ca89b 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +// TODO(@notfilippo): make most of these accept LogicalType + //! Coercion rules for matching argument types for binary operators use std::collections::HashSet; diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 5f060a4a4f16..e6f89efb9dfa 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -585,7 +585,7 @@ fn coerced_from<'a>( // List or LargeList with different dimensions should be handled in TypeSignature or other places before this (List(_) | LargeList(_), _) if datafusion_common::utils::base_type(type_from).eq(&Null) - || list_ndims(type_from) == list_ndims(type_into) => + || list_ndims(&type_from) == list_ndims(&type_into) => { Some(type_into.clone()) } diff --git a/datafusion/expr/src/type_coercion/mod.rs b/datafusion/expr/src/type_coercion/mod.rs index 86005da3dafa..275700adc9a1 100644 --- a/datafusion/expr/src/type_coercion/mod.rs +++ b/datafusion/expr/src/type_coercion/mod.rs @@ -36,52 +36,53 @@ pub mod binary; pub mod functions; pub mod other; -use arrow::datatypes::DataType; +use datafusion_common::logical_type::LogicalType; + /// Determine whether the given data type `dt` represents signed numeric values. -pub fn is_signed_numeric(dt: &DataType) -> bool { +pub fn is_signed_numeric(dt: &LogicalType) -> bool { matches!( dt, - DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::Float16 - | DataType::Float32 - | DataType::Float64 - | DataType::Decimal128(_, _) - | DataType::Decimal256(_, _), + LogicalType::Int8 + | LogicalType::Int16 + | LogicalType::Int32 + | LogicalType::Int64 + | LogicalType::Float16 + | LogicalType::Float32 + | LogicalType::Float64 + | LogicalType::Decimal128(_, _) + | LogicalType::Decimal256(_, _), ) } /// Determine whether the given data type `dt` is `Null`. -pub fn is_null(dt: &DataType) -> bool { - *dt == DataType::Null +pub fn is_null(dt: &LogicalType) -> bool { + *dt == LogicalType::Null } /// Determine whether the given data type `dt` is a `Timestamp`. -pub fn is_timestamp(dt: &DataType) -> bool { - matches!(dt, DataType::Timestamp(_, _)) +pub fn is_timestamp(dt: &LogicalType) -> bool { + matches!(dt, LogicalType::Timestamp(_, _)) } /// Determine whether the given data type 'dt' is a `Interval`. -pub fn is_interval(dt: &DataType) -> bool { - matches!(dt, DataType::Interval(_)) +pub fn is_interval(dt: &LogicalType) -> bool { + matches!(dt, LogicalType::Interval(_)) } /// Determine whether the given data type `dt` is a `Date` or `Timestamp`. -pub fn is_datetime(dt: &DataType) -> bool { +pub fn is_datetime(dt: &LogicalType) -> bool { matches!( dt, - DataType::Date32 | DataType::Date64 | DataType::Timestamp(_, _) + LogicalType::Date32 | LogicalType::Date64 | LogicalType::Timestamp(_, _) ) } /// Determine whether the given data type `dt` is a `Utf8` or `LargeUtf8`. -pub fn is_utf8_or_large_utf8(dt: &DataType) -> bool { - matches!(dt, DataType::Utf8 | DataType::LargeUtf8) +pub fn is_utf8_or_large_utf8(dt: &LogicalType) -> bool { + matches!(dt, LogicalType::Utf8 | LogicalType::LargeUtf8) } /// Determine whether the given data type `dt` is a `Decimal`. -pub fn is_decimal(dt: &DataType) -> bool { - matches!(dt, DataType::Decimal128(_, _) | DataType::Decimal256(_, _)) +pub fn is_decimal(dt: &LogicalType) -> bool { + matches!(dt, LogicalType::Decimal128(_, _) | LogicalType::Decimal256(_, _)) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 286f05309ea7..3a55297d39ba 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -28,7 +28,7 @@ use crate::{ and, BinaryExpr, Expr, ExprSchemable, Filter, GroupingSet, LogicalPlan, Operator, }; -use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -39,6 +39,9 @@ use datafusion_common::{ }; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; /// The value to which `COUNT(*)` is expanded to in /// `COUNT()` expressions @@ -429,7 +432,7 @@ pub fn expand_qualified_wildcard( return plan_err!("Invalid qualifier {qualifier}"); } - let qualified_schema = Arc::new(Schema::new(fields_with_qualified)); + let qualified_schema = Arc::new(LogicalSchema::new(fields_with_qualified)); let qualified_dfschema = DFSchema::try_from_qualified_schema(qualifier.clone(), &qualified_schema)? .with_functional_dependencies(projected_func_dependencies)?; @@ -727,7 +730,7 @@ pub fn from_plan( pub fn exprlist_to_fields<'a>( exprs: impl IntoIterator, plan: &LogicalPlan, -) -> Result, Arc)>> { +) -> Result, Arc)>> { // look for exact match in plan's output schema let input_schema = &plan.schema(); exprs @@ -830,40 +833,35 @@ pub(crate) fn find_column_indexes_referenced_by_expr( /// can this data type be used in hash join equal conditions?? /// data types here come from function 'equal_rows', if more data types are supported /// in equal_rows(hash join), add those data types here to generate join logical plan. -pub fn can_hash(data_type: &DataType) -> bool { +pub fn can_hash(data_type: &LogicalType) -> bool { match data_type { - DataType::Null => true, - DataType::Boolean => true, - DataType::Int8 => true, - DataType::Int16 => true, - DataType::Int32 => true, - DataType::Int64 => true, - DataType::UInt8 => true, - DataType::UInt16 => true, - DataType::UInt32 => true, - DataType::UInt64 => true, - DataType::Float32 => true, - DataType::Float64 => true, - DataType::Timestamp(time_unit, _) => match time_unit { + LogicalType::Null => true, + LogicalType::Boolean => true, + LogicalType::Int8 => true, + LogicalType::Int16 => true, + LogicalType::Int32 => true, + LogicalType::Int64 => true, + LogicalType::UInt8 => true, + LogicalType::UInt16 => true, + LogicalType::UInt32 => true, + LogicalType::UInt64 => true, + LogicalType::Float32 => true, + LogicalType::Float64 => true, + LogicalType::Timestamp(time_unit, _) => match time_unit { TimeUnit::Second => true, TimeUnit::Millisecond => true, TimeUnit::Microsecond => true, TimeUnit::Nanosecond => true, }, - DataType::Utf8 => true, - DataType::LargeUtf8 => true, - DataType::Decimal128(_, _) => true, - DataType::Date32 => true, - DataType::Date64 => true, - DataType::FixedSizeBinary(_) => true, - DataType::Dictionary(key_type, value_type) - if *value_type.as_ref() == DataType::Utf8 => - { - DataType::is_dictionary_key_type(key_type) - } - DataType::List(_) => true, - DataType::LargeList(_) => true, - DataType::FixedSizeList(_, _) => true, + LogicalType::Utf8 => true, + LogicalType::LargeUtf8 => true, + LogicalType::Decimal128(_, _) => true, + LogicalType::Date32 => true, + LogicalType::Date64 => true, + LogicalType::FixedSizeBinary(_) => true, + LogicalType::List(_) => true, + LogicalType::LargeList(_) => true, + LogicalType::FixedSizeList(_, _) => true, _ => false, } } @@ -1249,6 +1247,7 @@ impl AggregateOrderSensitivity { #[cfg(test)] mod tests { + use datafusion_common::logical_type::LogicalType; use super::*; use crate::{ col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, @@ -1703,11 +1702,11 @@ mod tests { fn test_collect_expr() -> Result<()> { let mut accum: HashSet = HashSet::new(); expr_to_columns( - &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), + &Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Float64)), &mut accum, )?; expr_to_columns( - &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), + &Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Float64)), &mut accum, )?; assert_eq!(1, accum.len()); diff --git a/datafusion/expr/src/var_provider.rs b/datafusion/expr/src/var_provider.rs index e00cf7407237..b746955630b3 100644 --- a/datafusion/expr/src/var_provider.rs +++ b/datafusion/expr/src/var_provider.rs @@ -17,8 +17,8 @@ //! Variable provider -use arrow::datatypes::DataType; use datafusion_common::{Result, ScalarValue}; +use datafusion_common::logical_type::LogicalType; /// Variable type, system/user defined #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -35,7 +35,7 @@ pub trait VarProvider: std::fmt::Debug { fn get_value(&self, var_names: Vec) -> Result; /// Return the type of the given variable - fn get_type(&self, var_names: &[String]) -> Option; + fn get_type(&self, var_names: &[String]) -> Option; } pub fn is_system_variables(variable_names: &[String]) -> bool { diff --git a/datafusion/functions/src/core/arrow_cast.rs b/datafusion/functions/src/core/arrow_cast.rs index 9c410d4e18e8..d3200a0a10d0 100644 --- a/datafusion/functions/src/core/arrow_cast.rs +++ b/datafusion/functions/src/core/arrow_cast.rs @@ -107,7 +107,7 @@ impl ScalarUDFImpl for ArrowCastFunc { info: &dyn SimplifyInfo, ) -> Result { // convert this into a real cast - let target_type = data_type_from_args(&args)?; + let target_type = data_type_from_args(&args)?.into(); // remove second (type) argument args.pop().unwrap(); let arg = args.pop().unwrap(); @@ -130,6 +130,8 @@ impl ScalarUDFImpl for ArrowCastFunc { /// Returns the requested type from the arguments fn data_type_from_args(args: &[Expr]) -> Result { + // TODO(@notfilippo): maybe parse LogicalType? + if args.len() != 2 { return plan_err!("arrow_cast needs 2 arguments, {} provided", args.len()); } diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index b76da15c52ca..8aaae7b740d2 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -26,6 +26,8 @@ use datafusion_common::{ use datafusion_expr::{ColumnarValue, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; #[derive(Debug)] pub struct GetFieldFunc { @@ -105,35 +107,36 @@ impl ScalarUDFImpl for GetFieldFunc { ); } }; + // TODO(@notfilippo): avoid converting to physical type let data_type = args[0].get_type(schema)?; match (data_type, name) { - (DataType::Map(fields, _), _) => { + (LogicalType::Map(fields, _), _) => { match fields.data_type() { - DataType::Struct(fields) if fields.len() == 2 => { + LogicalType::Struct(fields) if fields.len() == 2 => { // Arrow's MapArray is essentially a ListArray of structs with two columns. They are // often named "key", and "value", but we don't require any specific naming here; // instead, we assume that the second columnis the "value" column both here and in // execution. let value_field = fields.get(1).expect("fields should have exactly two members"); - Ok(value_field.data_type().clone()) + Ok(value_field.data_type().physical_type()) }, _ => plan_err!("Map fields must contain a Struct with exactly 2 fields"), } } - (DataType::Struct(fields), ScalarValue::Utf8(Some(s))) => { + (LogicalType::Struct(fields), ScalarValue::Utf8(Some(s))) => { if s.is_empty() { plan_err!( "Struct based indexed access requires a non empty string" ) } else { let field = fields.iter().find(|f| f.name() == s); - field.ok_or(plan_datafusion_err!("Field {s} not found in struct")).map(|f| f.data_type().clone()) + field.ok_or(plan_datafusion_err!("Field {s} not found in struct")).map(|f| f.data_type().clone().physical_type()) } } - (DataType::Struct(_), _) => plan_err!( + (LogicalType::Struct(_), _) => plan_err!( "Only UTF8 strings are valid as an indexed field in a struct" ), - (DataType::Null, _) => Ok(DataType::Null), + (LogicalType::Null, _) => Ok(DataType::Null), (other, _) => plan_err!("The expression to get an indexed field is only valid for `List`, `Struct`, `Map` or `Null` types, got {other}"), } } diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index 8ccda977f3a4..eeadd18c6215 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -22,6 +22,7 @@ use datafusion_expr::{ColumnarValue, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; use std::sync::Arc; +use datafusion_common::logical_type::extension::ExtensionType; /// put values in a struct array. fn named_struct_expr(args: &[ColumnarValue]) -> Result { @@ -139,7 +140,7 @@ impl ScalarUDFImpl for NamedStructFunc { let value = &chunk[1]; if let Expr::Literal(ScalarValue::Utf8(Some(name))) = name { - Ok(Field::new(name, value.get_type(schema)?, true)) + Ok(Field::new(name, value.get_type(schema)?.physical_type(), true)) } else { exec_err!("named_struct even arguments must be string literals, got {name} instead at position {}", i * 2) } diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index 0791561539e1..0d4d66eb5e42 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -28,6 +28,7 @@ use datafusion_common::{ exec_err, internal_err, plan_datafusion_err, plan_err, DataFusionError, Result, ScalarValue, }; +use datafusion_common::logical_type::extension::ExtensionType; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; @@ -158,6 +159,7 @@ impl ScalarUDFImpl for LogFunc { Ok(ColumnarValue::Array(arr)) } + // TODO(@notfilippo): avoid converting to physical type /// Simplify the `log` function by the relevant rules: /// 1. Log(a, 1) ===> 0 /// 2. Log(a, Power(a, b)) ===> b @@ -182,13 +184,13 @@ impl ScalarUDFImpl for LogFunc { let base = if let Some(base) = args.pop() { base } else { - lit(ScalarValue::new_ten(&number_datatype)?) + lit(ScalarValue::new_ten(&number_datatype.physical_type())?) }; match number { - Expr::Literal(value) if value == ScalarValue::new_one(&number_datatype)? => { + Expr::Literal(value) if value == ScalarValue::new_one(&number_datatype.physical_type())? => { Ok(ExprSimplifyResult::Simplified(lit(ScalarValue::new_zero( - &info.get_data_type(&base)?, + &info.get_data_type(&base)?.physical_type(), )?))) } Expr::ScalarFunction(ScalarFunction { func, mut args }) @@ -200,7 +202,7 @@ impl ScalarUDFImpl for LogFunc { number => { if number == base { Ok(ExprSimplifyResult::Simplified(lit(ScalarValue::new_one( - &number_datatype, + &number_datatype.physical_type(), )?))) } else { let args = match num_args { diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index 5b790fb56ddf..77a131997c41 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -32,7 +32,7 @@ use datafusion_expr::TypeSignature::*; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; use std::sync::Arc; - +use datafusion_common::logical_type::extension::ExtensionType; use super::log::LogFunc; #[derive(Debug)] @@ -127,6 +127,7 @@ impl ScalarUDFImpl for PowerFunc { Ok(ColumnarValue::Array(arr)) } + // TODO(@notfilippo): avoid converting to physical type /// Simplify the `power` function by the relevant rules: /// 1. Power(a, 0) ===> 0 /// 2. Power(a, 1) ===> a @@ -143,11 +144,11 @@ impl ScalarUDFImpl for PowerFunc { plan_datafusion_err!("Expected power to have 2 arguments, got 1") })?; - let exponent_type = info.get_data_type(&exponent)?; + let exponent_type = info.get_data_type(&exponent)?.physical_type(); match exponent { Expr::Literal(value) if value == ScalarValue::new_zero(&exponent_type)? => { Ok(ExprSimplifyResult::Simplified(Expr::Literal( - ScalarValue::new_one(&info.get_data_type(&base)?)?, + ScalarValue::new_one(&info.get_data_type(&base)?.physical_type())?, ))) } Expr::Literal(value) if value == ScalarValue::new_one(&exponent_type)? => { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 34f9802b1fd9..1074c04aa395 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -98,7 +98,6 @@ fn analyze_internal(plan: LogicalPlan) -> Result> { mod tests { use super::*; use crate::test::*; - use arrow::datatypes::DataType; use datafusion_common::ScalarValue; use datafusion_expr::expr::Sort; use datafusion_expr::{ @@ -108,7 +107,7 @@ mod tests { }; use datafusion_functions_aggregate::count::count_udaf; use std::sync::Arc; - + use datafusion_common::logical_type::LogicalType; use datafusion_functions_aggregate::expr_fn::{count, sum}; fn assert_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { @@ -192,7 +191,7 @@ mod tests { .filter( scalar_subquery(Arc::new( LogicalPlanBuilder::from(table_scan_t2) - .filter(out_ref_col(DataType::UInt32, "t1.a").eq(col("t2.a")))? + .filter(out_ref_col(LogicalType::UInt32, "t1.a").eq(col("t2.a")))? .aggregate( Vec::::new(), vec![count(lit(COUNT_STAR_EXPANSION))], diff --git a/datafusion/optimizer/src/analyzer/function_rewrite.rs b/datafusion/optimizer/src/analyzer/function_rewrite.rs index 098c934bf7e1..86d106bb8975 100644 --- a/datafusion/optimizer/src/analyzer/function_rewrite.rs +++ b/datafusion/optimizer/src/analyzer/function_rewrite.rs @@ -53,7 +53,7 @@ impl ApplyFunctionRewrites { if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( ts.table_name.clone(), - &ts.source.schema(), + &ts.source.schema().as_ref().clone().into(), )?; schema.merge(&source_schema); } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 64d9c508f3f6..fb479257ee3c 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -27,6 +27,8 @@ use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::{ self, AggregateFunctionDefinition, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, ScalarFunction, WindowFunction, @@ -93,7 +95,7 @@ fn analyze_internal( if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( ts.table_name.clone(), - &ts.source.schema(), + &ts.source.schema().as_ref().clone().into(), )?; schema.merge(&source_schema); } @@ -161,13 +163,13 @@ impl<'a> TypeCoercionRewriter<'a> { right: Expr, ) -> Result<(Expr, Expr)> { let (left_type, right_type) = get_input_types( - &left.get_type(self.schema)?, + &left.get_type(self.schema)?.physical_type(), op, - &right.get_type(self.schema)?, + &right.get_type(self.schema)?.physical_type(), )?; Ok(( - left.cast_to(&left_type, self.schema)?, - right.cast_to(&right_type, self.schema)?, + left.cast_to(&left_type.into(), self.schema)?, + right.cast_to(&right_type.into(), self.schema)?, )) } } @@ -210,7 +212,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { analyze_internal(self.schema, unwrap_arc(subquery.subquery))?.data; let expr_type = expr.get_type(self.schema)?; let subquery_type = new_plan.schema().field(0).data_type(); - let common_type = comparison_coercion(&expr_type, subquery_type).ok_or(plan_datafusion_err!( + let common_type = comparison_coercion(&expr_type.physical_type(), &subquery_type.physical_type()).ok_or(plan_datafusion_err!( "expr type {expr_type:?} can't cast to {subquery_type:?} in InSubquery" ), )?; @@ -219,8 +221,8 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { outer_ref_columns: subquery.outer_ref_columns, }; Ok(Transformed::yes(Expr::InSubquery(InSubquery::new( - Box::new(expr.cast_to(&common_type, self.schema)?), - cast_subquery(new_subquery, &common_type)?, + Box::new(expr.cast_to(&common_type.clone().into(), self.schema)?), + cast_subquery(new_subquery, &common_type.into())?, negated, )))) } @@ -255,7 +257,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { }) => { let left_type = expr.get_type(self.schema)?; let right_type = pattern.get_type(self.schema)?; - let coerced_type = like_coercion(&left_type, &right_type).ok_or_else(|| { + let coerced_type = like_coercion(&left_type.physical_type(), &right_type.physical_type()).ok_or_else(|| { let op_name = if case_insensitive { "ILIKE" } else { @@ -266,10 +268,9 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { ) })?; let expr = match left_type { - DataType::Dictionary(_, inner) if *inner == DataType::Utf8 => expr, - _ => Box::new(expr.cast_to(&coerced_type, self.schema)?), + _ => Box::new(expr.cast_to(&coerced_type.clone().into(), self.schema)?), }; - let pattern = Box::new(pattern.cast_to(&coerced_type, self.schema)?); + let pattern = Box::new(pattern.cast_to(&coerced_type.into(), self.schema)?); Ok(Transformed::yes(Expr::Like(Like::new( negated, expr, @@ -294,14 +295,14 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { }) => { let expr_type = expr.get_type(self.schema)?; let low_type = low.get_type(self.schema)?; - let low_coerced_type = comparison_coercion(&expr_type, &low_type) + let low_coerced_type = comparison_coercion(&expr_type.physical_type(), &low_type.physical_type()) .ok_or_else(|| { DataFusionError::Internal(format!( "Failed to coerce types {expr_type} and {low_type} in BETWEEN expression" )) })?; let high_type = high.get_type(self.schema)?; - let high_coerced_type = comparison_coercion(&expr_type, &low_type) + let high_coerced_type = comparison_coercion(&expr_type.physical_type(), &low_type.physical_type()) .ok_or_else(|| { DataFusionError::Internal(format!( "Failed to coerce types {expr_type} and {high_type} in BETWEEN expression" @@ -313,7 +314,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { DataFusionError::Internal(format!( "Failed to coerce types {expr_type} and {high_type} in BETWEEN expression" )) - })?; + })?.into(); Ok(Transformed::yes(Expr::Between(Between::new( Box::new(expr.cast_to(&coercion_type, self.schema)?), negated, @@ -326,24 +327,26 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { list, negated, }) => { + println!("{:?}", self.schema); let expr_data_type = expr.get_type(self.schema)?; let list_data_types = list .iter() - .map(|list_expr| list_expr.get_type(self.schema)) + .map(|list_expr| list_expr.get_type(self.schema).map(|t| t.physical_type())) .collect::>>()?; let result_type = - get_coerce_type_for_list(&expr_data_type, &list_data_types); + get_coerce_type_for_list(&expr_data_type.physical_type(), &list_data_types); match result_type { None => plan_err!( "Can not find compatible types to compare {expr_data_type:?} with {list_data_types:?}" ), Some(coerced_type) => { // find the coerced type - let cast_expr = expr.cast_to(&coerced_type, self.schema)?; + let logical_coerced_type = coerced_type.into(); + let cast_expr = expr.cast_to(&logical_coerced_type, self.schema)?; let cast_list_expr = list .into_iter() .map(|list_expr| { - list_expr.cast_to(&coerced_type, self.schema) + list_expr.cast_to(&logical_coerced_type, self.schema) }) .collect::>>()?; Ok(Transformed::yes(Expr::InList(InList ::new( @@ -473,11 +476,11 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { /// Casts the given `value` to `target_type`. Note that this function /// only considers `Null` or `Utf8` values. -fn coerce_scalar(target_type: &DataType, value: &ScalarValue) -> Result { +fn coerce_scalar(target_type: &LogicalType, value: &ScalarValue) -> Result { match value { // Coerce Utf8 values: ScalarValue::Utf8(Some(val)) => { - ScalarValue::try_from_string(val.clone(), target_type) + ScalarValue::try_from_string(val.clone(), &target_type.physical_type()) } s => { if s.is_null() { @@ -500,7 +503,7 @@ fn coerce_scalar(target_type: &DataType, value: &ScalarValue) -> Result Result { coerce_scalar(target_type, &value).or_else(|err| { @@ -519,18 +522,18 @@ fn coerce_scalar_range_aware( /// This function returns the widest type in the family of `given_type`. /// If the given type is already the widest type, it returns `None`. /// For example, if `given_type` is `Int8`, it returns `Int64`. -fn get_widest_type_in_family(given_type: &DataType) -> Option<&DataType> { +fn get_widest_type_in_family(given_type: &LogicalType) -> Option<&LogicalType> { match given_type { - DataType::UInt8 | DataType::UInt16 | DataType::UInt32 => Some(&DataType::UInt64), - DataType::Int8 | DataType::Int16 | DataType::Int32 => Some(&DataType::Int64), - DataType::Float16 | DataType::Float32 => Some(&DataType::Float64), + LogicalType::UInt8 | LogicalType::UInt16 | LogicalType::UInt32 => Some(&LogicalType::UInt64), + LogicalType::Int8 | LogicalType::Int16 | LogicalType::Int32 => Some(&LogicalType::Int64), + LogicalType::Float16 | LogicalType::Float32 => Some(&LogicalType::Float64), _ => None, } } /// Coerces the given (window frame) `bound` to `target_type`. fn coerce_frame_bound( - target_type: &DataType, + target_type: &LogicalType, bound: WindowFrameBound, ) -> Result { match bound { @@ -561,11 +564,11 @@ fn coerce_window_frame( if let Some(col_type) = current_types.first() { if col_type.is_numeric() || is_utf8_or_large_utf8(col_type) - || matches!(col_type, DataType::Null) + || matches!(col_type, LogicalType::Null) { col_type } else if is_datetime(col_type) { - &DataType::Interval(IntervalUnit::MonthDayNano) + &LogicalType::Interval(IntervalUnit::MonthDayNano) } else { return internal_err!( "Cannot run range queries on datatype: {col_type:?}" @@ -575,7 +578,7 @@ fn coerce_window_frame( return internal_err!("ORDER BY column cannot be empty"); } } - WindowFrameUnits::Rows | WindowFrameUnits::Groups => &DataType::UInt64, + WindowFrameUnits::Rows | WindowFrameUnits::Groups => &LogicalType::UInt64, }; window_frame.start_bound = coerce_frame_bound(target_type, window_frame.start_bound)?; window_frame.end_bound = coerce_frame_bound(target_type, window_frame.end_bound)?; @@ -586,8 +589,8 @@ fn coerce_window_frame( // The above op will be rewrite to the binary op when creating the physical op. fn get_casted_expr_for_bool_op(expr: Expr, schema: &DFSchema) -> Result { let left_type = expr.get_type(schema)?; - get_input_types(&left_type, &Operator::IsDistinctFrom, &DataType::Boolean)?; - expr.cast_to(&DataType::Boolean, schema) + get_input_types(&left_type.physical_type(), &Operator::IsDistinctFrom, &DataType::Boolean)?; + expr.cast_to(&LogicalType::Boolean, schema) } /// Returns `expressions` coerced to types compatible with @@ -605,15 +608,15 @@ fn coerce_arguments_for_signature_with_scalar_udf( let current_types = expressions .iter() - .map(|e| e.get_type(schema)) + .map(|e| e.get_type(schema).map(|t| t.physical_type())) .collect::>>()?; let new_types = data_types_with_scalar_udf(¤t_types, func)?; expressions .into_iter() - .enumerate() - .map(|(i, expr)| expr.cast_to(&new_types[i], schema)) + .zip(new_types) + .map(|(expr, t)| expr.cast_to(&t.into(), schema)) .collect() } @@ -632,15 +635,15 @@ fn coerce_arguments_for_signature_with_aggregate_udf( let current_types = expressions .iter() - .map(|e| e.get_type(schema)) + .map(|e| e.get_type(schema).map(|t| t.physical_type())) .collect::>>()?; let new_types = data_types_with_aggregate_udf(¤t_types, func)?; expressions .into_iter() - .enumerate() - .map(|(i, expr)| expr.cast_to(&new_types[i], schema)) + .zip(new_types) + .map(|(expr, t)| expr.cast_to(&t.into(), schema)) .collect() } @@ -655,8 +658,8 @@ fn coerce_arguments_for_fun( .into_iter() .map(|expr| { let data_type = expr.get_type(schema).unwrap(); - if let DataType::FixedSizeList(field, _) = data_type { - let to_type = DataType::List(field.clone()); + if let LogicalType::FixedSizeList(field, _) = data_type { + let to_type = LogicalType::List(field.clone()); expr.cast_to(&to_type, schema) } else { Ok(expr) @@ -682,7 +685,7 @@ fn coerce_agg_exprs_for_signature( } let current_types = input_exprs .iter() - .map(|e| e.get_type(schema)) + .map(|e| e.get_type(schema).map(|t| t.physical_type())) .collect::>>()?; let coerced_types = @@ -690,8 +693,8 @@ fn coerce_agg_exprs_for_signature( input_exprs .into_iter() - .enumerate() - .map(|(i, expr)| expr.cast_to(&coerced_types[i], schema)) + .zip(coerced_types) + .map(|(expr, t)| expr.cast_to(&t.into(), schema)) .collect() } @@ -735,12 +738,12 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { let then_types = case .when_then_expr .iter() - .map(|(_when, then)| then.get_type(schema)) + .map(|(_when, then)| then.get_type(schema).map(|t| t.physical_type())) .collect::>>()?; let else_type = case .else_expr .as_ref() - .map(|expr| expr.get_type(schema)) + .map(|expr| expr.get_type(schema).map(|t| t.physical_type())) .transpose()?; // find common coercible types @@ -750,10 +753,10 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { let when_types = case .when_then_expr .iter() - .map(|(when, _then)| when.get_type(schema)) + .map(|(when, _then)| when.get_type(schema).map(|t| t.physical_type())) .collect::>>()?; let coerced_type = - get_coerce_type_for_case_expression(&when_types, Some(case_type)); + get_coerce_type_for_case_expression(&when_types, Some(&case_type.physical_type())); coerced_type.ok_or_else(|| { plan_datafusion_err!( "Failed to coerce case ({case_type:?}) and when ({when_types:?}) \ @@ -776,7 +779,7 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { let case_expr = case .expr .zip(case_when_coerce_type.as_ref()) - .map(|(case_expr, coercible_type)| case_expr.cast_to(coercible_type, schema)) + .map(|(case_expr, coercible_type)| case_expr.cast_to(&coercible_type.into(), schema)) .transpose()? .map(Box::new); let when_then = case @@ -784,7 +787,7 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { .into_iter() .map(|(when, then)| { let when_type = case_when_coerce_type.as_ref().unwrap_or(&DataType::Boolean); - let when = when.cast_to(when_type, schema).map_err(|e| { + let when = when.cast_to(&when_type.into(), schema).map_err(|e| { DataFusionError::Context( format!( "WHEN expressions in CASE couldn't be \ @@ -793,13 +796,13 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { Box::new(e), ) })?; - let then = then.cast_to(&then_else_coerce_type, schema)?; + let then = then.cast_to(&then_else_coerce_type.clone().into(), schema)?; Ok((Box::new(when), Box::new(then))) }) .collect::>>()?; let else_expr = case .else_expr - .map(|expr| expr.cast_to(&then_else_coerce_type, schema)) + .map(|expr| expr.cast_to(&then_else_coerce_type.into(), schema)) .transpose()? .map(Box::new); @@ -816,6 +819,8 @@ mod test { use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{DFSchema, DFSchemaRef, Result, ScalarValue}; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::{self, InSubquery, Like, ScalarFunction}; use datafusion_expr::logical_plan::{EmptyRelation, Projection}; use datafusion_expr::test::function_stub::avg_udaf; @@ -839,12 +844,12 @@ mod test { })) } - fn empty_with_type(data_type: DataType) -> Arc { + fn empty_with_type(data_type: LogicalType) -> Arc { Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::new( DFSchema::from_unqualified_fields( - vec![Field::new("a", data_type, true)].into(), + vec![LogicalField::new("a", data_type, true)].into(), std::collections::HashMap::new(), ) .unwrap(), @@ -855,7 +860,7 @@ mod test { #[test] fn simple_case() -> Result<()> { let expr = col("a").lt(lit(2_u32)); - let empty = empty_with_type(DataType::Float64); + let empty = empty_with_type(LogicalType::Float64); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a < CAST(UInt32(2) AS Float64)\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected) @@ -864,7 +869,7 @@ mod test { #[test] fn nested_case() -> Result<()> { let expr = col("a").lt(lit(2_u32)); - let empty = empty_with_type(DataType::Float64); + let empty = empty_with_type(LogicalType::Float64); let plan = LogicalPlan::Projection(Projection::try_new( vec![expr.clone().or(expr)], @@ -894,7 +899,7 @@ mod test { } fn return_type(&self, _args: &[DataType]) -> Result { - Ok(DataType::Utf8) + Ok(Utf8) } fn invoke(&self, _args: &[ColumnarValue]) -> Result { @@ -1019,10 +1024,10 @@ mod test { let expected = "Projection: avg(Float64(12))\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(DataType::Int32); + let empty = empty_with_type(LogicalType::Int32); let agg_expr = Expr::AggregateFunction(expr::AggregateFunction::new_udf( avg_udaf(), - vec![cast(col("a"), DataType::Float64)], + vec![cast(col("a"), LogicalType::Float64)], false, None, None, @@ -1056,7 +1061,7 @@ mod test { #[test] fn binary_op_date32_op_interval() -> Result<()> { // CAST(Utf8("1998-03-18") AS Date32) + IntervalDayTime("...") - let expr = cast(lit("1998-03-18"), DataType::Date32) + let expr = cast(lit("1998-03-18"), LogicalType::Date32) + lit(ScalarValue::new_interval_dt(123, 456)); let empty = empty(); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); @@ -1070,7 +1075,7 @@ mod test { fn inlist_case() -> Result<()> { // a in (1,4,8), a is int64 let expr = col("a").in_list(vec![lit(1_i32), lit(4_i8), lit(8_i64)], false); - let empty = empty_with_type(DataType::Int64); + let empty = empty_with_type(LogicalType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\ @@ -1082,7 +1087,7 @@ mod test { let empty = Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::new(DFSchema::from_unqualified_fields( - vec![Field::new("a", DataType::Decimal128(12, 4), true)].into(), + vec![LogicalField::new("a", LogicalType::Decimal128(12, 4), true)].into(), std::collections::HashMap::new(), )?), })); @@ -1098,10 +1103,10 @@ mod test { let expr = col("a").between( lit("2002-05-08"), // (cast('2002-05-08' as date) + interval '1 months') - cast(lit("2002-05-08"), DataType::Date32) + cast(lit("2002-05-08"), LogicalType::Date32) + lit(ScalarValue::new_interval_ym(0, 1)), ); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Filter(Filter::try_new(expr, empty)?); let expected = "Filter: a BETWEEN Utf8(\"2002-05-08\") AND CAST(CAST(Utf8(\"2002-05-08\") AS Date32) + IntervalYearMonth(\"1\") AS Utf8)\ @@ -1113,11 +1118,11 @@ mod test { fn between_infer_cheap_type() -> Result<()> { let expr = col("a").between( // (cast('2002-05-08' as date) + interval '1 months') - cast(lit("2002-05-08"), DataType::Date32) + cast(lit("2002-05-08"), LogicalType::Date32) + lit(ScalarValue::new_interval_ym(0, 1)), lit("2002-12-08"), ); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Filter(Filter::try_new(expr, empty)?); // TODO: we should cast col(a). let expected = @@ -1130,13 +1135,13 @@ mod test { fn is_bool_for_type_coercion() -> Result<()> { // is true let expr = col("a").is_true(); - let empty = empty_with_type(DataType::Boolean); + let empty = empty_with_type(LogicalType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr.clone()], empty)?); let expected = "Projection: a IS TRUE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(DataType::Int64); + let empty = empty_with_type(LogicalType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let ret = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, ""); let err = ret.unwrap_err().to_string(); @@ -1144,21 +1149,21 @@ mod test { // is not true let expr = col("a").is_not_true(); - let empty = empty_with_type(DataType::Boolean); + let empty = empty_with_type(LogicalType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS NOT TRUE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; // is false let expr = col("a").is_false(); - let empty = empty_with_type(DataType::Boolean); + let empty = empty_with_type(LogicalType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS FALSE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; // is not false let expr = col("a").is_not_false(); - let empty = empty_with_type(DataType::Boolean); + let empty = empty_with_type(LogicalType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS NOT FALSE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1172,7 +1177,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let expected = "Projection: a LIKE Utf8(\"abc\")\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1180,7 +1185,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::Null)); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let expected = "Projection: a LIKE CAST(NULL AS Utf8) AS a LIKE NULL\ \n EmptyRelation"; @@ -1189,10 +1194,11 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(DataType::Int64); + let empty = empty_with_type(LogicalType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); assert!(err.is_err()); + println!("{:?}", err); assert!(err.unwrap_err().to_string().contains( "There isn't a common type to coerce Int64 and Utf8 in LIKE expression" )); @@ -1201,7 +1207,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let expected = "Projection: a ILIKE Utf8(\"abc\")\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1209,7 +1215,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::Null)); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let expected = "Projection: a ILIKE CAST(NULL AS Utf8) AS a ILIKE NULL\ \n EmptyRelation"; @@ -1218,7 +1224,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(DataType::Int64); + let empty = empty_with_type(LogicalType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); assert!(err.is_err()); @@ -1232,13 +1238,13 @@ mod test { fn unknown_for_type_coercion() -> Result<()> { // unknown let expr = col("a").is_unknown(); - let empty = empty_with_type(DataType::Boolean); + let empty = empty_with_type(LogicalType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr.clone()], empty)?); let expected = "Projection: a IS UNKNOWN\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let ret = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); let err = ret.unwrap_err().to_string(); @@ -1246,7 +1252,7 @@ mod test { // is not unknown let expr = col("a").is_not_unknown(); - let empty = empty_with_type(DataType::Boolean); + let empty = empty_with_type(LogicalType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS NOT UNKNOWN\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1256,7 +1262,7 @@ mod test { #[test] fn concat_for_type_coercion() -> Result<()> { - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(LogicalType::Utf8); let args = [col("a"), lit("b"), lit(true), lit(false), lit(13)]; // concat-type signature @@ -1279,34 +1285,34 @@ mod test { fn test_type_coercion_rewrite() -> Result<()> { // gt let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![Field::new("a", DataType::Int64, true)].into(), + vec![LogicalField::new("a", LogicalType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; let expr = is_true(lit(12i32).gt(lit(13i64))); - let expected = is_true(cast(lit(12i32), DataType::Int64).gt(lit(13i64))); + let expected = is_true(cast(lit(12i32), LogicalType::Int64).gt(lit(13i64))); let result = expr.rewrite(&mut rewriter).data()?; assert_eq!(expected, result); // eq let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![Field::new("a", DataType::Int64, true)].into(), + vec![LogicalField::new("a", LogicalType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; let expr = is_true(lit(12i32).eq(lit(13i64))); - let expected = is_true(cast(lit(12i32), DataType::Int64).eq(lit(13i64))); + let expected = is_true(cast(lit(12i32), LogicalType::Int64).eq(lit(13i64))); let result = expr.rewrite(&mut rewriter).data()?; assert_eq!(expected, result); // lt let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![Field::new("a", DataType::Int64, true)].into(), + vec![LogicalField::new("a", LogicalType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; let expr = is_true(lit(12i32).lt(lit(13i64))); - let expected = is_true(cast(lit(12i32), DataType::Int64).lt(lit(13i64))); + let expected = is_true(cast(lit(12i32), LogicalType::Int64).lt(lit(13i64))); let result = expr.rewrite(&mut rewriter).data()?; assert_eq!(expected, result); @@ -1317,9 +1323,9 @@ mod test { fn binary_op_date32_eq_ts() -> Result<()> { let expr = cast( lit("1998-03-18"), - DataType::Timestamp(TimeUnit::Nanosecond, None), + LogicalType::Timestamp(TimeUnit::Nanosecond, None), ) - .eq(cast(lit("1998-03-18"), DataType::Date32)); + .eq(cast(lit("1998-03-18"), LogicalType::Date32)); let empty = empty(); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); dbg!(&plan); @@ -1331,7 +1337,7 @@ mod test { fn cast_if_not_same_type( expr: Box, - data_type: &DataType, + data_type: &LogicalType, schema: &DFSchemaRef, ) -> Box { if &expr.get_type(schema).unwrap() != data_type { @@ -1343,8 +1349,8 @@ mod test { fn cast_helper( case: Case, - case_when_type: DataType, - then_else_type: DataType, + case_when_type: LogicalType, + then_else_type: LogicalType, schema: &DFSchemaRef, ) -> Case { let expr = case @@ -1375,23 +1381,23 @@ mod test { fn test_case_expression_coercion() -> Result<()> { let schema = Arc::new(DFSchema::from_unqualified_fields( vec![ - Field::new("boolean", DataType::Boolean, true), - Field::new("integer", DataType::Int32, true), - Field::new("float", DataType::Float32, true), - Field::new( + LogicalField::new("boolean", LogicalType::Boolean, true), + LogicalField::new("integer", LogicalType::Int32, true), + LogicalField::new("float", LogicalType::Float32, true), + LogicalField::new( "timestamp", - DataType::Timestamp(TimeUnit::Nanosecond, None), + LogicalType::Timestamp(TimeUnit::Nanosecond, None), true, ), - Field::new("date", DataType::Date32, true), - Field::new( + LogicalField::new("date", LogicalType::Date32, true), + LogicalField::new( "interval", - DataType::Interval(arrow::datatypes::IntervalUnit::MonthDayNano), + LogicalType::Interval(arrow::datatypes::IntervalUnit::MonthDayNano), true, ), - Field::new("binary", DataType::Binary, true), - Field::new("string", DataType::Utf8, true), - Field::new("decimal", DataType::Decimal128(10, 10), true), + LogicalField::new("binary", LogicalType::Binary, true), + LogicalField::new("string", LogicalType::Utf8, true), + LogicalField::new("decimal", LogicalType::Decimal128(10, 10), true), ] .into(), std::collections::HashMap::new(), @@ -1406,8 +1412,8 @@ mod test { ], else_expr: None, }; - let case_when_common_type = DataType::Boolean; - let then_else_common_type = DataType::Utf8; + let case_when_common_type = LogicalType::Boolean; + let then_else_common_type = LogicalType::Utf8; let expected = cast_helper( case.clone(), case_when_common_type, @@ -1426,8 +1432,8 @@ mod test { ], else_expr: Some(Box::new(col("string"))), }; - let case_when_common_type = DataType::Utf8; - let then_else_common_type = DataType::Utf8; + let case_when_common_type = LogicalType::Utf8; + let then_else_common_type = LogicalType::Utf8; let expected = cast_helper( case.clone(), case_when_common_type, @@ -1484,7 +1490,7 @@ mod test { Operator::Plus, Box::new(cast( lit("2000-01-01T00:00:00"), - DataType::Timestamp(TimeUnit::Nanosecond, None), + LogicalType::Timestamp(TimeUnit::Nanosecond, None), )), )); let empty = empty(); @@ -1499,12 +1505,12 @@ mod test { let expr = Expr::BinaryExpr(BinaryExpr::new( Box::new(cast( lit("1998-03-18"), - DataType::Timestamp(TimeUnit::Nanosecond, None), + LogicalType::Timestamp(TimeUnit::Nanosecond, None), )), Operator::Minus, Box::new(cast( lit("1998-03-18"), - DataType::Timestamp(TimeUnit::Nanosecond, None), + LogicalType::Timestamp(TimeUnit::Nanosecond, None), )), )); let empty = empty(); @@ -1518,8 +1524,8 @@ mod test { #[test] fn in_subquery_cast_subquery() -> Result<()> { - let empty_int32 = empty_with_type(DataType::Int32); - let empty_int64 = empty_with_type(DataType::Int64); + let empty_int32 = empty_with_type(LogicalType::Int32); + let empty_int64 = empty_with_type(LogicalType::Int64); let in_subquery_expr = Expr::InSubquery(InSubquery::new( Box::new(col("a")), @@ -1543,8 +1549,8 @@ mod test { #[test] fn in_subquery_cast_expr() -> Result<()> { - let empty_int32 = empty_with_type(DataType::Int32); - let empty_int64 = empty_with_type(DataType::Int64); + let empty_int32 = empty_with_type(LogicalType::Int32); + let empty_int64 = empty_with_type(LogicalType::Int64); let in_subquery_expr = Expr::InSubquery(InSubquery::new( Box::new(col("a")), @@ -1567,8 +1573,8 @@ mod test { #[test] fn in_subquery_cast_all() -> Result<()> { - let empty_inside = empty_with_type(DataType::Decimal128(10, 5)); - let empty_outside = empty_with_type(DataType::Decimal128(8, 8)); + let empty_inside = empty_with_type(LogicalType::Decimal128(10, 5)); + let empty_outside = empty_with_type(LogicalType::Decimal128(8, 8)); let in_subquery_expr = Expr::InSubquery(InSubquery::new( Box::new(col("a")), diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 81b987b0d4fc..d3032966b5ca 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1094,6 +1094,8 @@ mod test { use std::iter; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::AggregateFunction; use datafusion_expr::logical_plan::{table_scan, JoinType}; use datafusion_expr::{ @@ -1644,9 +1646,9 @@ mod test { let plan = table_scan(Some("table"), &schema, None) .unwrap() .filter( - cast(col("a"), DataType::Int64) + cast(col("a"), LogicalType::Int64) .lt(lit(1_i64)) - .and(cast(col("a"), DataType::Int64).not_eq(lit(1_i64))), + .and(cast(col("a"), LogicalType::Int64).not_eq(lit(1_i64))), ) .unwrap() .build() @@ -1704,9 +1706,9 @@ mod test { let grouping = grouping_set(vec![vec![col("a"), col("b")], vec![col("c")]]); let schema = DFSchema::from_unqualified_fields( vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - Field::new("c", DataType::Int32, false), + LogicalField::new("a", LogicalType::Int32, false), + LogicalField::new("b", LogicalType::Int32, false), + LogicalField::new("c", LogicalType::Int32, false), ] .into(), HashMap::default(), @@ -1723,8 +1725,8 @@ mod test { let grouping = grouping_set(vec![vec![col("a"), col("b")], vec![col("a")]]); let schema = DFSchema::from_unqualified_fields( vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), + LogicalField::new("a", LogicalType::Int32, false), + LogicalField::new("b", LogicalType::Int32, false), ] .into(), HashMap::default(), @@ -1791,7 +1793,7 @@ mod test { fn test_extract_expressions_from_col() -> Result<()> { let mut result = Vec::with_capacity(1); let schema = DFSchema::from_unqualified_fields( - vec![Field::new("a", DataType::Int32, false)].into(), + vec![LogicalField::new("a", LogicalType::Int32, false)].into(), HashMap::default(), )?; extract_expressions(&col("a"), &schema, &mut result)?; diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 81d6dc863af6..7f009cbd1164 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -369,7 +369,7 @@ mod tests { use super::*; use crate::test::*; - use arrow::datatypes::DataType; + use datafusion_common::logical_type::LogicalType; use datafusion_expr::{and, binary_expr, col, lit, not, or, out_ref_col}; fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { @@ -564,7 +564,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), )? .project(vec![col("orders.o_custkey")])? .build()?, @@ -605,7 +605,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("lineitem")) .filter( col("lineitem.l_orderkey") - .eq(out_ref_col(DataType::Int64, "orders.o_orderkey")), + .eq(out_ref_col(LogicalType::Int64, "orders.o_orderkey")), )? .project(vec![col("lineitem.l_orderkey")])? .build()?, @@ -616,7 +616,7 @@ mod tests { .filter( in_subquery(col("orders.o_orderkey"), lineitem).and( col("orders.o_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), ), )? .project(vec![col("orders.o_custkey")])? @@ -653,7 +653,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .and(col("o_orderkey").eq(lit(1))), )? @@ -688,8 +688,8 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + out_ref_col(LogicalType::Int64, "customer.c_custkey") + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), )? .project(vec![col("orders.o_custkey")])? .build()?, @@ -752,7 +752,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .not_eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -785,7 +785,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .lt(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -818,7 +818,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .or(col("o_orderkey").eq(lit(1))), )? @@ -876,7 +876,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -909,7 +909,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey").add(lit(1))])? @@ -942,7 +942,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey"), col("orders.o_orderkey")])? @@ -971,7 +971,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1008,7 +1008,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1045,7 +1045,7 @@ mod tests { fn in_subquery_correlated() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(test_table_scan_with_name("sq")?) - .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq.a")))? .project(vec![col("c")])? .build()?, ); @@ -1203,7 +1203,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( - out_ref_col(DataType::UInt32, "test.a") + out_ref_col(LogicalType::UInt32, "test.a") .eq(col("sq.a")) .and(col("sq.a").add(lit(1u32)).eq(col("sq.b"))), )? @@ -1238,8 +1238,8 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( - out_ref_col(DataType::UInt32, "test.a") - .add(out_ref_col(DataType::UInt32, "test.b")) + out_ref_col(LogicalType::UInt32, "test.a") + .add(out_ref_col(LogicalType::UInt32, "test.b")) .eq(col("sq.a").add(col("sq.b"))) .and(col("sq.a").add(lit(1u32)).eq(col("sq.b"))), )? @@ -1274,12 +1274,12 @@ mod tests { let subquery_scan2 = test_table_scan_with_name("sq2")?; let subquery1 = LogicalPlanBuilder::from(subquery_scan1) - .filter(out_ref_col(DataType::UInt32, "test.a").gt(col("sq1.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").gt(col("sq1.a")))? .project(vec![col("c") * lit(2u32)])? .build()?; let subquery2 = LogicalPlanBuilder::from(subquery_scan2) - .filter(out_ref_col(DataType::UInt32, "test.a").gt(col("sq2.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").gt(col("sq2.a")))? .project(vec![col("c") * lit(2u32)])? .build()?; @@ -1351,7 +1351,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), )? .project(vec![col("orders.o_custkey")])? .build()?, @@ -1382,7 +1382,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("lineitem")) .filter( col("lineitem.l_orderkey") - .eq(out_ref_col(DataType::Int64, "orders.o_orderkey")), + .eq(out_ref_col(LogicalType::Int64, "orders.o_orderkey")), )? .project(vec![col("lineitem.l_orderkey")])? .build()?, @@ -1393,7 +1393,7 @@ mod tests { .filter( exists(lineitem).and( col("orders.o_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), ), )? .project(vec![col("orders.o_custkey")])? @@ -1424,7 +1424,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .and(col("o_orderkey").eq(lit(1))), )? @@ -1452,7 +1452,7 @@ mod tests { fn exists_subquery_no_cols() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) - .filter(out_ref_col(DataType::Int64, "customer.c_custkey").eq(lit(1u32)))? + .filter(out_ref_col(LogicalType::Int64, "customer.c_custkey").eq(lit(1u32)))? .project(vec![col("orders.o_custkey")])? .build()?, ); @@ -1497,7 +1497,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .not_eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1525,7 +1525,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .lt(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1553,7 +1553,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .or(col("o_orderkey").eq(lit(1))), )? @@ -1582,7 +1582,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .build()?, @@ -1608,7 +1608,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey").add(lit(1))])? @@ -1636,7 +1636,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1690,7 +1690,7 @@ mod tests { fn exists_subquery_correlated() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(test_table_scan_with_name("sq")?) - .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq.a")))? .project(vec![col("c")])? .build()?, ); @@ -1741,12 +1741,12 @@ mod tests { let subquery_scan2 = test_table_scan_with_name("sq2")?; let subquery1 = LogicalPlanBuilder::from(subquery_scan1) - .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq1.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq1.a")))? .project(vec![col("c")])? .build()?; let subquery2 = LogicalPlanBuilder::from(subquery_scan2) - .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq2.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq2.a")))? .project(vec![col("c")])? .build()?; @@ -1780,7 +1780,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), )? .project(vec![lit(1u32)])? .build()?; @@ -1832,7 +1832,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), )? .project(vec![col("sq.c")])? .distinct()? @@ -1860,7 +1860,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), )? .project(vec![col("sq.b") + col("sq.c")])? .distinct()? @@ -1888,7 +1888,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), )? .project(vec![lit(1u32), col("sq.c")])? .distinct()? diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index edf6b72d7e17..d88491be1979 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -65,11 +65,12 @@ mod tests { use super::*; use crate::test::*; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::ToDFSchema; use datafusion_expr::{ expr_rewriter::coerce_plan_expr_for_schema, logical_plan::table_scan, }; use std::sync::Arc; + use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::ToDFSchema; fn schema() -> Schema { Schema::new(vec![ @@ -108,7 +109,7 @@ mod tests { fn eliminate_one_union() -> Result<()> { let table_plan = coerce_plan_expr_for_schema( &table_scan(Some("table"), &schema(), None)?.build()?, - &schema().to_dfschema()?, + &LogicalSchema::from(schema()).to_dfschema()?, )?; let schema = table_plan.schema().clone(); let single_union_plan = LogicalPlan::Union(Union { diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index ccc637a0eb01..882499cfa68b 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -301,7 +301,7 @@ fn extract_non_nullable_columns( mod tests { use super::*; use crate::test::*; - use arrow::datatypes::DataType; + use datafusion_common::logical_type::LogicalType; use datafusion_expr::{ binary_expr, cast, col, lit, logical_plan::builder::LogicalPlanBuilder, @@ -427,9 +427,9 @@ mod tests { None, )? .filter(binary_expr( - cast(col("t1.b"), DataType::Int64).gt(lit(10u32)), + cast(col("t1.b"), LogicalType::Int64).gt(lit(10u32)), And, - try_cast(col("t2.c"), DataType::Int64).lt(lit(20u32)), + try_cast(col("t2.c"), LogicalType::Int64).lt(lit(20u32)), ))? .build()?; let expected = "\ diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 87d205139e8e..820ef149c347 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -156,11 +156,11 @@ fn split_eq_and_noneq_join_predicate( mod tests { use super::*; use crate::test::*; - use arrow::datatypes::DataType; use datafusion_expr::{ col, lit, logical_plan::builder::LogicalPlanBuilder, JoinType, }; use std::sync::Arc; + use datafusion_common::logical_type::LogicalType; fn assert_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_display_indent( @@ -362,8 +362,8 @@ mod tests { // filter: t1.a + CAST(Int64(1), UInt32) = t2.a + CAST(Int64(2), UInt32) as t1.a + 1 = t2.a + 2 let filter = Expr::eq( - col("t1.a") + lit(1i64).cast_to(&DataType::UInt32, &t1_schema)?, - col("t2.a") + lit(2i32).cast_to(&DataType::UInt32, &t2_schema)?, + col("t1.a") + lit(1i64).cast_to(&LogicalType::UInt32, &t1_schema)?, + col("t2.a") + lit(2i32).cast_to(&LogicalType::UInt32, &t2_schema)?, ) .alias("t1.a + 1 = t2.a + 2"); let plan = LogicalPlanBuilder::from(t1) diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 2fbf77523bd1..e3d0bbf243c2 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -810,6 +810,8 @@ mod tests { use datafusion_common::{ Column, DFSchema, DFSchemaRef, JoinType, Result, TableReference, }; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::LogicalType; use datafusion_expr::AggregateExt; use datafusion_expr::{ binary_expr, build_join_schema, @@ -1172,7 +1174,7 @@ mod tests { fn test_try_cast() -> Result<()> { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![try_cast(col("a"), DataType::Float64)])? + .project(vec![try_cast(col("a"), LogicalType::Float64)])? .build()?; let expected = "Projection: TRY_CAST(test.a AS Float64)\ @@ -1544,15 +1546,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(Field::new("a", DataType::UInt32, false)) + Arc::new(LogicalField::new("a", LogicalType::UInt32, false)) ), ( Some("test".into()), - Arc::new(Field::new("b", DataType::UInt32, false)) + Arc::new(LogicalField::new("b", LogicalType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(Field::new("c1", DataType::UInt32, true)) + Arc::new(LogicalField::new("c1", LogicalType::UInt32, true)) ), ], HashMap::new() @@ -1596,15 +1598,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(Field::new("a", DataType::UInt32, false)) + Arc::new(LogicalField::new("a", LogicalType::UInt32, false)) ), ( Some("test".into()), - Arc::new(Field::new("b", DataType::UInt32, false)) + Arc::new(LogicalField::new("b", LogicalType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(Field::new("c1", DataType::UInt32, true)) + Arc::new(LogicalField::new("c1", LogicalType::UInt32, true)) ), ], HashMap::new() @@ -1646,15 +1648,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(Field::new("a", DataType::UInt32, false)) + Arc::new(LogicalField::new("a", LogicalType::UInt32, false)) ), ( Some("test".into()), - Arc::new(Field::new("b", DataType::UInt32, false)) + Arc::new(LogicalField::new("b", LogicalType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(Field::new("a", DataType::UInt32, true)) + Arc::new(LogicalField::new("a", LogicalType::UInt32, true)) ), ], HashMap::new() @@ -1671,7 +1673,7 @@ mod tests { let projection = LogicalPlanBuilder::from(table_scan) .project(vec![Expr::Cast(Cast::new( Box::new(col("c")), - DataType::Float64, + LogicalType::Float64, ))])? .build()?; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 14e5ac141eeb..37fd448eff43 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -540,17 +540,17 @@ mod tests { "Optimizer rule 'get table_scan rule' failed\n\ caused by\nget table_scan rule\ncaused by\n\ Internal error: Failed due to a difference in schemas, \ - original schema: DFSchema { inner: Schema { \ + original schema: DFSchema { inner: LogicalSchema { \ fields: [], \ metadata: {} }, \ field_qualifiers: [], \ functional_dependencies: FunctionalDependencies { deps: [] } \ }, \ - new schema: DFSchema { inner: Schema { \ + new schema: DFSchema { inner: LogicalSchema { \ fields: [\ - Field { name: \"a\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, \ - Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, \ - Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }\ + LogicalField { name: \"a\", data_type: UInt32, nullable: false, metadata: {} }, \ + LogicalField { name: \"b\", data_type: UInt32, nullable: false, metadata: {} }, \ + LogicalField { name: \"c\", data_type: UInt32, nullable: false, metadata: {} }\ ], \ metadata: {} }, \ field_qualifiers: [Some(Bare { table: \"test\" }), Some(Bare { table: \"test\" }), Some(Bare { table: \"test\" })], \ diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 576dabe305e6..5212b8df6e1b 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -246,9 +246,10 @@ fn empty_child(plan: &LogicalPlan) -> Result> { mod tests { use std::sync::Arc; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::datatypes::{DataType, Field, Fields, Schema}; use datafusion_common::{Column, DFSchema, JoinType, ScalarValue}; + use datafusion_common::logical_type::fields::LogicalFields; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, Expr, Operator, @@ -574,7 +575,7 @@ mod tests { fn test_empty_with_non_empty() -> Result<()> { let table_scan = test_table_scan()?; - let fields = test_table_scan_fields(); + let fields = LogicalFields::from(Fields::from(test_table_scan_fields())); let empty = LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index fa432ad76de5..91edc6844313 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -1149,7 +1149,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use async_trait::async_trait; - + use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_common::ScalarValue; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::table_scan; @@ -2387,7 +2387,7 @@ mod tests { table_name: "test".into(), filters: vec![], projected_schema: Arc::new(DFSchema::try_from( - (*test_provider.schema()).clone(), + LogicalSchema::from((*test_provider.schema()).clone()), )?), projection: None, source: Arc::new(test_provider), @@ -2459,7 +2459,7 @@ mod tests { table_name: "test".into(), filters: vec![col("a").eq(lit(10i64)), col("b").gt(lit(11i64))], projected_schema: Arc::new(DFSchema::try_from( - (*test_provider.schema()).clone(), + LogicalSchema::from((*test_provider.schema()).clone()), )?), projection: Some(vec![0]), source: Arc::new(test_provider), @@ -2488,7 +2488,7 @@ mod tests { table_name: "test".into(), filters: vec![], projected_schema: Arc::new(DFSchema::try_from( - (*test_provider.schema()).clone(), + LogicalSchema::from((*test_provider.schema()).clone()), )?), projection: Some(vec![0]), source: Arc::new(test_provider), diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index 0333cc8dde36..b79343f9b041 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -392,7 +392,7 @@ mod tests { use super::*; use crate::test::*; - use arrow::datatypes::DataType; + use datafusion_common::logical_type::LogicalType; use datafusion_expr::test::function_stub::sum; use datafusion_expr::{col, lit, max, min, out_ref_col, scalar_subquery, Between}; @@ -403,7 +403,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? .project(vec![max(col("orders.o_custkey"))])? @@ -447,7 +447,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("lineitem")) .filter( col("lineitem.l_orderkey") - .eq(out_ref_col(DataType::Int64, "orders.o_orderkey")), + .eq(out_ref_col(LogicalType::Int64, "orders.o_orderkey")), )? .aggregate( Vec::::new(), @@ -461,7 +461,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")) + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")) .and(col("orders.o_totalprice").lt(scalar_subquery(lineitem))), )? .aggregate(Vec::::new(), vec![sum(col("orders.o_totalprice"))])? @@ -502,7 +502,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .and(col("o_orderkey").eq(lit(1))), )? @@ -540,8 +540,8 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") - .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), + out_ref_col(LogicalType::Int64, "customer.c_custkey") + .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? .project(vec![max(col("orders.o_custkey"))])? @@ -610,7 +610,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .not_eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -637,7 +637,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .lt(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -664,7 +664,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .or(col("o_orderkey").eq(lit(1))), )? @@ -713,7 +713,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -775,7 +775,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -814,7 +814,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -854,7 +854,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -893,7 +893,7 @@ mod tests { fn exists_subquery_correlated() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(test_table_scan_with_name("sq")?) - .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq.a")))? + .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq.a")))? .aggregate(Vec::::new(), vec![min(col("c"))])? .project(vec![min(col("c"))])? .build()?, @@ -989,7 +989,7 @@ mod tests { let sq1 = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![min(col("orders.o_custkey"))])? @@ -999,7 +999,7 @@ mod tests { let sq2 = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(DataType::Int64, "customer.c_custkey") + out_ref_col(LogicalType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 36dd85ac96e1..c2acdd17c70e 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -32,6 +32,7 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, }; use datafusion_common::{internal_err, DFSchema, DataFusionError, Result, ScalarValue}; +use datafusion_common::logical_type::extension::ExtensionType; use datafusion_expr::expr::{ AggregateFunctionDefinition, InList, InSubquery, WindowFunction, }; @@ -1050,7 +1051,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&left, &right) && !info.nullable(&right)? => { Transformed::yes(Expr::Literal(ScalarValue::new_zero( - &info.get_data_type(&left)?, + &info.get_data_type(&left)?.physical_type(), )?)) } @@ -1061,7 +1062,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&right, &left) && !info.nullable(&left)? => { Transformed::yes(Expr::Literal(ScalarValue::new_zero( - &info.get_data_type(&left)?, + &info.get_data_type(&left)?.physical_type(), )?)) } @@ -1136,7 +1137,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&left, &right) && !info.nullable(&right)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?, + &info.get_data_type(&left)?.physical_type(), )?)) } @@ -1147,7 +1148,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&right, &left) && !info.nullable(&left)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?, + &info.get_data_type(&left)?.physical_type(), )?)) } @@ -1222,7 +1223,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&left, &right) && !info.nullable(&right)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?, + &info.get_data_type(&left)?.physical_type(), )?)) } @@ -1233,7 +1234,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&right, &left) && !info.nullable(&left)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?, + &info.get_data_type(&left)?.physical_type(), )?)) } @@ -1245,7 +1246,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if expr_contains(&left, &right, BitwiseXor) => { let expr = delete_xor_in_complex_expr(&left, &right, false); Transformed::yes(if expr == *right { - Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&right)?)?) + Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&right)?.physical_type())?) } else { expr }) @@ -1259,7 +1260,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if expr_contains(&right, &left, BitwiseXor) => { let expr = delete_xor_in_complex_expr(&right, &left, true); Transformed::yes(if expr == *left { - Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&left)?)?) + Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&left)?.physical_type())?) } else { expr }) @@ -1783,7 +1784,9 @@ mod tests { ops::{BitAnd, BitOr, BitXor}, sync::Arc, }; - + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::LogicalType; + use datafusion_common::logical_type::schema::LogicalSchema; use crate::simplify_expressions::SimplifyContext; use crate::test::test_table_scan_with_name; @@ -1822,9 +1825,9 @@ mod tests { } fn test_schema() -> DFSchemaRef { - Schema::new(vec![ - Field::new("i", DataType::Int64, false), - Field::new("b", DataType::Boolean, true), + LogicalSchema::new(vec![ + LogicalField::new("i", LogicalType::Int64, false), + LogicalField::new("b", LogicalType::Boolean, true), ]) .to_dfschema_ref() .unwrap() @@ -3007,14 +3010,14 @@ mod tests { Arc::new( DFSchema::from_unqualified_fields( vec![ - Field::new("c1", DataType::Utf8, true), - Field::new("c2", DataType::Boolean, true), - Field::new("c3", DataType::Int64, true), - Field::new("c4", DataType::UInt32, true), - Field::new("c1_non_null", DataType::Utf8, false), - Field::new("c2_non_null", DataType::Boolean, false), - Field::new("c3_non_null", DataType::Int64, false), - Field::new("c4_non_null", DataType::UInt32, false), + LogicalField::new("c1", LogicalType::Utf8, true), + LogicalField::new("c2", LogicalType::Boolean, true), + LogicalField::new("c3", LogicalType::Int64, true), + LogicalField::new("c4", LogicalType::UInt32, true), + LogicalField::new("c1_non_null", LogicalType::Utf8, false), + LogicalField::new("c2_non_null", LogicalType::Boolean, false), + LogicalField::new("c3_non_null", LogicalType::Int64, false), + LogicalField::new("c4_non_null", LogicalType::UInt32, false), ] .into(), HashMap::new(), @@ -3102,7 +3105,7 @@ mod tests { #[test] fn simplify_expr_eq() { let schema = expr_test_schema(); - assert_eq!(col("c2").get_type(&schema).unwrap(), DataType::Boolean); + assert_eq!(col("c2").get_type(&schema).unwrap(), LogicalType::Boolean); // true = true -> true assert_eq!(simplify(lit(true).eq(lit(true))), lit(true)); @@ -3126,7 +3129,7 @@ mod tests { // expression to non-boolean. // // Make sure c1 column to be used in tests is not boolean type - assert_eq!(col("c1").get_type(&schema).unwrap(), DataType::Utf8); + assert_eq!(col("c1").get_type(&schema).unwrap(), LogicalType::Utf8); // don't fold c1 = foo assert_eq!(simplify(col("c1").eq(lit("foo"))), col("c1").eq(lit("foo")),); @@ -3136,7 +3139,7 @@ mod tests { fn simplify_expr_not_eq() { let schema = expr_test_schema(); - assert_eq!(col("c2").get_type(&schema).unwrap(), DataType::Boolean); + assert_eq!(col("c2").get_type(&schema).unwrap(), LogicalType::Boolean); // c2 != true -> !c2 assert_eq!(simplify(col("c2").not_eq(lit(true))), col("c2").not(),); @@ -3157,7 +3160,7 @@ mod tests { // when one of the operand is not of boolean type, folding the // other boolean constant will change return type of // expression to non-boolean. - assert_eq!(col("c1").get_type(&schema).unwrap(), DataType::Utf8); + assert_eq!(col("c1").get_type(&schema).unwrap(), LogicalType::Utf8); assert_eq!( simplify(col("c1").not_eq(lit("foo"))), diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index e650d4c09c23..628da80bb3dd 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -93,7 +93,7 @@ impl SimplifyExpressions { // projection applied for simplification Arc::new(DFSchema::try_from_qualified_schema( scan.table_name.clone(), - &scan.source.schema(), + &scan.source.schema().as_ref().clone().into(), )?) } else { Arc::new(DFSchema::empty()) @@ -151,7 +151,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use chrono::{DateTime, Utc}; - + use datafusion_common::logical_type::LogicalType; use crate::optimizer::Optimizer; use datafusion_expr::logical_plan::builder::table_scan_with_filters; use datafusion_expr::logical_plan::table_scan; @@ -445,7 +445,7 @@ mod tests { #[test] fn cast_expr() -> Result<()> { let table_scan = test_table_scan(); - let proj = vec![Expr::Cast(Cast::new(Box::new(lit("0")), DataType::Int32))]; + let proj = vec![Expr::Cast(Cast::new(Box::new(lit("0")), LogicalType::Int32))]; let plan = LogicalPlanBuilder::from(table_scan) .project(proj)? .build()?; @@ -703,9 +703,9 @@ mod tests { let t1 = test_table_scan_with_name("t1")?; let t2 = test_table_scan_with_name("t2")?; - let left_key = col("t1.a") + lit(1i64).cast_to(&DataType::UInt32, t1.schema())?; + let left_key = col("t1.a") + lit(1i64).cast_to(&LogicalType::UInt32, t1.schema())?; let right_key = - col("t2.a") + lit(2i64).cast_to(&DataType::UInt32, t2.schema())?; + col("t2.a") + lit(2i64).cast_to(&LogicalType::UInt32, t2.schema())?; let plan = LogicalPlanBuilder::from(t1) .join_with_expr_keys( t2, diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 2c7e8644026e..dbabdb5c926f 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -59,7 +59,7 @@ pub fn assert_fields_eq(plan: &LogicalPlan, expected: Vec<&str>) { .schema() .fields() .iter() - .map(|f| f.name().clone()) + .map(|f| f.name().to_string()) .collect(); assert_eq!(actual, expected); } diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index de471d59c466..5135272119af 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -26,11 +26,12 @@ use crate::{OptimizerConfig, OptimizerRule}; use crate::utils::NamePreserver; use arrow::datatypes::{ - DataType, TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION, + TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION, }; use arrow::temporal_conversions::{MICROSECONDS, MILLISECONDS, NANOSECONDS}; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{internal_err, DFSchema, DFSchemaRef, Result, ScalarValue}; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::{BinaryExpr, Cast, InList, TryCast}; use datafusion_expr::utils::merge_schema; use datafusion_expr::{lit, Expr, ExprSchemable, LogicalPlan, Operator}; @@ -104,7 +105,7 @@ impl OptimizerRule for UnwrapCastInComparison { if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( ts.table_name.clone(), - &ts.source.schema(), + &ts.source.schema().as_ref().clone().into(), )?; schema.merge(&source_schema); } @@ -275,46 +276,39 @@ fn is_comparison_op(op: &Operator) -> bool { } /// Returns true if [UnwrapCastExprRewriter] supports this data type -fn is_supported_type(data_type: &DataType) -> bool { +fn is_supported_type(data_type: &LogicalType) -> bool { is_supported_numeric_type(data_type) || is_supported_string_type(data_type) - || is_supported_dictionary_type(data_type) } /// Returns true if [[UnwrapCastExprRewriter]] suppors this numeric type -fn is_supported_numeric_type(data_type: &DataType) -> bool { +fn is_supported_numeric_type(data_type: &LogicalType) -> bool { matches!( data_type, - DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 - | DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::Decimal128(_, _) - | DataType::Timestamp(_, _) + LogicalType::UInt8 + | LogicalType::UInt16 + | LogicalType::UInt32 + | LogicalType::UInt64 + | LogicalType::Int8 + | LogicalType::Int16 + | LogicalType::Int32 + | LogicalType::Int64 + | LogicalType::Decimal128(_, _) + | LogicalType::Timestamp(_, _) ) } /// Returns true if [UnwrapCastExprRewriter] supports casting this value as a string -fn is_supported_string_type(data_type: &DataType) -> bool { - matches!(data_type, DataType::Utf8 | DataType::LargeUtf8) -} - -/// Returns true if [UnwrapCastExprRewriter] supports casting this value as a dictionary -fn is_supported_dictionary_type(data_type: &DataType) -> bool { - matches!(data_type, - DataType::Dictionary(_, inner) if is_supported_type(inner)) +fn is_supported_string_type(data_type: &LogicalType) -> bool { + matches!(data_type, LogicalType::Utf8 | LogicalType::LargeUtf8) } /// Convert a literal value from one data type to another fn try_cast_literal_to_type( lit_value: &ScalarValue, - target_type: &DataType, + target_type: &LogicalType, ) -> Option { - let lit_data_type = lit_value.data_type(); + let lit_data_type = lit_value.data_type().into(); if !is_supported_type(&lit_data_type) || !is_supported_type(target_type) { return None; } @@ -324,15 +318,14 @@ fn try_cast_literal_to_type( } try_cast_numeric_literal(lit_value, target_type) .or_else(|| try_cast_string_literal(lit_value, target_type)) - .or_else(|| try_cast_dictionary(lit_value, target_type)) } /// Convert a numeric value from one numeric data type to another fn try_cast_numeric_literal( lit_value: &ScalarValue, - target_type: &DataType, + target_type: &LogicalType, ) -> Option { - let lit_data_type = lit_value.data_type(); + let lit_data_type = lit_value.data_type().into(); if !is_supported_numeric_type(&lit_data_type) || !is_supported_numeric_type(target_type) { @@ -340,29 +333,29 @@ fn try_cast_numeric_literal( } let mul = match target_type { - DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 - | DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 => 1_i128, - DataType::Timestamp(_, _) => 1_i128, - DataType::Decimal128(_, scale) => 10_i128.pow(*scale as u32), + LogicalType::UInt8 + | LogicalType::UInt16 + | LogicalType::UInt32 + | LogicalType::UInt64 + | LogicalType::Int8 + | LogicalType::Int16 + | LogicalType::Int32 + | LogicalType::Int64 => 1_i128, + LogicalType::Timestamp(_, _) => 1_i128, + LogicalType::Decimal128(_, scale) => 10_i128.pow(*scale as u32), _ => return None, }; let (target_min, target_max) = match target_type { - DataType::UInt8 => (u8::MIN as i128, u8::MAX as i128), - DataType::UInt16 => (u16::MIN as i128, u16::MAX as i128), - DataType::UInt32 => (u32::MIN as i128, u32::MAX as i128), - DataType::UInt64 => (u64::MIN as i128, u64::MAX as i128), - DataType::Int8 => (i8::MIN as i128, i8::MAX as i128), - DataType::Int16 => (i16::MIN as i128, i16::MAX as i128), - DataType::Int32 => (i32::MIN as i128, i32::MAX as i128), - DataType::Int64 => (i64::MIN as i128, i64::MAX as i128), - DataType::Timestamp(_, _) => (i64::MIN as i128, i64::MAX as i128), - DataType::Decimal128(precision, _) => ( + LogicalType::UInt8 => (u8::MIN as i128, u8::MAX as i128), + LogicalType::UInt16 => (u16::MIN as i128, u16::MAX as i128), + LogicalType::UInt32 => (u32::MIN as i128, u32::MAX as i128), + LogicalType::UInt64 => (u64::MIN as i128, u64::MAX as i128), + LogicalType::Int8 => (i8::MIN as i128, i8::MAX as i128), + LogicalType::Int16 => (i16::MIN as i128, i16::MAX as i128), + LogicalType::Int32 => (i32::MIN as i128, i32::MAX as i128), + LogicalType::Int64 => (i64::MIN as i128, i64::MAX as i128), + LogicalType::Timestamp(_, _) => (i64::MIN as i128, i64::MAX as i128), + LogicalType::Decimal128(precision, _) => ( // Different precision for decimal128 can store different range of value. // For example, the precision is 3, the max of value is `999` and the min // value is `-999` @@ -413,47 +406,47 @@ fn try_cast_numeric_literal( // the value casted from lit to the target type is in the range of target type. // return the target type of scalar value let result_scalar = match target_type { - DataType::Int8 => ScalarValue::Int8(Some(value as i8)), - DataType::Int16 => ScalarValue::Int16(Some(value as i16)), - DataType::Int32 => ScalarValue::Int32(Some(value as i32)), - DataType::Int64 => ScalarValue::Int64(Some(value as i64)), - DataType::UInt8 => ScalarValue::UInt8(Some(value as u8)), - DataType::UInt16 => ScalarValue::UInt16(Some(value as u16)), - DataType::UInt32 => ScalarValue::UInt32(Some(value as u32)), - DataType::UInt64 => ScalarValue::UInt64(Some(value as u64)), - DataType::Timestamp(TimeUnit::Second, tz) => { + LogicalType::Int8 => ScalarValue::Int8(Some(value as i8)), + LogicalType::Int16 => ScalarValue::Int16(Some(value as i16)), + LogicalType::Int32 => ScalarValue::Int32(Some(value as i32)), + LogicalType::Int64 => ScalarValue::Int64(Some(value as i64)), + LogicalType::UInt8 => ScalarValue::UInt8(Some(value as u8)), + LogicalType::UInt16 => ScalarValue::UInt16(Some(value as u16)), + LogicalType::UInt32 => ScalarValue::UInt32(Some(value as u32)), + LogicalType::UInt64 => ScalarValue::UInt64(Some(value as u64)), + LogicalType::Timestamp(TimeUnit::Second, tz) => { let value = cast_between_timestamp( lit_data_type, - DataType::Timestamp(TimeUnit::Second, tz.clone()), + LogicalType::Timestamp(TimeUnit::Second, tz.clone()), value, ); ScalarValue::TimestampSecond(value, tz.clone()) } - DataType::Timestamp(TimeUnit::Millisecond, tz) => { + LogicalType::Timestamp(TimeUnit::Millisecond, tz) => { let value = cast_between_timestamp( lit_data_type, - DataType::Timestamp(TimeUnit::Millisecond, tz.clone()), + LogicalType::Timestamp(TimeUnit::Millisecond, tz.clone()), value, ); ScalarValue::TimestampMillisecond(value, tz.clone()) } - DataType::Timestamp(TimeUnit::Microsecond, tz) => { + LogicalType::Timestamp(TimeUnit::Microsecond, tz) => { let value = cast_between_timestamp( lit_data_type, - DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), + LogicalType::Timestamp(TimeUnit::Microsecond, tz.clone()), value, ); ScalarValue::TimestampMicrosecond(value, tz.clone()) } - DataType::Timestamp(TimeUnit::Nanosecond, tz) => { + LogicalType::Timestamp(TimeUnit::Nanosecond, tz) => { let value = cast_between_timestamp( lit_data_type, - DataType::Timestamp(TimeUnit::Nanosecond, tz.clone()), + LogicalType::Timestamp(TimeUnit::Nanosecond, tz.clone()), value, ); ScalarValue::TimestampNanosecond(value, tz.clone()) } - DataType::Decimal128(p, s) => { + LogicalType::Decimal128(p, s) => { ScalarValue::Decimal128(Some(value), *p, *s) } _ => { @@ -470,62 +463,36 @@ fn try_cast_numeric_literal( fn try_cast_string_literal( lit_value: &ScalarValue, - target_type: &DataType, + target_type: &LogicalType, ) -> Option { let string_value = match lit_value { ScalarValue::Utf8(s) | ScalarValue::LargeUtf8(s) => s.clone(), _ => return None, }; let scalar_value = match target_type { - DataType::Utf8 => ScalarValue::Utf8(string_value), - DataType::LargeUtf8 => ScalarValue::LargeUtf8(string_value), + LogicalType::Utf8 => ScalarValue::Utf8(string_value), + LogicalType::LargeUtf8 => ScalarValue::LargeUtf8(string_value), _ => return None, }; Some(scalar_value) } -/// Attempt to cast to/from a dictionary type by wrapping/unwrapping the dictionary -fn try_cast_dictionary( - lit_value: &ScalarValue, - target_type: &DataType, -) -> Option { - let lit_value_type = lit_value.data_type(); - let result_scalar = match (lit_value, target_type) { - // Unwrap dictionary when inner type matches target type - (ScalarValue::Dictionary(_, inner_value), _) - if inner_value.data_type() == *target_type => - { - (**inner_value).clone() - } - // Wrap type when target type is dictionary - (_, DataType::Dictionary(index_type, inner_type)) - if **inner_type == lit_value_type => - { - ScalarValue::Dictionary(index_type.clone(), Box::new(lit_value.clone())) - } - _ => { - return None; - } - }; - Some(result_scalar) -} - /// Cast a timestamp value from one unit to another -fn cast_between_timestamp(from: DataType, to: DataType, value: i128) -> Option { +fn cast_between_timestamp(from: LogicalType, to: LogicalType, value: i128) -> Option { let value = value as i64; let from_scale = match from { - DataType::Timestamp(TimeUnit::Second, _) => 1, - DataType::Timestamp(TimeUnit::Millisecond, _) => MILLISECONDS, - DataType::Timestamp(TimeUnit::Microsecond, _) => MICROSECONDS, - DataType::Timestamp(TimeUnit::Nanosecond, _) => NANOSECONDS, + LogicalType::Timestamp(TimeUnit::Second, _) => 1, + LogicalType::Timestamp(TimeUnit::Millisecond, _) => MILLISECONDS, + LogicalType::Timestamp(TimeUnit::Microsecond, _) => MICROSECONDS, + LogicalType::Timestamp(TimeUnit::Nanosecond, _) => NANOSECONDS, _ => return Some(value), }; let to_scale = match to { - DataType::Timestamp(TimeUnit::Second, _) => 1, - DataType::Timestamp(TimeUnit::Millisecond, _) => MILLISECONDS, - DataType::Timestamp(TimeUnit::Microsecond, _) => MICROSECONDS, - DataType::Timestamp(TimeUnit::Nanosecond, _) => NANOSECONDS, + LogicalType::Timestamp(TimeUnit::Second, _) => 1, + LogicalType::Timestamp(TimeUnit::Millisecond, _) => MILLISECONDS, + LogicalType::Timestamp(TimeUnit::Microsecond, _) => MICROSECONDS, + LogicalType::Timestamp(TimeUnit::Nanosecond, _) => NANOSECONDS, _ => return Some(value), }; @@ -543,7 +510,8 @@ mod tests { use super::*; use arrow::compute::{cast_with_options, CastOptions}; - use arrow::datatypes::Field; + use datafusion_common::logical_type::extension::ExtensionType; + use datafusion_common::logical_type::field::LogicalField; use datafusion_common::tree_node::TransformedResult; use datafusion_expr::{cast, col, in_list, try_cast}; @@ -551,7 +519,7 @@ mod tests { fn test_not_unwrap_cast_comparison() { let schema = expr_test_schema(); // cast(INT32(c1), INT64) > INT64(c2) - let c1_gt_c2 = cast(col("c1"), DataType::Int64).gt(col("c2")); + let c1_gt_c2 = cast(col("c1"), LogicalType::Int64).gt(col("c2")); assert_eq!(optimize_test(c1_gt_c2.clone(), &schema), c1_gt_c2); // INT32(c1) < INT32(16), the type is same @@ -559,7 +527,7 @@ mod tests { assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // the 99999999999 is not within the range of MAX(int32) and MIN(int32), we don't cast the lit(99999999999) to int32 type - let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(99999999999i64)); + let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(99999999999i64)); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); } @@ -568,25 +536,25 @@ mod tests { let schema = expr_test_schema(); // cast(c1, INT64) < INT64(16) -> INT32(c1) < cast(INT32(16)) // the 16 is within the range of MAX(int32) and MIN(int32), we can cast the 16 to int32(16) - let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(16i64)); + let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(16i64)); let expected = col("c1").lt(lit(16i32)); assert_eq!(optimize_test(expr_lt, &schema), expected); - let expr_lt = try_cast(col("c1"), DataType::Int64).lt(lit(16i64)); + let expr_lt = try_cast(col("c1"), LogicalType::Int64).lt(lit(16i64)); let expected = col("c1").lt(lit(16i32)); assert_eq!(optimize_test(expr_lt, &schema), expected); // cast(c2, INT32) = INT32(16) => INT64(c2) = INT64(16) - let c2_eq_lit = cast(col("c2"), DataType::Int32).eq(lit(16i32)); + let c2_eq_lit = cast(col("c2"), LogicalType::Int32).eq(lit(16i32)); let expected = col("c2").eq(lit(16i64)); assert_eq!(optimize_test(c2_eq_lit, &schema), expected); // cast(c1, INT64) < INT64(NULL) => INT32(c1) < INT32(NULL) - let c1_lt_lit_null = cast(col("c1"), DataType::Int64).lt(null_i64()); + let c1_lt_lit_null = cast(col("c1"), LogicalType::Int64).lt(null_i64()); let expected = col("c1").lt(null_i32()); assert_eq!(optimize_test(c1_lt_lit_null, &schema), expected); // cast(INT8(NULL), INT32) < INT32(12) => INT8(NULL) < INT8(12) - let lit_lt_lit = cast(null_i8(), DataType::Int32).lt(lit(12i32)); + let lit_lt_lit = cast(null_i8(), LogicalType::Int32).lt(lit(12i32)); let expected = null_i8().lt(lit(12i8)); assert_eq!(optimize_test(lit_lt_lit, &schema), expected); } @@ -595,77 +563,38 @@ mod tests { fn test_unwrap_cast_comparison_unsigned() { // "cast(c6, UINT64) = 0u64 => c6 = 0u32 let schema = expr_test_schema(); - let expr_input = cast(col("c6"), DataType::UInt64).eq(lit(0u64)); + let expr_input = cast(col("c6"), LogicalType::UInt64).eq(lit(0u64)); let expected = col("c6").eq(lit(0u32)); assert_eq!(optimize_test(expr_input, &schema), expected); } - #[test] - fn test_unwrap_cast_comparison_string() { - let schema = expr_test_schema(); - let dict = ScalarValue::Dictionary( - Box::new(DataType::Int32), - Box::new(ScalarValue::from("value")), - ); - - // cast(str1 as Dictionary) = arrow_cast('value', 'Dictionary') => str1 = Utf8('value1') - let expr_input = cast(col("str1"), dict.data_type()).eq(lit(dict.clone())); - let expected = col("str1").eq(lit("value")); - assert_eq!(optimize_test(expr_input, &schema), expected); - - // cast(tag as Utf8) = Utf8('value') => tag = arrow_cast('value', 'Dictionary') - let expr_input = cast(col("tag"), DataType::Utf8).eq(lit("value")); - let expected = col("tag").eq(lit(dict.clone())); - assert_eq!(optimize_test(expr_input, &schema), expected); - - // Verify reversed argument order - // arrow_cast('value', 'Dictionary') = cast(str1 as Dictionary) => Utf8('value1') = str1 - let expr_input = lit(dict.clone()).eq(cast(col("str1"), dict.data_type())); - let expected = lit("value").eq(col("str1")); - assert_eq!(optimize_test(expr_input, &schema), expected); - } - - #[test] - fn test_unwrap_cast_comparison_large_string() { - let schema = expr_test_schema(); - // cast(largestr as Dictionary) = arrow_cast('value', 'Dictionary') => str1 = LargeUtf8('value1') - let dict = ScalarValue::Dictionary( - Box::new(DataType::Int32), - Box::new(ScalarValue::LargeUtf8(Some("value".to_owned()))), - ); - let expr_input = cast(col("largestr"), dict.data_type()).eq(lit(dict.clone())); - let expected = - col("largestr").eq(lit(ScalarValue::LargeUtf8(Some("value".to_owned())))); - assert_eq!(optimize_test(expr_input, &schema), expected); - } - #[test] fn test_not_unwrap_cast_with_decimal_comparison() { let schema = expr_test_schema(); // integer to decimal: value is out of the bounds of the decimal // cast(c3, INT64) = INT64(100000000000000000) - let expr_eq = cast(col("c3"), DataType::Int64).eq(lit(100000000000000000i64)); + let expr_eq = cast(col("c3"), LogicalType::Int64).eq(lit(100000000000000000i64)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // cast(c4, INT64) = INT64(1000) will overflow the i128 - let expr_eq = cast(col("c4"), DataType::Int64).eq(lit(1000i64)); + let expr_eq = cast(col("c4"), LogicalType::Int64).eq(lit(1000i64)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // decimal to decimal: value will lose the scale when convert to the target data type // c3 = DECIMAL(12340,20,4) let expr_eq = - cast(col("c3"), DataType::Decimal128(20, 4)).eq(lit_decimal(12340, 20, 4)); + cast(col("c3"), LogicalType::Decimal128(20, 4)).eq(lit_decimal(12340, 20, 4)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // decimal to integer // c1 = DECIMAL(123, 10, 1): value will lose the scale when convert to the target data type let expr_eq = - cast(col("c1"), DataType::Decimal128(10, 1)).eq(lit_decimal(123, 10, 1)); + cast(col("c1"), LogicalType::Decimal128(10, 1)).eq(lit_decimal(123, 10, 1)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // c1 = DECIMAL(1230, 10, 2): value will lose the scale when convert to the target data type let expr_eq = - cast(col("c1"), DataType::Decimal128(10, 2)).eq(lit_decimal(1230, 10, 2)); + cast(col("c1"), LogicalType::Decimal128(10, 2)).eq(lit_decimal(1230, 10, 2)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); } @@ -674,32 +603,32 @@ mod tests { let schema = expr_test_schema(); // integer to decimal // c3 < INT64(16) -> c3 < (CAST(INT64(16) AS DECIMAL(18,2)); - let expr_lt = try_cast(col("c3"), DataType::Int64).lt(lit(16i64)); + let expr_lt = try_cast(col("c3"), LogicalType::Int64).lt(lit(16i64)); let expected = col("c3").lt(lit_decimal(1600, 18, 2)); assert_eq!(optimize_test(expr_lt, &schema), expected); // c3 < INT64(NULL) - let c1_lt_lit_null = cast(col("c3"), DataType::Int64).lt(null_i64()); + let c1_lt_lit_null = cast(col("c3"), LogicalType::Int64).lt(null_i64()); let expected = col("c3").lt(null_decimal(18, 2)); assert_eq!(optimize_test(c1_lt_lit_null, &schema), expected); // decimal to decimal // c3 < Decimal(123,10,0) -> c3 < CAST(DECIMAL(123,10,0) AS DECIMAL(18,2)) -> c3 < DECIMAL(12300,18,2) let expr_lt = - cast(col("c3"), DataType::Decimal128(10, 0)).lt(lit_decimal(123, 10, 0)); + cast(col("c3"), LogicalType::Decimal128(10, 0)).lt(lit_decimal(123, 10, 0)); let expected = col("c3").lt(lit_decimal(12300, 18, 2)); assert_eq!(optimize_test(expr_lt, &schema), expected); // c3 < Decimal(1230,10,3) -> c3 < CAST(DECIMAL(1230,10,3) AS DECIMAL(18,2)) -> c3 < DECIMAL(123,18,2) let expr_lt = - cast(col("c3"), DataType::Decimal128(10, 3)).lt(lit_decimal(1230, 10, 3)); + cast(col("c3"), LogicalType::Decimal128(10, 3)).lt(lit_decimal(1230, 10, 3)); let expected = col("c3").lt(lit_decimal(123, 18, 2)); assert_eq!(optimize_test(expr_lt, &schema), expected); // decimal to integer // c1 < Decimal(12300, 10, 2) -> c1 < CAST(DECIMAL(12300,10,2) AS INT32) -> c1 < INT32(123) let expr_lt = - cast(col("c1"), DataType::Decimal128(10, 2)).lt(lit_decimal(12300, 10, 2)); + cast(col("c1"), LogicalType::Decimal128(10, 2)).lt(lit_decimal(12300, 10, 2)); let expected = col("c1").lt(lit(123i32)); assert_eq!(optimize_test(expr_lt, &schema), expected); } @@ -710,21 +639,21 @@ mod tests { // internal left type is not supported // FLOAT32(C5) in ... let expr_lt = - cast(col("c5"), DataType::Int64).in_list(vec![lit(12i64), lit(12i64)], false); + cast(col("c5"), LogicalType::Int64).in_list(vec![lit(12i64), lit(12i64)], false); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // cast(INT32(C1), Float32) in (FLOAT32(1.23), Float32(12), Float32(12)) - let expr_lt = cast(col("c1"), DataType::Float32) + let expr_lt = cast(col("c1"), LogicalType::Float32) .in_list(vec![lit(12.0f32), lit(12.0f32), lit(1.23f32)], false); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // INT32(C1) in (INT64(99999999999), INT64(12)) - let expr_lt = cast(col("c1"), DataType::Int64) + let expr_lt = cast(col("c1"), LogicalType::Int64) .in_list(vec![lit(12i32), lit(99999999999i64)], false); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // DECIMAL(C3) in (INT64(12), INT32(12), DECIMAL(128,12,3)) - let expr_lt = cast(col("c3"), DataType::Decimal128(12, 3)).in_list( + let expr_lt = cast(col("c3"), LogicalType::Decimal128(12, 3)).in_list( vec![ lit_decimal(12, 12, 3), lit_decimal(12, 12, 3), @@ -740,19 +669,19 @@ mod tests { let schema = expr_test_schema(); // INT32(C1) IN (INT32(12),INT64(24)) -> INT32(C1) IN (INT32(12),INT32(24)) let expr_lt = - cast(col("c1"), DataType::Int64).in_list(vec![lit(12i64), lit(24i64)], false); + cast(col("c1"), LogicalType::Int64).in_list(vec![lit(12i64), lit(24i64)], false); let expected = col("c1").in_list(vec![lit(12i32), lit(24i32)], false); assert_eq!(optimize_test(expr_lt, &schema), expected); // INT32(C2) IN (INT64(NULL),INT64(24)) -> INT32(C1) IN (INT32(12),INT32(24)) let expr_lt = - cast(col("c2"), DataType::Int32).in_list(vec![null_i32(), lit(14i32)], false); + cast(col("c2"), LogicalType::Int32).in_list(vec![null_i32(), lit(14i32)], false); let expected = col("c2").in_list(vec![null_i64(), lit(14i64)], false); assert_eq!(optimize_test(expr_lt, &schema), expected); // decimal test case // c3 is decimal(18,2) - let expr_lt = cast(col("c3"), DataType::Decimal128(19, 3)).in_list( + let expr_lt = cast(col("c3"), LogicalType::Decimal128(19, 3)).in_list( vec![ lit_decimal(12000, 19, 3), lit_decimal(24000, 19, 3), @@ -773,7 +702,7 @@ mod tests { assert_eq!(optimize_test(expr_lt, &schema), expected); // cast(INT32(12), INT64) IN (.....) - let expr_lt = cast(lit(12i32), DataType::Int64) + let expr_lt = cast(lit(12i32), LogicalType::Int64) .in_list(vec![lit(13i64), lit(12i64)], false); let expected = lit(12i32).in_list(vec![lit(13i32), lit(12i32)], false); assert_eq!(optimize_test(expr_lt, &schema), expected); @@ -784,7 +713,7 @@ mod tests { let schema = expr_test_schema(); // c1 < INT64(16) -> c1 < cast(INT32(16)) // the 16 is within the range of MAX(int32) and MIN(int32), we can cast the 16 to int32(16) - let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(16i64)).alias("x"); + let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(16i64)).alias("x"); let expected = col("c1").lt(lit(16i32)).alias("x"); assert_eq!(optimize_test(expr_lt, &schema), expected); } @@ -794,9 +723,9 @@ mod tests { let schema = expr_test_schema(); // c1 < INT64(16) OR c1 > INT64(32) -> c1 < INT32(16) OR c1 > INT32(32) // the 16 and 32 are within the range of MAX(int32) and MIN(int32), we can cast them to int32 - let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(16i64)).or(cast( + let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(16i64)).or(cast( col("c1"), - DataType::Int64, + LogicalType::Int64, ) .gt(lit(32i64))); let expected = col("c1").lt(lit(16i32)).or(col("c1").gt(lit(32i32))); @@ -809,12 +738,12 @@ mod tests { // but the type of c6 is uint32 // the rewriter will not throw error and just return the original expr let schema = expr_test_schema(); - let expr_input = cast(col("c6"), DataType::Float64).eq(lit(0f64)); + let expr_input = cast(col("c6"), LogicalType::Float64).eq(lit(0f64)); assert_eq!(optimize_test(expr_input.clone(), &schema), expr_input); // inlist for unsupported data type let expr_input = - in_list(cast(col("c6"), DataType::Float64), vec![lit(0f64)], false); + in_list(cast(col("c6"), LogicalType::Float64), vec![lit(0f64)], false); assert_eq!(optimize_test(expr_input.clone(), &schema), expr_input); } @@ -841,17 +770,16 @@ mod tests { Arc::new( DFSchema::from_unqualified_fields( vec![ - Field::new("c1", DataType::Int32, false), - Field::new("c2", DataType::Int64, false), - Field::new("c3", DataType::Decimal128(18, 2), false), - Field::new("c4", DataType::Decimal128(38, 37), false), - Field::new("c5", DataType::Float32, false), - Field::new("c6", DataType::UInt32, false), - Field::new("ts_nano_none", timestamp_nano_none_type(), false), - Field::new("ts_nano_utf", timestamp_nano_utc_type(), false), - Field::new("str1", DataType::Utf8, false), - Field::new("largestr", DataType::LargeUtf8, false), - Field::new("tag", dictionary_tag_type(), false), + LogicalField::new("c1", LogicalType::Int32, false), + LogicalField::new("c2", LogicalType::Int64, false), + LogicalField::new("c3", LogicalType::Decimal128(18, 2), false), + LogicalField::new("c4", LogicalType::Decimal128(38, 37), false), + LogicalField::new("c5", LogicalType::Float32, false), + LogicalField::new("c6", LogicalType::UInt32, false), + LogicalField::new("ts_nano_none", timestamp_nano_none_type(), false), + LogicalField::new("ts_nano_utf", timestamp_nano_utc_type(), false), + LogicalField::new("str1", LogicalType::Utf8, false), + LogicalField::new("largestr", LogicalType::LargeUtf8, false), ] .into(), HashMap::new(), @@ -889,19 +817,14 @@ mod tests { lit(ScalarValue::Decimal128(None, precision, scale)) } - fn timestamp_nano_none_type() -> DataType { - DataType::Timestamp(TimeUnit::Nanosecond, None) + fn timestamp_nano_none_type() -> LogicalType { + LogicalType::Timestamp(TimeUnit::Nanosecond, None) } // this is the type that now() returns - fn timestamp_nano_utc_type() -> DataType { + fn timestamp_nano_utc_type() -> LogicalType { let utc = Some("+0:00".into()); - DataType::Timestamp(TimeUnit::Nanosecond, utc) - } - - // a dictonary type for storing string tags - fn dictionary_tag_type() -> DataType { - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + LogicalType::Timestamp(TimeUnit::Nanosecond, utc) } #[test] @@ -926,7 +849,7 @@ mod tests { for s2 in &scalars { let expected_value = ExpectedCast::Value(s2.clone()); - expect_cast(s1.clone(), s2.data_type(), expected_value); + expect_cast(s1.clone(), s2.data_type().into(), expected_value); } } } @@ -951,28 +874,28 @@ mod tests { for s2 in &scalars { let expected_value = ExpectedCast::Value(s2.clone()); - expect_cast(s1.clone(), s2.data_type(), expected_value); + expect_cast(s1.clone(), s2.data_type().into(), expected_value); } } let max_i32 = ScalarValue::Int32(Some(i32::MAX)); expect_cast( max_i32, - DataType::UInt64, + LogicalType::UInt64, ExpectedCast::Value(ScalarValue::UInt64(Some(i32::MAX as u64))), ); let min_i32 = ScalarValue::Int32(Some(i32::MIN)); expect_cast( min_i32, - DataType::Int64, + LogicalType::Int64, ExpectedCast::Value(ScalarValue::Int64(Some(i32::MIN as i64))), ); let max_i64 = ScalarValue::Int64(Some(i64::MAX)); expect_cast( max_i64, - DataType::UInt64, + LogicalType::UInt64, ExpectedCast::Value(ScalarValue::UInt64(Some(i64::MAX as u64))), ); } @@ -984,28 +907,28 @@ mod tests { let max_i64 = ScalarValue::Int64(Some(i64::MAX)); let max_u64 = ScalarValue::UInt64(Some(u64::MAX)); - expect_cast(max_i64.clone(), DataType::Int8, ExpectedCast::NoValue); + expect_cast(max_i64.clone(), LogicalType::Int8, ExpectedCast::NoValue); - expect_cast(max_i64.clone(), DataType::Int16, ExpectedCast::NoValue); + expect_cast(max_i64.clone(), LogicalType::Int16, ExpectedCast::NoValue); - expect_cast(max_i64, DataType::Int32, ExpectedCast::NoValue); + expect_cast(max_i64, LogicalType::Int32, ExpectedCast::NoValue); - expect_cast(max_u64, DataType::Int64, ExpectedCast::NoValue); + expect_cast(max_u64, LogicalType::Int64, ExpectedCast::NoValue); - expect_cast(min_i64, DataType::UInt64, ExpectedCast::NoValue); + expect_cast(min_i64, LogicalType::UInt64, ExpectedCast::NoValue); - expect_cast(min_i32, DataType::UInt64, ExpectedCast::NoValue); + expect_cast(min_i32, LogicalType::UInt64, ExpectedCast::NoValue); // decimal out of range expect_cast( ScalarValue::Decimal128(Some(99999999999999999999999999999999999900), 38, 0), - DataType::Int64, + LogicalType::Int64, ExpectedCast::NoValue, ); expect_cast( ScalarValue::Decimal128(Some(-9999999999999999999999999999999999), 37, 1), - DataType::Int64, + LogicalType::Int64, ExpectedCast::NoValue, ); } @@ -1014,19 +937,19 @@ mod tests { fn test_try_decimal_cast_in_range() { expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - DataType::Decimal128(3, 0), + LogicalType::Decimal128(3, 0), ExpectedCast::Value(ScalarValue::Decimal128(Some(123), 3, 0)), ); expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - DataType::Decimal128(8, 0), + LogicalType::Decimal128(8, 0), ExpectedCast::Value(ScalarValue::Decimal128(Some(123), 8, 0)), ); expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - DataType::Decimal128(8, 5), + LogicalType::Decimal128(8, 5), ExpectedCast::Value(ScalarValue::Decimal128(Some(12300000), 8, 5)), ); } @@ -1036,14 +959,14 @@ mod tests { // decimal would lose precision expect_cast( ScalarValue::Decimal128(Some(12345), 5, 2), - DataType::Decimal128(3, 0), + LogicalType::Decimal128(3, 0), ExpectedCast::NoValue, ); // decimal would lose precision expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - DataType::Decimal128(2, 0), + LogicalType::Decimal128(2, 0), ExpectedCast::NoValue, ); } @@ -1084,11 +1007,11 @@ mod tests { // so double check it here assert_eq!(lit_tz_none, lit_tz_utc); - // e.g. DataType::Timestamp(_, None) - let dt_tz_none = lit_tz_none.data_type(); + // e.g. LogicalType::Timestamp(_, None) + let dt_tz_none: LogicalType = lit_tz_none.data_type().into(); - // e.g. DataType::Timestamp(_, Some(utc)) - let dt_tz_utc = lit_tz_utc.data_type(); + // e.g. LogicalType::Timestamp(_, Some(utc)) + let dt_tz_utc: LogicalType = lit_tz_utc.data_type().into(); // None <--> None expect_cast( @@ -1121,7 +1044,7 @@ mod tests { // timestamp to int64 expect_cast( lit_tz_utc.clone(), - DataType::Int64, + LogicalType::Int64, ExpectedCast::Value(ScalarValue::Int64(Some(12345))), ); @@ -1142,7 +1065,7 @@ mod tests { // timestamp to string (not supported yet) expect_cast( lit_tz_utc.clone(), - DataType::LargeUtf8, + LogicalType::LargeUtf8, ExpectedCast::NoValue, ); } @@ -1153,7 +1076,7 @@ mod tests { // int64 to list expect_cast( ScalarValue::Int64(Some(12345)), - DataType::List(Arc::new(Field::new("f", DataType::Int32, true))), + LogicalType::new_list(LogicalType::Int32, true), ExpectedCast::NoValue, ); } @@ -1171,7 +1094,7 @@ mod tests { /// casting is consistent with the Arrow kernels fn expect_cast( literal: ScalarValue, - target_type: DataType, + target_type: LogicalType, expected_result: ExpectedCast, ) { let actual_value = try_cast_literal_to_type(&literal, &target_type); @@ -1199,7 +1122,7 @@ mod tests { .expect("Failed to convert to array of size"); let cast_array = cast_with_options( &literal_array, - &target_type, + &target_type.physical_type(), &CastOptions::default(), ) .expect("Expected to be cast array with arrow cast kernel"); @@ -1212,9 +1135,9 @@ mod tests { // Verify that for timestamp types the timezones are the same // (ScalarValue::cmp doesn't account for timezones); if let ( - DataType::Timestamp(left_unit, left_tz), - DataType::Timestamp(right_unit, right_tz), - ) = (actual_value.data_type(), expected_value.data_type()) + LogicalType::Timestamp(left_unit, left_tz), + LogicalType::Timestamp(right_unit, right_tz), + ) = (actual_value.data_type().into(), expected_value.data_type().into()) { assert_eq!(left_unit, right_unit); assert_eq!(left_tz, right_tz); @@ -1234,7 +1157,7 @@ mod tests { // same timestamp let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &DataType::Timestamp(TimeUnit::Nanosecond, None), + &LogicalType::Timestamp(TimeUnit::Nanosecond, None), ) .unwrap(); @@ -1246,7 +1169,7 @@ mod tests { // TimestampNanosecond to TimestampMicrosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &DataType::Timestamp(TimeUnit::Microsecond, None), + &LogicalType::Timestamp(TimeUnit::Microsecond, None), ) .unwrap(); @@ -1258,7 +1181,7 @@ mod tests { // TimestampNanosecond to TimestampMillisecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &DataType::Timestamp(TimeUnit::Millisecond, None), + &LogicalType::Timestamp(TimeUnit::Millisecond, None), ) .unwrap(); @@ -1267,7 +1190,7 @@ mod tests { // TimestampNanosecond to TimestampSecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &DataType::Timestamp(TimeUnit::Second, None), + &LogicalType::Timestamp(TimeUnit::Second, None), ) .unwrap(); @@ -1276,7 +1199,7 @@ mod tests { // TimestampMicrosecond to TimestampNanosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMicrosecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Nanosecond, None), + &LogicalType::Timestamp(TimeUnit::Nanosecond, None), ) .unwrap(); @@ -1288,7 +1211,7 @@ mod tests { // TimestampMicrosecond to TimestampMillisecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMicrosecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Millisecond, None), + &LogicalType::Timestamp(TimeUnit::Millisecond, None), ) .unwrap(); @@ -1297,7 +1220,7 @@ mod tests { // TimestampMicrosecond to TimestampSecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMicrosecond(Some(123456789), None), - &DataType::Timestamp(TimeUnit::Second, None), + &LogicalType::Timestamp(TimeUnit::Second, None), ) .unwrap(); assert_eq!(new_scalar, ScalarValue::TimestampSecond(Some(123), None)); @@ -1305,7 +1228,7 @@ mod tests { // TimestampMillisecond to TimestampNanosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMillisecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Nanosecond, None), + &LogicalType::Timestamp(TimeUnit::Nanosecond, None), ) .unwrap(); assert_eq!( @@ -1316,7 +1239,7 @@ mod tests { // TimestampMillisecond to TimestampMicrosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMillisecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Microsecond, None), + &LogicalType::Timestamp(TimeUnit::Microsecond, None), ) .unwrap(); assert_eq!( @@ -1326,7 +1249,7 @@ mod tests { // TimestampMillisecond to TimestampSecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMillisecond(Some(123456789), None), - &DataType::Timestamp(TimeUnit::Second, None), + &LogicalType::Timestamp(TimeUnit::Second, None), ) .unwrap(); assert_eq!(new_scalar, ScalarValue::TimestampSecond(Some(123456), None)); @@ -1334,7 +1257,7 @@ mod tests { // TimestampSecond to TimestampNanosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Nanosecond, None), + &LogicalType::Timestamp(TimeUnit::Nanosecond, None), ) .unwrap(); assert_eq!( @@ -1345,7 +1268,7 @@ mod tests { // TimestampSecond to TimestampMicrosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Microsecond, None), + &LogicalType::Timestamp(TimeUnit::Microsecond, None), ) .unwrap(); assert_eq!( @@ -1356,7 +1279,7 @@ mod tests { // TimestampSecond to TimestampMillisecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(123), None), - &DataType::Timestamp(TimeUnit::Millisecond, None), + &LogicalType::Timestamp(TimeUnit::Millisecond, None), ) .unwrap(); assert_eq!( @@ -1367,7 +1290,7 @@ mod tests { // overflow let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(i64::MAX), None), - &DataType::Timestamp(TimeUnit::Millisecond, None), + &LogicalType::Timestamp(TimeUnit::Millisecond, None), ) .unwrap(); assert_eq!(new_scalar, ScalarValue::TimestampMillisecond(None, None)); @@ -1384,33 +1307,8 @@ mod tests { for s2 in &scalars { let expected_value = ExpectedCast::Value(s2.clone()); - expect_cast(s1.clone(), s2.data_type(), expected_value); + expect_cast(s1.clone(), s2.data_type().into(), expected_value); } } } - #[test] - fn test_try_cast_to_dictionary_type() { - fn dictionary_type(t: DataType) -> DataType { - DataType::Dictionary(Box::new(DataType::Int32), Box::new(t)) - } - fn dictionary_value(value: ScalarValue) -> ScalarValue { - ScalarValue::Dictionary(Box::new(DataType::Int32), Box::new(value)) - } - let scalars = vec![ - ScalarValue::from("string"), - ScalarValue::LargeUtf8(Some("string".to_owned())), - ]; - for s in &scalars { - expect_cast( - s.clone(), - dictionary_type(s.data_type()), - ExpectedCast::Value(dictionary_value(s.clone())), - ); - expect_cast( - dictionary_value(s.clone()), - s.data_type(), - ExpectedCast::Value(s.clone()), - ) - } - } } diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index c501d5aaa4bf..33678a98da08 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -23,6 +23,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use datafusion_common::config::ConfigOptions; use datafusion_common::{plan_err, Result}; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::test::function_stub::sum_udaf; use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF}; use datafusion_functions_aggregate::average::avg_udaf; @@ -399,7 +400,7 @@ impl ContextProvider for MyContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index d5cd3c6f4af0..39305401288d 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -22,6 +22,7 @@ use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::Schema; use datafusion_common::{exec_err, Result}; +use datafusion_common::logical_type::extension::ExtensionType; use datafusion_expr::expr::Alias; use datafusion_expr::sort_properties::ExprProperties; use datafusion_expr::Expr; @@ -127,7 +128,7 @@ pub fn limited_convert_logical_expr_to_physical_expr( cast_expr.expr.as_ref(), schema, )?, - cast_expr.data_type.clone(), + cast_expr.data_type.physical_type(), None, ))), Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 53c790ff6b54..98d501c97854 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -429,7 +429,7 @@ pub fn in_list( let expr_data_type = expr.data_type(schema)?; for list_expr in list.iter() { let list_expr_data_type = list_expr.data_type(schema)?; - if !DFSchema::datatype_is_logically_equal(&expr_data_type, &list_expr_data_type) { + if !DFSchema::datatype_is_logically_equal(&expr_data_type.clone().into(), &list_expr_data_type.clone().into()) { return internal_err!( "The data type inlist should be same, the value type is {expr_data_type}, one of list expr type is {list_expr_data_type}" ); diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index aed2675e0447..f3ed3526b45a 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -160,7 +160,7 @@ pub fn negative( arg: Arc, input_schema: &Schema, ) -> Result> { - let data_type = arg.data_type(input_schema)?; + let data_type = arg.data_type(input_schema)?.into(); if is_null(&data_type) { Ok(arg) } else if !is_signed_numeric(&data_type) diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 8fe99cdca591..ae0f6e455532 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -24,9 +24,9 @@ use crate::{ }; use arrow::datatypes::Schema; -use datafusion_common::{ - exec_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, ToDFSchema, -}; +use datafusion_common::{exec_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, ToDFSchema}; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction}; use datafusion_expr::var_provider::is_system_variables; @@ -259,12 +259,12 @@ pub fn create_physical_expr( Expr::Cast(Cast { expr, data_type }) => expressions::cast( create_physical_expr(expr, input_dfschema, execution_props)?, input_schema, - data_type.clone(), + data_type.clone().physical_type(), ), Expr::TryCast(TryCast { expr, data_type }) => expressions::try_cast( create_physical_expr(expr, input_dfschema, execution_props)?, input_schema, - data_type.clone(), + data_type.clone().physical_type(), ), Expr::Not(expr) => { expressions::not(create_physical_expr(expr, input_dfschema, execution_props)?) @@ -359,7 +359,7 @@ where /// Convert a logical expression to a physical expression (without any simplification, etc) pub fn logical2physical(expr: &Expr, schema: &Schema) -> Arc { - let df_schema = schema.clone().to_dfschema().unwrap(); + let df_schema = LogicalSchema::from(schema.clone()).to_dfschema().unwrap(); let execution_props = ExecutionProps::new(); create_physical_expr(expr, &df_schema, &execution_props).unwrap() } @@ -378,7 +378,7 @@ mod tests { let expr = col("letter").eq(lit("A")); let schema = Schema::new(vec![Field::new("letter", DataType::Utf8, false)]); - let df_schema = DFSchema::try_from_qualified_schema("data", &schema)?; + let df_schema = DFSchema::try_from_qualified_schema("data", &schema.clone().into())?; let p = create_physical_expr(&expr, &df_schema, &ExecutionProps::new())?; let batch = RecordBatch::try_new( diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index 070034116fb4..67d20837afdc 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -420,11 +420,10 @@ impl<'a> ColOpLit<'a> { #[cfg(test)] mod test { use std::sync::OnceLock; - + use arrow_schema::{DataType, Field, Schema, SchemaRef}; use super::*; use crate::planner::logical2physical; - use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_expr::expr_fn::*; use datafusion_expr::{lit, Expr}; @@ -835,7 +834,7 @@ mod test { fn test_analyze(expr: Expr, expected: Vec) { println!("Begin analyze of {expr}"); let schema = schema(); - let physical_expr = logical2physical(&expr, &schema); + let physical_expr = logical2physical(&expr, &schema.as_ref().clone().into()); let actual = LiteralGuarantee::analyze(&physical_expr); assert_eq!( diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index de9fede9ee86..6f54ac19626f 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -42,6 +42,8 @@ use datafusion_common::{ Column, ColumnStatistics, Constraint, Constraints, DFSchema, DFSchemaRef, DataFusionError, JoinSide, ScalarValue, Statistics, TableReference, }; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; #[derive(Debug)] pub enum Error { @@ -158,10 +160,10 @@ impl TryFrom<&protobuf::DfSchema> for DFSchema { df_schema: &protobuf::DfSchema, ) -> datafusion_common::Result { let df_fields = df_schema.columns.clone(); - let qualifiers_and_fields: Vec<(Option, Arc)> = df_fields + let qualifiers_and_fields: Vec<(Option, Arc)> = df_fields .iter() .map(|df_field| { - let field: Field = df_field.field.as_ref().required("field")?; + let field: LogicalField = df_field.field.as_ref().required("field")?; Ok(( df_field .qualifier @@ -190,6 +192,16 @@ impl TryFrom for DFSchemaRef { } } +impl TryFrom<&protobuf::ArrowType> for LogicalType { + type Error = Error; + + fn try_from( + arrow_type: &protobuf::ArrowType, + ) -> datafusion_common::Result { + DataType::try_from(arrow_type).map(|t| t.into()) + } +} + impl TryFrom<&protobuf::ArrowType> for DataType { type Error = Error; @@ -332,6 +344,14 @@ impl TryFrom<&protobuf::Field> for Field { } } + +impl TryFrom<&protobuf::Field> for LogicalField { + type Error = Error; + fn try_from(field: &protobuf::Field) -> Result { + Field::try_from(field).map(|t| t.into()) + } +} + impl TryFrom<&protobuf::Schema> for Schema { type Error = Error; diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 877043f66809..d343554e6fe6 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -23,10 +23,7 @@ use crate::protobuf_common::{ }; use arrow::array::{ArrayRef, RecordBatch}; use arrow::csv::WriterBuilder; -use arrow::datatypes::{ - DataType, Field, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, - SchemaRef, TimeUnit, UnionMode, -}; +use arrow::datatypes::{DataType, Field, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionMode}; use arrow::ipc::writer::{DictionaryTracker, IpcDataGenerator}; use datafusion_common::{ config::{ @@ -39,6 +36,8 @@ use datafusion_common::{ Column, ColumnStatistics, Constraint, Constraints, DFSchema, DFSchemaRef, DataFusionError, JoinSide, ScalarValue, Statistics, }; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; #[derive(Debug)] pub enum Error { @@ -112,6 +111,17 @@ impl TryFrom<&DataType> for protobuf::ArrowType { } } +impl TryFrom<&LogicalType> for protobuf::ArrowType { + type Error = Error; + + fn try_from(val: &LogicalType) -> Result { + let arrow_type_enum: ArrowTypeEnum = (&val.physical_type()).try_into()?; + Ok(Self { + arrow_type_enum: Some(arrow_type_enum), + }) + } +} + impl TryFrom<&DataType> for protobuf::arrow_type::ArrowTypeEnum { type Error = Error; @@ -262,8 +272,9 @@ impl TryFrom<&DFSchema> for protobuf::DfSchema { let columns = s .iter() .map(|(qualifier, field)| { + let field: Field = field.as_ref().clone().into(); Ok(protobuf::DfField { - field: Some(field.as_ref().try_into()?), + field: Some((&field).try_into()?), qualifier: qualifier.map(|r| protobuf::ColumnRelation { relation: r.to_string(), }), diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 664cd7e11555..be21a5aa2178 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -65,7 +65,9 @@ use datafusion_expr::{ use prost::bytes::BufMut; use prost::Message; - +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; +use datafusion_proto_common::ArrowType; use self::to_proto::serialize_expr; pub mod file_formats; @@ -836,10 +838,10 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Prepare(prepare) => { let input: LogicalPlan = into_logical_plan!(prepare.input, ctx, extension_codec)?; - let data_types: Vec = prepare + let data_types: Vec = prepare .data_types .iter() - .map(DataType::try_from) + .map(|t| DataType::try_from(t).map(|t| t.into())) .collect::>()?; LogicalPlanBuilder::from(input) .prepare(prepare.name.clone(), data_types)? @@ -1560,8 +1562,8 @@ impl AsLogicalPlan for LogicalPlanNode { name: name.clone(), data_types: data_types .iter() - .map(|t| t.try_into()) - .collect::, _>>()?, + .map(|t| (&t.physical_type()).try_into()) + .collect::, _>>()?, input: Some(Box::new(input)), }, ))), diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index fe3da3d05854..906f1762d1e1 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -26,6 +26,8 @@ use arrow::datatypes::{ DataType, Field, Fields, Int32Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, }; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::LogicalType; use datafusion::datasource::file_format::arrow::ArrowFormatFactory; use datafusion::datasource::file_format::csv::CsvFormatFactory; use datafusion::datasource::file_format::format_as_file_type; @@ -590,7 +592,7 @@ async fn roundtrip_expr_api() -> Result<()> { // list of expressions to round trip let expr_list = vec![ - encode(col("a").cast_to(&DataType::Utf8, &schema)?, lit("hex")), + encode(col("a").cast_to(&LogicalType::Utf8, &schema)?, lit("hex")), decode(lit("1234"), lit("hex")), array_to_string(make_array(vec![lit(1), lit(2), lit(3)]), lit(",")), array_dims(make_array(vec![lit(1), lit(2), lit(3)])), @@ -698,7 +700,7 @@ async fn roundtrip_expr_api() -> Result<()> { bit_and(lit(2)), bit_or(lit(2)), bit_xor(lit(2)), - string_agg(col("a").cast_to(&DataType::Utf8, &schema)?, lit("|")), + string_agg(col("a").cast_to(&LogicalType::Utf8, &schema)?, lit("|")), bool_and(lit(true)), bool_or(lit(true)), ]; @@ -1559,11 +1561,11 @@ fn roundtrip_schema() { fn roundtrip_dfschema() { let dfschema = DFSchema::new_with_metadata( vec![ - (None, Arc::new(Field::new("a", DataType::Int64, false))), + (None, Arc::new(LogicalField::new("a", LogicalType::Int64, false))), ( Some("t".into()), Arc::new( - Field::new("b", DataType::Decimal128(15, 2), true).with_metadata( + LogicalField::new("b", LogicalType::Decimal128(15, 2), true).with_metadata( HashMap::from([(String::from("k1"), String::from("v1"))]), ), ), @@ -1692,7 +1694,7 @@ fn roundtrip_null_literal() { #[test] fn roundtrip_cast() { - let test_expr = Expr::Cast(Cast::new(Box::new(lit(1.0_f32)), DataType::Boolean)); + let test_expr = Expr::Cast(Cast::new(Box::new(lit(1.0_f32)), LogicalType::Boolean)); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); @@ -1701,13 +1703,13 @@ fn roundtrip_cast() { #[test] fn roundtrip_try_cast() { let test_expr = - Expr::TryCast(TryCast::new(Box::new(lit(1.0_f32)), DataType::Boolean)); + Expr::TryCast(TryCast::new(Box::new(lit(1.0_f32)), LogicalType::Boolean)); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); let test_expr = - Expr::TryCast(TryCast::new(Box::new(lit("not a bool")), DataType::Boolean)); + Expr::TryCast(TryCast::new(Box::new(lit("not a bool")), LogicalType::Boolean)); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); diff --git a/datafusion/sql/examples/sql.rs b/datafusion/sql/examples/sql.rs index aee4cf5a38ed..159bf46f97c6 100644 --- a/datafusion/sql/examples/sql.rs +++ b/datafusion/sql/examples/sql.rs @@ -30,6 +30,7 @@ use datafusion_sql::{ TableReference, }; use std::{collections::HashMap, sync::Arc}; +use datafusion_common::logical_type::LogicalType; fn main() { let sql = "SELECT \ @@ -132,7 +133,7 @@ impl ContextProvider for MyContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index ea460cb3efc2..a1e4719d7c70 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -16,7 +16,6 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use arrow_schema::DataType; use datafusion_common::{ internal_datafusion_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, Dependency, Result, @@ -36,6 +35,7 @@ use sqlparser::ast::{ }; use std::str::FromStr; use strum::IntoEnumIterator; +use datafusion_common::logical_type::LogicalType; /// Suggest a valid function based on an invalid input function name pub fn suggest_valid_function( @@ -474,11 +474,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn check_unnest_arg(arg: &Expr, schema: &DFSchema) -> Result<()> { // Check argument type, array types are supported match arg.get_type(schema)? { - DataType::List(_) - | DataType::LargeList(_) - | DataType::FixedSizeList(_, _) - | DataType::Struct(_) => Ok(()), - DataType::Null => { + LogicalType::List(_) + | LogicalType::LargeList(_) + | LogicalType::FixedSizeList(_, _) + | LogicalType::Struct(_) => Ok(()), + LogicalType::Null => { not_impl_err!("unnest() does not support null yet") } _ => { diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index d297b2e4df5b..87419d97351e 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -16,13 +16,13 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use arrow_schema::Field; use datafusion_common::{ internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, ScalarValue, TableReference, }; use datafusion_expr::{expr::ScalarFunction, lit, Case, Expr}; use sqlparser::ast::{Expr as SQLExpr, Ident}; +use datafusion_common::logical_type::field::LogicalField; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(super) fn sql_identifier_to_expr( @@ -280,7 +280,7 @@ fn search_dfschema<'ids, 'schema>( ids: &'ids [String], schema: &'schema DFSchema, ) -> Option<( - &'schema Field, + &'schema LogicalField, Option<&'schema TableReference>, &'ids [String], )> { diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index a8af37ee6a37..04ecdce3f164 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::DataType; use arrow_schema::TimeUnit; use datafusion_common::utils::list_ndims; use sqlparser::ast::{CastKind, Expr as SQLExpr, Subscript, TrimWhereField, Value}; @@ -24,6 +23,8 @@ use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, }; +use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::InList; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ @@ -113,8 +114,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if op == Operator::StringConcat { let left_type = left.get_type(schema)?; let right_type = right.get_type(schema)?; - let left_list_ndims = list_ndims(&left_type); - let right_list_ndims = list_ndims(&right_type); + let left_list_ndims = list_ndims(&left_type.physical_type()); + let right_list_ndims = list_ndims(&right_type.physical_type()); // We determine the target function to rewrite based on the list n-dimension, the check is not exact but sufficient. // The exact validity check is handled in the actual function, so even if there is 3d list appended with 1d list, it is also fine to rewrite. @@ -351,12 +352,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // numeric constants are treated as seconds (rather as nanoseconds) // to align with postgres / duckdb semantics let expr = match &dt { - DataType::Timestamp(TimeUnit::Nanosecond, tz) - if expr.get_type(schema)? == DataType::Int64 => + LogicalType::Timestamp(TimeUnit::Nanosecond, tz) + if expr.get_type(schema)? == LogicalType::Int64 => { Expr::Cast(Cast::new( Box::new(expr), - DataType::Timestamp(TimeUnit::Second, tz.clone()), + LogicalType::Timestamp(TimeUnit::Second, tz.clone()), )) } _ => expr, @@ -635,7 +636,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { )?), match *time_zone { SQLExpr::Value(Value::SingleQuotedString(s)) => { - DataType::Timestamp(TimeUnit::Nanosecond, Some(s.into())) + LogicalType::Timestamp(TimeUnit::Nanosecond, Some(s.into())) } _ => { return not_impl_err!( @@ -813,7 +814,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let pattern = self.sql_expr_to_logical_expr(pattern, schema, planner_context)?; let pattern_type = pattern.get_type(schema)?; - if pattern_type != DataType::Utf8 && pattern_type != DataType::Null { + if pattern_type != LogicalType::Utf8 && pattern_type != LogicalType::Null { return plan_err!("Invalid pattern in LIKE expression"); } let escape_char = if let Some(char) = escape_char { @@ -844,7 +845,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let pattern = self.sql_expr_to_logical_expr(pattern, schema, planner_context)?; let pattern_type = pattern.get_type(schema)?; - if pattern_type != DataType::Utf8 && pattern_type != DataType::Null { + if pattern_type != LogicalType::Utf8 && pattern_type != LogicalType::Null { return plan_err!("Invalid pattern in SIMILAR TO expression"); } let escape_char = if let Some(char) = escape_char { @@ -1023,6 +1024,7 @@ mod tests { use std::sync::Arc; use arrow::datatypes::{Field, Schema}; + use arrow_schema::DataType; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -1074,7 +1076,7 @@ mod tests { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index fa95fc2e051d..a9a2698397a9 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -18,7 +18,6 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; use arrow::datatypes::DECIMAL128_MAX_PRECISION; -use arrow_schema::DataType; use datafusion_common::{ not_impl_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -28,12 +27,13 @@ use log::debug; use sqlparser::ast::{BinaryOperator, Expr as SQLExpr, Interval, Value}; use sqlparser::parser::ParserError::ParserError; use std::borrow::Cow; +use datafusion_common::logical_type::LogicalType; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn parse_value( &self, value: Value, - param_data_types: &[DataType], + param_data_types: &[LogicalType], ) -> Result { match value { Value::Number(n, _) => self.parse_sql_number(&n, false), @@ -96,7 +96,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// number 1, 2, ... etc. For example, `$1` is the first placeholder; $2 is the second one and so on. fn create_placeholder_expr( param: String, - param_data_types: &[DataType], + param_data_types: &[LogicalType], ) -> Result { // Parse the placeholder as a number because it is the only support from sqlparser and postgres let index = param[1..].parse::(); diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 00f221200624..c0dd69a2d828 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -37,6 +37,10 @@ use datafusion_common::{ not_impl_err, plan_err, unqualified_field_not_found, DFSchema, DataFusionError, Result, }; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::logical_plan::{LogicalPlan, LogicalPlanBuilder}; use datafusion_expr::utils::find_column_exprs; use datafusion_expr::TableSource; @@ -83,7 +87,7 @@ pub trait ContextProvider { /// Getter for a UDWF fn get_window_meta(&self, name: &str) -> Option>; /// Getter for system/user-defined variable type - fn get_variable_type(&self, variable_names: &[String]) -> Option; + fn get_variable_type(&self, variable_names: &[String]) -> Option; /// Get configuration options fn options(&self) -> &ConfigOptions; @@ -158,7 +162,7 @@ impl IdentNormalizer { pub struct PlannerContext { /// Data types for numbered parameters ($1, $2, etc), if supplied /// in `PREPARE` statement - prepare_param_data_types: Arc>, + prepare_param_data_types: Arc>, /// Map of CTE name to logical plan of the WITH clause. /// Use `Arc` to allow cheap cloning ctes: HashMap>, @@ -185,7 +189,7 @@ impl PlannerContext { /// Update the PlannerContext with provided prepare_param_data_types pub fn with_prepare_param_data_types( mut self, - prepare_param_data_types: Vec, + prepare_param_data_types: Vec, ) -> Self { self.prepare_param_data_types = prepare_param_data_types.into(); self @@ -207,7 +211,7 @@ impl PlannerContext { } /// Return the types of parameters (`$1`, `$2`, etc) if known - pub fn prepare_param_data_types(&self) -> &[DataType] { + pub fn prepare_param_data_types(&self) -> &[LogicalType] { &self.prepare_param_data_types } @@ -259,7 +263,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - pub fn build_schema(&self, columns: Vec) -> Result { + pub fn build_schema(&self, columns: Vec) -> Result { let mut fields = Vec::with_capacity(columns.len()); for column in columns { @@ -268,14 +272,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .options .iter() .any(|x| x.option == ColumnOption::NotNull); - fields.push(Field::new( + fields.push(LogicalField::new( self.normalizer.normalize(column.name), data_type, !not_nullable, )); } - Ok(Schema::new(fields)) + Ok(LogicalSchema::new(fields)) } /// Returns a vector of (column_name, default_expr) pairs @@ -379,13 +383,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) } - pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { + pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { SQLDataType::Array(ArrayElemTypeDef::AngleBracket(inner_sql_type)) | SQLDataType::Array(ArrayElemTypeDef::SquareBracket(inner_sql_type, _)) => { // Arrays may be multi-dimensional. let inner_data_type = self.convert_data_type(inner_sql_type)?; - Ok(DataType::new_list(inner_data_type, true)) + Ok(LogicalType::new_list(inner_data_type, true).into()) } SQLDataType::Array(ArrayElemTypeDef::None) => { not_impl_err!("Arrays with unspecified type is not supported") @@ -394,31 +398,31 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - fn convert_simple_data_type(&self, sql_type: &SQLDataType) -> Result { + fn convert_simple_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { - SQLDataType::Boolean | SQLDataType::Bool => Ok(DataType::Boolean), - SQLDataType::TinyInt(_) => Ok(DataType::Int8), - SQLDataType::SmallInt(_) | SQLDataType::Int2(_) => Ok(DataType::Int16), - SQLDataType::Int(_) | SQLDataType::Integer(_) | SQLDataType::Int4(_) => Ok(DataType::Int32), - SQLDataType::BigInt(_) | SQLDataType::Int8(_) => Ok(DataType::Int64), - SQLDataType::UnsignedTinyInt(_) => Ok(DataType::UInt8), - SQLDataType::UnsignedSmallInt(_) | SQLDataType::UnsignedInt2(_) => Ok(DataType::UInt16), + SQLDataType::Boolean | SQLDataType::Bool => Ok(LogicalType::Boolean), + SQLDataType::TinyInt(_) => Ok(LogicalType::Int8), + SQLDataType::SmallInt(_) | SQLDataType::Int2(_) => Ok(LogicalType::Int16), + SQLDataType::Int(_) | SQLDataType::Integer(_) | SQLDataType::Int4(_) => Ok(LogicalType::Int32), + SQLDataType::BigInt(_) | SQLDataType::Int8(_) => Ok(LogicalType::Int64), + SQLDataType::UnsignedTinyInt(_) => Ok(LogicalType::UInt8), + SQLDataType::UnsignedSmallInt(_) | SQLDataType::UnsignedInt2(_) => Ok(LogicalType::UInt16), SQLDataType::UnsignedInt(_) | SQLDataType::UnsignedInteger(_) | SQLDataType::UnsignedInt4(_) => { - Ok(DataType::UInt32) + Ok(LogicalType::UInt32) } SQLDataType::Varchar(length) => { match (length, self.options.support_varchar_with_length) { (Some(_), false) => plan_err!("does not support Varchar with length, please set `support_varchar_with_length` to be true"), - _ => Ok(DataType::Utf8), + _ => Ok(LogicalType::Utf8), } } - SQLDataType::UnsignedBigInt(_) | SQLDataType::UnsignedInt8(_) => Ok(DataType::UInt64), - SQLDataType::Float(_) => Ok(DataType::Float32), - SQLDataType::Real | SQLDataType::Float4 => Ok(DataType::Float32), - SQLDataType::Double | SQLDataType::DoublePrecision | SQLDataType::Float8 => Ok(DataType::Float64), + SQLDataType::UnsignedBigInt(_) | SQLDataType::UnsignedInt8(_) => Ok(LogicalType::UInt64), + SQLDataType::Float(_) => Ok(LogicalType::Float32), + SQLDataType::Real | SQLDataType::Float4 => Ok(LogicalType::Float32), + SQLDataType::Double | SQLDataType::DoublePrecision | SQLDataType::Float8 => Ok(LogicalType::Float64), SQLDataType::Char(_) | SQLDataType::Text - | SQLDataType::String(_) => Ok(DataType::Utf8), + | SQLDataType::String(_) => Ok(LogicalType::Utf8), SQLDataType::Timestamp(None, tz_info) => { let tz = if matches!(tz_info, TimezoneInfo::Tz) || matches!(tz_info, TimezoneInfo::WithTimeZone) @@ -431,14 +435,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Timestamp Without Time zone None }; - Ok(DataType::Timestamp(TimeUnit::Nanosecond, tz.map(Into::into))) + Ok(LogicalType::Timestamp(TimeUnit::Nanosecond, tz.map(Into::into))) } - SQLDataType::Date => Ok(DataType::Date32), + SQLDataType::Date => Ok(LogicalType::Date32), SQLDataType::Time(None, tz_info) => { if matches!(tz_info, TimezoneInfo::None) || matches!(tz_info, TimezoneInfo::WithoutTimeZone) { - Ok(DataType::Time64(TimeUnit::Nanosecond)) + Ok(LogicalType::Time64(TimeUnit::Nanosecond)) } else { // We dont support TIMETZ and TIME WITH TIME ZONE for now not_impl_err!( @@ -457,8 +461,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; make_decimal_type(precision, scale) } - SQLDataType::Bytea => Ok(DataType::Binary), - SQLDataType::Interval => Ok(DataType::Interval(IntervalUnit::MonthDayNano)), + SQLDataType::Bytea => Ok(LogicalType::Binary), + SQLDataType::Interval => Ok(LogicalType::Interval(IntervalUnit::MonthDayNano)), SQLDataType::Struct(fields) => { let fields = fields .iter() @@ -469,14 +473,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Some(ident) => ident.clone(), None => Ident::new(format!("c{idx}")) }; - Ok(Arc::new(Field::new( + Ok(Arc::new(LogicalField::new( self.normalizer.normalize(field_name), data_type, true, ))) }) .collect::>>()?; - Ok(DataType::Struct(Fields::from(fields))) + Ok(LogicalType::Struct(LogicalFields::from(fields))) } // Explicitly list all other types so that if sqlparser // adds/changes the `SQLDataType` the compiler will tell us on upgrade diff --git a/datafusion/sql/src/relation/join.rs b/datafusion/sql/src/relation/join.rs index ee2e35b550f6..8978cb10b768 100644 --- a/datafusion/sql/src/relation/join.rs +++ b/datafusion/sql/src/relation/join.rs @@ -122,7 +122,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .build() } JoinConstraint::Natural => { - let left_cols: HashSet<&String> = + let left_cols: HashSet<&str> = left.schema().fields().iter().map(|f| f.name()).collect(); let keys: Vec = right .schema() diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 6cdb2f959cd8..bb2f32853245 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -29,7 +29,6 @@ use crate::planner::{ }; use crate::utils::normalize_ident; -use arrow_schema::{DataType, Fields}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ exec_err, not_impl_err, plan_datafusion_err, plan_err, schema_err, @@ -60,6 +59,9 @@ use sqlparser::ast::{ TableConstraint, TableFactor, TableWithJoins, TransactionMode, UnaryOperator, Value, }; use sqlparser::parser::ParserError::ParserError; +use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchema; fn ident_to_string(ident: &Ident) -> String { normalize_ident(ident.to_owned()) @@ -453,7 +455,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { statement, } => { // Convert parser data types to DataFusion data types - let data_types: Vec = data_types + let data_types: Vec = data_types .into_iter() .map(|t| self.convert_data_type(&t)) .collect::>()?; @@ -1223,7 +1225,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_ref = self.object_name_to_table_reference(table_name.clone())?; let table_source = self.context_provider.get_table_source(table_ref.clone())?; - let schema = (*table_source.schema()).clone(); + let schema: LogicalSchema = (*table_source.schema()).clone().into(); let schema = DFSchema::try_from(schema)?; let scan = LogicalPlanBuilder::scan( object_name_to_string(&table_name), @@ -1276,7 +1278,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let table_source = self.context_provider.get_table_source(table_name.clone())?; let table_schema = Arc::new(DFSchema::try_from_qualified_schema( table_name.clone(), - &table_source.schema(), + &table_source.schema().as_ref().clone().into(), )?); // Overwrite with assignment expressions @@ -1381,7 +1383,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_name = self.object_name_to_table_reference(table_name)?; let table_source = self.context_provider.get_table_source(table_name.clone())?; - let arrow_schema = (*table_source.schema()).clone(); + let arrow_schema: LogicalSchema = (*table_source.schema()).clone().into(); let table_schema = DFSchema::try_from(arrow_schema)?; // Get insert fields and target table's value indices @@ -1419,7 +1421,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(table_schema.field(column_index).clone()) }) .collect::>>()?; - (Fields::from(fields), value_indices) + (LogicalFields::from(fields), value_indices) }; // infer types for Values clause... other types should be resolvable the regular way diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index ad898de5987a..33186312ac0c 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -27,7 +27,6 @@ use arrow_array::types::{ TimestampNanosecondType, TimestampSecondType, }; use arrow_array::{Date32Array, Date64Array, PrimitiveArray}; -use arrow_schema::DataType; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ self, Expr as AstExpr, Function, FunctionArg, Ident, Interval, TimezoneInfo, @@ -38,6 +37,7 @@ use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, Column, Result, ScalarValue, }; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::{ expr::{Alias, Exists, InList, ScalarFunction, Sort, WindowFunction}, Between, BinaryExpr, Case, Cast, Expr, GroupingSet, Like, Operator, TryCast, @@ -957,85 +957,67 @@ impl Unparser<'_> { } } - fn arrow_dtype_to_ast_dtype(&self, data_type: &DataType) -> Result { + fn arrow_dtype_to_ast_dtype(&self, data_type: &LogicalType) -> Result { match data_type { - DataType::Null => { + LogicalType::Null => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Boolean => Ok(ast::DataType::Bool), - DataType::Int8 => Ok(ast::DataType::TinyInt(None)), - DataType::Int16 => Ok(ast::DataType::SmallInt(None)), - DataType::Int32 => Ok(ast::DataType::Integer(None)), - DataType::Int64 => Ok(ast::DataType::BigInt(None)), - DataType::UInt8 => Ok(ast::DataType::UnsignedTinyInt(None)), - DataType::UInt16 => Ok(ast::DataType::UnsignedSmallInt(None)), - DataType::UInt32 => Ok(ast::DataType::UnsignedInteger(None)), - DataType::UInt64 => Ok(ast::DataType::UnsignedBigInt(None)), - DataType::Float16 => { + LogicalType::Boolean => Ok(ast::DataType::Bool), + LogicalType::Int8 => Ok(ast::DataType::TinyInt(None)), + LogicalType::Int16 => Ok(ast::DataType::SmallInt(None)), + LogicalType::Int32 => Ok(ast::DataType::Integer(None)), + LogicalType::Int64 => Ok(ast::DataType::BigInt(None)), + LogicalType::UInt8 => Ok(ast::DataType::UnsignedTinyInt(None)), + LogicalType::UInt16 => Ok(ast::DataType::UnsignedSmallInt(None)), + LogicalType::UInt32 => Ok(ast::DataType::UnsignedInteger(None)), + LogicalType::UInt64 => Ok(ast::DataType::UnsignedBigInt(None)), + LogicalType::Float16 => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Float32 => Ok(ast::DataType::Float(None)), - DataType::Float64 => Ok(ast::DataType::Double), - DataType::Timestamp(_, _) => { + LogicalType::Float32 => Ok(ast::DataType::Float(None)), + LogicalType::Float64 => Ok(ast::DataType::Double), + LogicalType::Timestamp(_, _) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Date32 => Ok(ast::DataType::Date), - DataType::Date64 => Ok(ast::DataType::Datetime(None)), - DataType::Time32(_) => { + LogicalType::Date32 => Ok(ast::DataType::Date), + LogicalType::Date64 => Ok(ast::DataType::Datetime(None)), + LogicalType::Time32(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Time64(_) => { + LogicalType::Time64(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Duration(_) => { + LogicalType::Duration(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Interval(_) => { + LogicalType::Interval(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Binary => { + LogicalType::Binary => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::FixedSizeBinary(_) => { + LogicalType::FixedSizeBinary(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::LargeBinary => { + LogicalType::LargeBinary => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::BinaryView => { + LogicalType::Utf8 => Ok(ast::DataType::Varchar(None)), + LogicalType::LargeUtf8 => Ok(ast::DataType::Text), + LogicalType::List(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::Utf8 => Ok(ast::DataType::Varchar(None)), - DataType::LargeUtf8 => Ok(ast::DataType::Text), - DataType::Utf8View => { + LogicalType::FixedSizeList(_, _) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::List(_) => { + LogicalType::LargeList(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::FixedSizeList(_, _) => { + LogicalType::Struct(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::LargeList(_) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - DataType::ListView(_) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - DataType::LargeListView(_) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - DataType::Struct(_) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - DataType::Union(_, _) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - DataType::Dictionary(_, _) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - DataType::Decimal128(precision, scale) - | DataType::Decimal256(precision, scale) => { + LogicalType::Decimal128(precision, scale) + | LogicalType::Decimal256(precision, scale) => { let mut new_precision = *precision as u64; let mut new_scale = *scale as u64; if *scale < 0 { @@ -1047,10 +1029,10 @@ impl Unparser<'_> { ast::ExactNumberInfo::PrecisionAndScale(new_precision, new_scale), )) } - DataType::Map(_, _) => { + LogicalType::Map(_, _) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - DataType::RunEndEncoded(_, _) => { + LogicalType::Extension(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } } @@ -1063,8 +1045,7 @@ mod tests { use std::{any::Any, sync::Arc, vec}; use arrow::datatypes::{Field, Schema}; - use arrow_schema::DataType::Int8; - + use arrow_schema::DataType; use datafusion_common::TableReference; use datafusion_expr::{ case, col, cube, exists, grouping_set, interval_datetime_lit, @@ -1153,14 +1134,14 @@ mod tests { ( Expr::Cast(Cast { expr: Box::new(col("a")), - data_type: DataType::Date64, + data_type: LogicalType::Date64, }), r#"CAST(a AS DATETIME)"#, ), ( Expr::Cast(Cast { expr: Box::new(col("a")), - data_type: DataType::UInt32, + data_type: LogicalType::UInt32, }), r#"CAST(a AS INTEGER UNSIGNED)"#, ), @@ -1386,27 +1367,27 @@ mod tests { r#"NOT EXISTS (SELECT t.a FROM t WHERE (t.a = 1))"#, ), ( - try_cast(col("a"), DataType::Date64), + try_cast(col("a"), LogicalType::Date64), r#"TRY_CAST(a AS DATETIME)"#, ), ( - try_cast(col("a"), DataType::UInt32), + try_cast(col("a"), LogicalType::UInt32), r#"TRY_CAST(a AS INTEGER UNSIGNED)"#, ), ( - Expr::ScalarVariable(Int8, vec![String::from("@a")]), + Expr::ScalarVariable(LogicalType::Int8, vec![String::from("@a")]), r#"@a"#, ), ( Expr::ScalarVariable( - Int8, + LogicalType::Int8, vec![String::from("@root"), String::from("foo")], ), r#"@root.foo"#, ), (col("x").eq(placeholder("$1")), r#"(x = $1)"#), ( - out_ref_col(DataType::Int32, "t.a").gt(lit(1)), + out_ref_col(LogicalType::Int32, "t.a").gt(lit(1)), r#"(t.a > 1)"#, ), ( @@ -1481,7 +1462,7 @@ mod tests { ( Expr::Cast(Cast { expr: Box::new(col("a")), - data_type: DataType::Decimal128(10, -2), + data_type: LogicalType::Decimal128(10, -2), }), r#"CAST(a AS DECIMAL(12,0))"#, ), diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index bc27d25cf216..fbe8cd099f2a 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -20,7 +20,7 @@ use std::collections::HashMap; use arrow_schema::{ - DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, + DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, }; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ @@ -31,6 +31,7 @@ use datafusion_expr::expr::{Alias, GroupingSet, Unnest, WindowFunction}; use datafusion_expr::utils::{expr_as_column_expr, find_column_exprs}; use datafusion_expr::{expr_vec_fmt, Expr, ExprSchemable, LogicalPlan}; use sqlparser::ast::Ident; +use datafusion_common::logical_type::LogicalType; /// Make a best-effort attempt at resolving all columns in the expression tree pub(crate) fn resolve_columns(expr: &Expr, plan: &LogicalPlan) -> Result { @@ -226,7 +227,7 @@ pub fn window_expr_common_partition_keys(window_exprs: &[Expr]) -> Result<&[Expr pub(crate) fn make_decimal_type( precision: Option, scale: Option, -) -> Result { +) -> Result { // postgres like behavior let (precision, scale) = match (precision, scale) { (Some(p), Some(s)) => (p as u8, s as i8), @@ -247,9 +248,9 @@ pub(crate) fn make_decimal_type( } else if precision > DECIMAL128_MAX_PRECISION && precision <= DECIMAL256_MAX_PRECISION { - Ok(DataType::Decimal256(precision, scale)) + Ok(LogicalType::Decimal256(precision, scale)) } else { - Ok(DataType::Decimal128(precision, scale)) + Ok(LogicalType::Decimal128(precision, scale)) } } @@ -316,7 +317,7 @@ pub(crate) fn recursive_transform_unnest( } = original_expr.transform_up(|expr: Expr| { if let Expr::Unnest(Unnest { expr: ref arg }) = expr { let (data_type, _) = arg.data_type_and_nullable(input.schema())?; - if let DataType::Struct(_) = data_type { + if let LogicalType::Struct(_) = data_type { return internal_err!("unnest on struct can ony be applied at the root level of select expression"); } let transformed_exprs = transform(&expr, arg)?; diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index 374403d853f9..eefa9e2b3471 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -30,7 +30,6 @@ use datafusion_sql::unparser::{expr_to_sql, plan_to_sql, Unparser}; use sqlparser::dialect::{Dialect, GenericDialect, MySqlDialect}; use sqlparser::parser::Parser; - use crate::common::MockContextProvider; #[test] diff --git a/datafusion/sql/tests/common/mod.rs b/datafusion/sql/tests/common/mod.rs index f5caaefb3ea0..d7124d5e0180 100644 --- a/datafusion/sql/tests/common/mod.rs +++ b/datafusion/sql/tests/common/mod.rs @@ -25,6 +25,7 @@ use arrow_schema::*; use datafusion_common::config::ConfigOptions; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{plan_err, GetExt, Result, TableReference}; +use datafusion_common::logical_type::LogicalType; use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource, WindowUDF}; use datafusion_sql::planner::ContextProvider; @@ -202,7 +203,7 @@ impl ContextProvider for MockContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _: &[String]) -> Option { + fn get_variable_type(&self, _: &[String]) -> Option { unimplemented!() } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index e72a439b323b..d89419c9edcb 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -43,6 +43,7 @@ use datafusion_functions_aggregate::{ }; use rstest::rstest; use sqlparser::dialect::{Dialect, GenericDialect, HiveDialect, MySqlDialect}; +use datafusion_common::logical_type::LogicalType; mod cases; mod common; @@ -3662,8 +3663,8 @@ fn test_prepare_statement_should_infer_types() { let plan = logical_plan(sql).unwrap(); let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(DataType::Int32)), - ("$2".to_string(), Some(DataType::Int64)), + ("$1".to_string(), Some(LogicalType::Int32)), + ("$2".to_string(), Some(LogicalType::Int64)), ]); assert_eq!(actual_types, expected_types); } @@ -3676,7 +3677,7 @@ fn test_non_prepare_statement_should_infer_types() { let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ // constant 1 is inferred to be int64 - ("$1".to_string(), Some(DataType::Int64)), + ("$1".to_string(), Some(LogicalType::Int64)), ]); assert_eq!(actual_types, expected_types); } @@ -3851,7 +3852,7 @@ Projection: person.id, orders.order_id let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(DataType::Int32))]); + let expected_types = HashMap::from([("$1".to_string(), Some(LogicalType::Int32))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3883,7 +3884,7 @@ Projection: person.id, person.age let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(DataType::Int32))]); + let expected_types = HashMap::from([("$1".to_string(), Some(LogicalType::Int32))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3915,8 +3916,8 @@ Projection: person.id, person.age let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(DataType::Int32)), - ("$2".to_string(), Some(DataType::Int32)), + ("$1".to_string(), Some(LogicalType::Int32)), + ("$2".to_string(), Some(LogicalType::Int32)), ]); assert_eq!(actual_types, expected_types); @@ -3954,7 +3955,7 @@ Projection: person.id, person.age let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(DataType::UInt32))]); + let expected_types = HashMap::from([("$1".to_string(), Some(LogicalType::UInt32))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3992,8 +3993,8 @@ Dml: op=[Update] table=[person] let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(DataType::Int32)), - ("$2".to_string(), Some(DataType::UInt32)), + ("$1".to_string(), Some(LogicalType::Int32)), + ("$2".to_string(), Some(LogicalType::UInt32)), ]); assert_eq!(actual_types, expected_types); @@ -4027,9 +4028,9 @@ fn test_prepare_statement_insert_infer() { let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(DataType::UInt32)), - ("$2".to_string(), Some(DataType::Utf8)), - ("$3".to_string(), Some(DataType::Utf8)), + ("$1".to_string(), Some(LogicalType::UInt32)), + ("$2".to_string(), Some(LogicalType::Utf8)), + ("$3".to_string(), Some(LogicalType::Utf8)), ]); assert_eq!(actual_types, expected_types); diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 520b6b53b32d..3dfcb34d1a3e 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. -use arrow::datatypes::Fields; use arrow::util::display::ArrayFormatter; use arrow::{array, array::ArrayRef, datatypes::DataType, record_batch::RecordBatch}; use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; +use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::LogicalType; use datafusion_common::DataFusionError; use std::path::PathBuf; use std::sync::OnceLock; @@ -243,31 +244,31 @@ pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { } /// Converts columns to a result as expected by sqllogicteset. -pub(crate) fn convert_schema_to_types(columns: &Fields) -> Vec { +pub(crate) fn convert_schema_to_types(columns: &LogicalFields) -> Vec { columns .iter() .map(|f| f.data_type()) .map(|data_type| match data_type { - DataType::Boolean => DFColumnType::Boolean, - DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 => DFColumnType::Integer, - DataType::Float16 - | DataType::Float32 - | DataType::Float64 - | DataType::Decimal128(_, _) - | DataType::Decimal256(_, _) => DFColumnType::Float, - DataType::Utf8 | DataType::LargeUtf8 => DFColumnType::Text, - DataType::Date32 - | DataType::Date64 - | DataType::Time32(_) - | DataType::Time64(_) => DFColumnType::DateTime, - DataType::Timestamp(_, _) => DFColumnType::Timestamp, + LogicalType::Boolean => DFColumnType::Boolean, + LogicalType::Int8 + | LogicalType::Int16 + | LogicalType::Int32 + | LogicalType::Int64 + | LogicalType::UInt8 + | LogicalType::UInt16 + | LogicalType::UInt32 + | LogicalType::UInt64 => DFColumnType::Integer, + LogicalType::Float16 + | LogicalType::Float32 + | LogicalType::Float64 + | LogicalType::Decimal128(_, _) + | LogicalType::Decimal256(_, _) => DFColumnType::Float, + LogicalType::Utf8 | LogicalType::LargeUtf8 => DFColumnType::Text, + LogicalType::Date32 + | LogicalType::Date64 + | LogicalType::Time32(_) + | LogicalType::Time64(_) => DFColumnType::DateTime, + LogicalType::Timestamp(_, _) => DFColumnType::Timestamp, _ => DFColumnType::Another, }) .collect() diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 9bc842a12af4..56f659f1d870 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -17,8 +17,13 @@ use async_recursion::async_recursion; use datafusion::arrow::datatypes::{ - DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, + Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, }; +use datafusion::common::logical_type::extension::ExtensionType; +use datafusion::common::logical_type::field::{LogicalField, LogicalFieldRef}; +use datafusion::common::logical_type::fields::LogicalFields; +use datafusion::common::logical_type::schema::LogicalSchema; +use datafusion::common::logical_type::LogicalType; use datafusion::common::{ not_impl_err, substrait_datafusion_err, substrait_err, DFSchema, DFSchemaRef, }; @@ -349,12 +354,12 @@ fn make_renamed_schema( dfs_names: &Vec, ) -> Result { fn rename_inner_fields( - dtype: &DataType, + dtype: &LogicalType, dfs_names: &Vec, name_idx: &mut usize, - ) -> Result { + ) -> Result { match dtype { - DataType::Struct(fields) => { + LogicalType::Struct(fields) => { let fields = fields .iter() .map(|f| { @@ -364,16 +369,16 @@ fn make_renamed_schema( )) }) .collect::>()?; - Ok(DataType::Struct(fields)) + Ok(LogicalType::Struct(fields)) } - DataType::List(inner) => Ok(DataType::List(FieldRef::new( + LogicalType::List(inner) => Ok(LogicalType::List(LogicalFieldRef::new( (**inner).to_owned().with_data_type(rename_inner_fields( inner.data_type(), dfs_names, name_idx, )?), ))), - DataType::LargeList(inner) => Ok(DataType::LargeList(FieldRef::new( + LogicalType::LargeList(inner) => Ok(LogicalType::LargeList(LogicalFieldRef::new( (**inner).to_owned().with_data_type(rename_inner_fields( inner.data_type(), dfs_names, @@ -386,7 +391,7 @@ fn make_renamed_schema( let mut name_idx = 0; - let (qualifiers, fields): (_, Vec) = schema + let (qualifiers, fields): (_, Vec) = schema .iter() .map(|(q, f)| { let name = next_struct_field_name(0, dfs_names, &mut name_idx)?; @@ -415,7 +420,7 @@ fn make_renamed_schema( DFSchema::from_field_specific_qualified_schema( qualifiers, - &Arc::new(Schema::new(fields)), + &Arc::new(LogicalSchema::new(fields)), ) } @@ -863,7 +868,7 @@ pub async fn from_substrait_rel( } /// (Re)qualify the sides of a join if needed, i.e. if the columns from one side would otherwise -/// conflict with the columns from the other. +/// conflict with the columns from the other. /// Substrait doesn't currently allow specifying aliases, neither for columns nor for tables. For /// Substrait the names don't matter since it only refers to columns by indices, however DataFusion /// requires columns to be uniquely identifiable, in some places (see e.g. DFSchema::check_names). @@ -1348,7 +1353,7 @@ pub async fn from_substrait_rex( } } -pub(crate) fn from_substrait_type_without_names(dt: &Type) -> Result { +pub(crate) fn from_substrait_type_without_names(dt: &Type) -> Result { from_substrait_type(dt, &[], &mut 0) } @@ -1356,77 +1361,77 @@ fn from_substrait_type( dt: &Type, dfs_names: &[String], name_idx: &mut usize, -) -> Result { +) -> Result { match &dt.kind { Some(s_kind) => match s_kind { - r#type::Kind::Bool(_) => Ok(DataType::Boolean), + r#type::Kind::Bool(_) => Ok(LogicalType::Boolean), r#type::Kind::I8(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int8), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt8), + DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int8), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt8), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::I16(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int16), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt16), + DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int16), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt16), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::I32(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int32), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt32), + DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int32), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt32), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::I64(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int64), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt64), + DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int64), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt64), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, - r#type::Kind::Fp32(_) => Ok(DataType::Float32), - r#type::Kind::Fp64(_) => Ok(DataType::Float64), + r#type::Kind::Fp32(_) => Ok(LogicalType::Float32), + r#type::Kind::Fp64(_) => Ok(LogicalType::Float64), r#type::Kind::Timestamp(ts) => match ts.type_variation_reference { TIMESTAMP_SECOND_TYPE_VARIATION_REF => { - Ok(DataType::Timestamp(TimeUnit::Second, None)) + Ok(LogicalType::Timestamp(TimeUnit::Second, None)) } TIMESTAMP_MILLI_TYPE_VARIATION_REF => { - Ok(DataType::Timestamp(TimeUnit::Millisecond, None)) + Ok(LogicalType::Timestamp(TimeUnit::Millisecond, None)) } TIMESTAMP_MICRO_TYPE_VARIATION_REF => { - Ok(DataType::Timestamp(TimeUnit::Microsecond, None)) + Ok(LogicalType::Timestamp(TimeUnit::Microsecond, None)) } TIMESTAMP_NANO_TYPE_VARIATION_REF => { - Ok(DataType::Timestamp(TimeUnit::Nanosecond, None)) + Ok(LogicalType::Timestamp(TimeUnit::Nanosecond, None)) } v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::Date(date) => match date.type_variation_reference { - DATE_32_TYPE_VARIATION_REF => Ok(DataType::Date32), - DATE_64_TYPE_VARIATION_REF => Ok(DataType::Date64), + DATE_32_TYPE_VARIATION_REF => Ok(LogicalType::Date32), + DATE_64_TYPE_VARIATION_REF => Ok(LogicalType::Date64), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::Binary(binary) => match binary.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Binary), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeBinary), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::Binary), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::LargeBinary), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::FixedBinary(fixed) => { - Ok(DataType::FixedSizeBinary(fixed.length)) + Ok(LogicalType::FixedSizeBinary(fixed.length)) } r#type::Kind::String(string) => match string.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Utf8), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeUtf8), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::Utf8), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::LargeUtf8), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), @@ -1435,15 +1440,15 @@ fn from_substrait_type( let inner_type = list.r#type.as_ref().ok_or_else(|| { substrait_datafusion_err!("List type must have inner type") })?; - let field = Arc::new(Field::new_list_field( + let field = Arc::new(LogicalField::new_list_field( from_substrait_type(inner_type, dfs_names, name_idx)?, - // We ignore Substrait's nullability here to match to_substrait_literal + // We ignore Substrait's nullability here to match to_substrait_literal // which always creates nullable lists true, )); match list.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::List(field)), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeList(field)), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::List(field)), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::LargeList(field)), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" )?, @@ -1456,19 +1461,19 @@ fn from_substrait_type( let value_type = map.value.as_ref().ok_or_else(|| { substrait_datafusion_err!("Map type must have value type") })?; - let key_field = Arc::new(Field::new( + let key_field = Arc::new(LogicalField::new( "key", from_substrait_type(key_type, dfs_names, name_idx)?, false, )); - let value_field = Arc::new(Field::new( + let value_field = Arc::new(LogicalField::new( "value", from_substrait_type(value_type, dfs_names, name_idx)?, true, )); match map.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => { - Ok(DataType::Map(Arc::new(Field::new_struct( + Ok(LogicalType::Map(Arc::new(LogicalField::new_struct( "entries", [key_field, value_field], false, // The inner map field is always non-nullable (Arrow #1697), @@ -1481,10 +1486,10 @@ fn from_substrait_type( } r#type::Kind::Decimal(d) => match d.type_variation_reference { DECIMAL_128_TYPE_VARIATION_REF => { - Ok(DataType::Decimal128(d.precision as u8, d.scale as i8)) + Ok(LogicalType::Decimal128(d.precision as u8, d.scale as i8)) } DECIMAL_256_TYPE_VARIATION_REF => { - Ok(DataType::Decimal256(d.precision as u8, d.scale as i8)) + Ok(LogicalType::Decimal256(d.precision as u8, d.scale as i8)) } v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" @@ -1493,13 +1498,13 @@ fn from_substrait_type( r#type::Kind::UserDefined(u) => { match u.type_reference { INTERVAL_YEAR_MONTH_TYPE_REF => { - Ok(DataType::Interval(IntervalUnit::YearMonth)) + Ok(LogicalType::Interval(IntervalUnit::YearMonth)) } INTERVAL_DAY_TIME_TYPE_REF => { - Ok(DataType::Interval(IntervalUnit::DayTime)) + Ok(LogicalType::Interval(IntervalUnit::DayTime)) } INTERVAL_MONTH_DAY_NANO_TYPE_REF => { - Ok(DataType::Interval(IntervalUnit::MonthDayNano)) + Ok(LogicalType::Interval(IntervalUnit::MonthDayNano)) } _ => not_impl_err!( "Unsupported Substrait user defined type with ref {} and variation {}", @@ -1508,11 +1513,11 @@ fn from_substrait_type( ), } }, - r#type::Kind::Struct(s) => Ok(DataType::Struct(from_substrait_struct_type( + r#type::Kind::Struct(s) => Ok(LogicalType::Struct(from_substrait_struct_type( s, dfs_names, name_idx, )?)), - r#type::Kind::Varchar(_) => Ok(DataType::Utf8), - r#type::Kind::FixedChar(_) => Ok(DataType::Utf8), + r#type::Kind::Varchar(_) => Ok(LogicalType::Utf8), + r#type::Kind::FixedChar(_) => Ok(LogicalType::Utf8), _ => not_impl_err!("Unsupported Substrait type: {s_kind:?}"), }, _ => not_impl_err!("`None` Substrait kind is not supported"), @@ -1523,10 +1528,10 @@ fn from_substrait_struct_type( s: &r#type::Struct, dfs_names: &[String], name_idx: &mut usize, -) -> Result { +) -> Result { let mut fields = vec![]; for (i, f) in s.types.iter().enumerate() { - let field = Field::new( + let field = LogicalField::new( next_struct_field_name(i, dfs_names, name_idx)?, from_substrait_type(f, dfs_names, name_idx)?, is_substrait_type_nullable(f)?, @@ -1778,10 +1783,10 @@ fn from_substrait_literal( )?; match lit.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => { - ScalarValue::List(ScalarValue::new_list_nullable(&[], &element_type)) + ScalarValue::List(ScalarValue::new_list_nullable(&[], &element_type.physical_type())) } LARGE_CONTAINER_TYPE_VARIATION_REF => ScalarValue::LargeList( - ScalarValue::new_large_list(&[], &element_type), + ScalarValue::new_large_list(&[], &element_type.physical_type()), ), others => { return substrait_err!("Unknown type variation reference {others}"); @@ -1958,7 +1963,7 @@ fn from_substrait_null( d.scale as i8, )), r#type::Kind::List(l) => { - let field = Field::new_list_field( + let field = LogicalField::new_list_field( from_substrait_type( l.r#type.clone().unwrap().as_ref(), dfs_names, @@ -1968,10 +1973,10 @@ fn from_substrait_null( ); match l.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(ScalarValue::List( - Arc::new(GenericListArray::new_null(field.into(), 1)), + Arc::new(GenericListArray::new_null(FieldRef::new(field.into()), 1)), )), LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(ScalarValue::LargeList( - Arc::new(GenericListArray::new_null(field.into(), 1)), + Arc::new(GenericListArray::new_null(FieldRef::new(field.into()), 1)), )), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {kind:?}" @@ -1979,7 +1984,7 @@ fn from_substrait_null( } } r#type::Kind::Struct(s) => { - let fields = from_substrait_struct_type(s, dfs_names, name_idx)?; + let fields: Fields = from_substrait_struct_type(s, dfs_names, name_idx)?.into(); Ok(ScalarStructBuilder::new_null(fields)) } _ => not_impl_err!("Unsupported Substrait type for null: {kind:?}"), diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 302f38606bfb..07c40f9a9fbd 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -26,14 +26,26 @@ use datafusion::logical_expr::{ CrossJoin, Distinct, Like, Partitioning, WindowFrameUnits, }; use datafusion::{ - arrow::datatypes::{DataType, TimeUnit}, + arrow::datatypes::TimeUnit, error::{DataFusionError, Result}, logical_expr::{WindowFrame, WindowFrameBound}, prelude::{JoinType, SessionContext}, scalar::ScalarValue, }; +use crate::variation_const::{ + DATE_32_TYPE_VARIATION_REF, DATE_64_TYPE_VARIATION_REF, + DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, + DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, + INTERVAL_DAY_TIME_TYPE_REF, INTERVAL_DAY_TIME_TYPE_URL, + INTERVAL_MONTH_DAY_NANO_TYPE_REF, INTERVAL_MONTH_DAY_NANO_TYPE_URL, + INTERVAL_YEAR_MONTH_TYPE_REF, INTERVAL_YEAR_MONTH_TYPE_URL, + LARGE_CONTAINER_TYPE_VARIATION_REF, TIMESTAMP_MICRO_TYPE_VARIATION_REF, + TIMESTAMP_MILLI_TYPE_VARIATION_REF, TIMESTAMP_NANO_TYPE_VARIATION_REF, + TIMESTAMP_SECOND_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, +}; use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; +use datafusion::common::logical_type::LogicalType; use datafusion::common::{ exec_err, internal_err, not_impl_err, plan_err, substrait_datafusion_err, }; @@ -91,18 +103,6 @@ use substrait::{ version, }; -use crate::variation_const::{ - DATE_32_TYPE_VARIATION_REF, DATE_64_TYPE_VARIATION_REF, - DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, - DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, - INTERVAL_DAY_TIME_TYPE_REF, INTERVAL_DAY_TIME_TYPE_URL, - INTERVAL_MONTH_DAY_NANO_TYPE_REF, INTERVAL_MONTH_DAY_NANO_TYPE_URL, - INTERVAL_YEAR_MONTH_TYPE_REF, INTERVAL_YEAR_MONTH_TYPE_URL, - LARGE_CONTAINER_TYPE_VARIATION_REF, TIMESTAMP_MICRO_TYPE_VARIATION_REF, - TIMESTAMP_MILLI_TYPE_VARIATION_REF, TIMESTAMP_NANO_TYPE_VARIATION_REF, - TIMESTAMP_SECOND_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, -}; - /// Convert DataFusion LogicalPlan to Substrait Plan pub fn to_substrait_plan(plan: &LogicalPlan, ctx: &SessionContext) -> Result> { // Parse relation nodes @@ -586,9 +586,9 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { // Substrait wants a list of all field names, including nested fields from structs, // also from within e.g. lists and maps. However, it does not want the list and map field names // themselves - only proper structs fields are considered to have useful names. - fn names_dfs(dtype: &DataType) -> Result> { + fn names_dfs(dtype: &LogicalType) -> Result> { match dtype { - DataType::Struct(fields) => { + LogicalType::Struct(fields) => { let mut names = Vec::new(); for field in fields { names.push(field.name().to_string()); @@ -596,10 +596,10 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { } Ok(names) } - DataType::List(l) => names_dfs(l.data_type()), - DataType::LargeList(l) => names_dfs(l.data_type()), - DataType::Map(m, _) => match m.data_type() { - DataType::Struct(key_and_value) if key_and_value.len() == 2 => { + LogicalType::List(l) => names_dfs(l.data_type()), + LogicalType::LargeList(l) => names_dfs(l.data_type()), + LogicalType::Map(m, _) => match m.data_type() { + LogicalType::Struct(key_and_value) if key_and_value.len() == 2 => { let key_names = names_dfs(key_and_value.first().unwrap().data_type())?; let value_names = @@ -1433,83 +1433,83 @@ pub fn to_substrait_rex( } } -fn to_substrait_type(dt: &DataType, nullable: bool) -> Result { +fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result { let nullability = if nullable { r#type::Nullability::Nullable as i32 } else { r#type::Nullability::Required as i32 }; match dt { - DataType::Null => internal_err!("Null cast is not valid"), - DataType::Boolean => Ok(substrait::proto::Type { + LogicalType::Null => internal_err!("Null cast is not valid"), + LogicalType::Boolean => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Bool(r#type::Boolean { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int8 => Ok(substrait::proto::Type { + LogicalType::Int8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I8(r#type::I8 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt8 => Ok(substrait::proto::Type { + LogicalType::UInt8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I8(r#type::I8 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int16 => Ok(substrait::proto::Type { + LogicalType::Int16 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I16(r#type::I16 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt16 => Ok(substrait::proto::Type { + LogicalType::UInt16 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I16(r#type::I16 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int32 => Ok(substrait::proto::Type { + LogicalType::Int32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I32(r#type::I32 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt32 => Ok(substrait::proto::Type { + LogicalType::UInt32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I32(r#type::I32 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int64 => Ok(substrait::proto::Type { + LogicalType::Int64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I64(r#type::I64 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt64 => Ok(substrait::proto::Type { + LogicalType::UInt64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I64(r#type::I64 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), // Float16 is not supported in Substrait - DataType::Float32 => Ok(substrait::proto::Type { + LogicalType::Float32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Fp32(r#type::Fp32 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::Float64 => Ok(substrait::proto::Type { + LogicalType::Float64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Fp64(r#type::Fp64 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), // Timezone is ignored. - DataType::Timestamp(unit, _) => { + LogicalType::Timestamp(unit, _) => { let type_variation_reference = match unit { TimeUnit::Second => TIMESTAMP_SECOND_TYPE_VARIATION_REF, TimeUnit::Millisecond => TIMESTAMP_MILLI_TYPE_VARIATION_REF, @@ -1523,19 +1523,19 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result Ok(substrait::proto::Type { + LogicalType::Date32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Date(r#type::Date { type_variation_reference: DATE_32_TYPE_VARIATION_REF, nullability, })), }), - DataType::Date64 => Ok(substrait::proto::Type { + LogicalType::Date64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Date(r#type::Date { type_variation_reference: DATE_64_TYPE_VARIATION_REF, nullability, })), }), - DataType::Interval(interval_unit) => { + LogicalType::Interval(interval_unit) => { // define two type parameters for convenience let i32_param = Parameter { parameter: Some(parameter::Parameter::DataType(substrait::proto::Type { @@ -1578,38 +1578,38 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result Ok(substrait::proto::Type { + LogicalType::Binary => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Binary(r#type::Binary { type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - DataType::FixedSizeBinary(length) => Ok(substrait::proto::Type { + LogicalType::FixedSizeBinary(length) => Ok(substrait::proto::Type { kind: Some(r#type::Kind::FixedBinary(r#type::FixedBinary { length: *length, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::LargeBinary => Ok(substrait::proto::Type { + LogicalType::LargeBinary => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Binary(r#type::Binary { type_variation_reference: LARGE_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Utf8 => Ok(substrait::proto::Type { + LogicalType::Utf8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::String(r#type::String { type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - DataType::LargeUtf8 => Ok(substrait::proto::Type { + LogicalType::LargeUtf8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::String(r#type::String { type_variation_reference: LARGE_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - DataType::List(inner) => { + LogicalType::List(inner) => { let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { @@ -1619,7 +1619,7 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result { + LogicalType::LargeList(inner) => { let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { @@ -1629,8 +1629,8 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result match inner.data_type() { - DataType::Struct(key_and_value) if key_and_value.len() == 2 => { + LogicalType::Map(inner, _) => match inner.data_type() { + LogicalType::Struct(key_and_value) if key_and_value.len() == 2 => { let key_type = to_substrait_type( key_and_value[0].data_type(), key_and_value[0].is_nullable(), @@ -1650,7 +1650,7 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result plan_err!("Map fields must contain a Struct with exactly 2 fields"), }, - DataType::Struct(fields) => { + LogicalType::Struct(fields) => { let field_types = fields .iter() .map(|field| to_substrait_type(field.data_type(), field.is_nullable())) @@ -1663,7 +1663,7 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result Ok(substrait::proto::Type { + LogicalType::Decimal128(p, s) => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Decimal(r#type::Decimal { type_variation_reference: DECIMAL_128_TYPE_VARIATION_REF, nullability, @@ -1671,7 +1671,7 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result Ok(substrait::proto::Type { + LogicalType::Decimal256(p, s) => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Decimal(r#type::Decimal { type_variation_reference: DECIMAL_256_TYPE_VARIATION_REF, nullability, @@ -1897,7 +1897,7 @@ fn to_substrait_literal(value: &ScalarValue) -> Result { nullable: true, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, literal_type: Some(LiteralType::Null(to_substrait_type( - &value.data_type(), + &value.data_type().into(), true, )?)), }); @@ -2097,7 +2097,7 @@ fn convert_array_to_literal_list( .collect::>>()?; if values.is_empty() { - let et = match to_substrait_type(array.data_type(), array.is_nullable())? { + let et = match to_substrait_type(&array.data_type().to_owned().into(), array.is_nullable())? { substrait::proto::Type { kind: Some(r#type::Kind::List(lt)), } => lt.as_ref().to_owned(), @@ -2223,7 +2223,7 @@ mod test { from_substrait_literal_without_names, from_substrait_type_without_names, }; use datafusion::arrow::array::GenericListArray; - use datafusion::arrow::datatypes::Field; + use datafusion::arrow::datatypes::{DataType, Field}; use datafusion::common::scalar::ScalarStructBuilder; use super::*; @@ -2376,9 +2376,10 @@ mod test { // As DataFusion doesn't consider nullability as a property of the type, but field, // it doesn't matter if we set nullability to true or false here. - let substrait = to_substrait_type(&dt, true)?; + let lt = dt.into(); + let substrait = to_substrait_type(<, true)?; let roundtrip_dt = from_substrait_type_without_names(&substrait)?; - assert_eq!(dt, roundtrip_dt); + assert_eq!(lt, roundtrip_dt); Ok(()) } } From 996bbdce9c470b58c3befdc3cbc1d8b35ee8ad01 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Tue, 2 Jul 2024 14:42:42 +0200 Subject: [PATCH 02/10] Add type relation --- .../examples/advanced_parquet_index.rs | 10 +- .../examples/custom_datasource.rs | 15 +- .../examples/dataframe_subquery.rs | 6 +- datafusion-examples/examples/expr_api.rs | 11 +- .../examples/function_factory.rs | 7 +- datafusion-examples/examples/logical_type.rs | 3 + datafusion-examples/examples/parquet_index.rs | 8 +- datafusion-examples/examples/rewrite_expr.rs | 9 +- datafusion-examples/examples/simple_udtf.rs | 8 +- datafusion/common/src/dfschema.rs | 29 +- .../common/src/logical_type/extension.rs | 289 ------------------ datafusion/common/src/logical_type/field.rs | 90 +++--- datafusion/common/src/logical_type/fields.rs | 58 +++- datafusion/common/src/logical_type/mod.rs | 226 ++++++++------ .../common/src/logical_type/registry.rs | 51 ---- datafusion/common/src/logical_type/schema.rs | 57 +++- .../common/src/logical_type/signature.rs | 135 ++++++++ .../common/src/logical_type/type_signature.rs | 42 --- datafusion/common/src/param_value.rs | 4 +- datafusion/common/src/scalar/mod.rs | 115 +------ .../core/src/catalog/information_schema.rs | 2 +- datafusion/core/src/dataframe/mod.rs | 36 +-- .../core/src/datasource/cte_worktable.rs | 5 +- .../src/datasource/default_table_source.rs | 4 +- datafusion/core/src/datasource/empty.rs | 5 +- .../core/src/datasource/listing/helpers.rs | 2 +- .../core/src/datasource/listing/table.rs | 26 +- datafusion/core/src/datasource/memory.rs | 13 +- .../physical_plan/parquet/row_filter.rs | 5 +- .../physical_plan/parquet/row_groups.rs | 10 +- datafusion/core/src/datasource/provider.rs | 4 +- datafusion/core/src/datasource/stream.rs | 5 +- datafusion/core/src/datasource/streaming.rs | 6 +- datafusion/core/src/datasource/view.rs | 5 +- .../core/src/execution/session_state.rs | 11 +- .../core/src/physical_optimizer/pruning.rs | 47 ++- datafusion/core/src/physical_planner.rs | 3 +- datafusion/core/src/test/variable.rs | 13 +- datafusion/core/src/test_util/mod.rs | 6 +- .../core/tests/custom_sources_cases/mod.rs | 6 +- .../provider_filter_pushdown.rs | 5 +- .../tests/custom_sources_cases/statistics.rs | 5 +- .../tests/dataframe/dataframe_functions.rs | 7 +- datafusion/core/tests/dataframe/mod.rs | 14 +- .../core/tests/expr_api/parse_sql_expr.rs | 8 +- .../core/tests/expr_api/simplification.rs | 37 +-- datafusion/core/tests/memory_limit/mod.rs | 7 +- .../core/tests/optimizer_integration.rs | 8 +- datafusion/core/tests/sql/create_drop.rs | 8 +- .../user_defined_scalar_functions.rs | 21 +- .../user_defined_table_functions.rs | 7 +- .../expr/src/conditional_expressions.rs | 11 +- datafusion/expr/src/expr.rs | 26 +- datafusion/expr/src/expr_fn.rs | 14 +- datafusion/expr/src/expr_rewriter/mod.rs | 6 +- datafusion/expr/src/expr_rewriter/order_by.rs | 9 +- datafusion/expr/src/expr_schema.rs | 73 ++--- datafusion/expr/src/logical_plan/builder.rs | 73 +++-- datafusion/expr/src/logical_plan/ddl.rs | 6 +- datafusion/expr/src/logical_plan/dml.rs | 4 +- datafusion/expr/src/logical_plan/plan.rs | 17 +- datafusion/expr/src/simplify.rs | 9 +- datafusion/expr/src/table_source.rs | 4 +- datafusion/expr/src/type_coercion/mod.rs | 51 ++-- datafusion/expr/src/utils.rs | 66 ++-- datafusion/expr/src/var_provider.rs | 4 +- datafusion/functions/src/core/getfield.rs | 12 +- datafusion/functions/src/core/named_struct.rs | 4 +- datafusion/functions/src/math/log.rs | 10 +- datafusion/functions/src/math/power.rs | 6 +- .../src/analyzer/count_wildcard_rule.rs | 4 +- .../src/analyzer/inline_table_scan.rs | 14 +- .../optimizer/src/analyzer/type_coercion.rs | 232 +++++++------- .../optimizer/src/common_subexpr_eliminate.rs | 17 +- .../src/decorrelate_predicate_subquery.rs | 78 ++--- .../optimizer/src/eliminate_outer_join.rs | 6 +- .../src/extract_equijoin_predicate.rs | 6 +- .../optimizer/src/optimize_projections/mod.rs | 23 +- datafusion/optimizer/src/push_down_filter.rs | 13 +- .../optimizer/src/scalar_subquery_to_join.rs | 34 +-- .../simplify_expressions/expr_simplifier.rs | 57 ++-- .../simplify_expressions/simplify_exprs.rs | 7 +- .../src/unwrap_cast_in_comparison.rs | 241 +++++++-------- .../optimizer/tests/optimizer_integration.rs | 9 +- datafusion/physical-expr-common/src/utils.rs | 5 +- datafusion/physical-expr/src/planner.rs | 6 +- datafusion/proto-common/src/from_proto/mod.rs | 4 +- datafusion/proto-common/src/to_proto/mod.rs | 9 +- .../proto/src/logical_plan/from_proto.rs | 5 +- datafusion/proto/src/logical_plan/mod.rs | 9 +- .../tests/cases/roundtrip_logical_plan.rs | 15 +- datafusion/sql/examples/sql.rs | 4 +- datafusion/sql/src/expr/function.rs | 9 +- datafusion/sql/src/expr/mod.rs | 24 +- datafusion/sql/src/expr/value.rs | 6 +- datafusion/sql/src/planner.rs | 56 ++-- datafusion/sql/src/statement.rs | 8 +- datafusion/sql/src/unparser/expr.rs | 43 +-- datafusion/sql/src/utils.rs | 13 +- datafusion/sql/tests/common/mod.rs | 9 +- datafusion/sql/tests/sql_integration.rs | 28 +- .../engines/datafusion_engine/normalize.rs | 10 +- datafusion/sqllogictest/src/test_context.rs | 5 +- .../substrait/src/logical_plan/consumer.rs | 120 ++++---- .../substrait/src/logical_plan/producer.rs | 86 +++--- 105 files changed, 1446 insertions(+), 1688 deletions(-) create mode 100644 datafusion-examples/examples/logical_type.rs delete mode 100644 datafusion/common/src/logical_type/extension.rs delete mode 100644 datafusion/common/src/logical_type/registry.rs create mode 100644 datafusion/common/src/logical_type/signature.rs delete mode 100644 datafusion/common/src/logical_type/type_signature.rs diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 9bf71e52c3de..a366ddc03c66 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -60,6 +60,7 @@ use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use tempfile::TempDir; use url::Url; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// This example demonstrates using low level DataFusion APIs to read only /// certain row groups and ranges from parquet files, based on external @@ -299,8 +300,9 @@ impl IndexTableProvider { // In this example, we use the PruningPredicate's literal guarantees to // analyze the predicate. In a real system, using // `PruningPredicate::prune` would likely be easier to do. + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); let pruning_predicate = - PruningPredicate::try_new(Arc::clone(predicate), self.schema().clone())?; + PruningPredicate::try_new(Arc::clone(predicate), schema)?; // The PruningPredicate's guarantees must all be satisfied in order for // the predicate to possibly evaluate to true. @@ -453,8 +455,8 @@ impl TableProvider for IndexTableProvider { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.indexed_file.schema) + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.indexed_file.schema.as_ref().clone().into()) } fn table_type(&self) -> TableType { @@ -482,7 +484,7 @@ impl TableProvider for IndexTableProvider { .with_extensions(Arc::new(access_plan) as _); // Prepare for scanning - let schema = self.schema(); + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); let object_store_url = ObjectStoreUrl::parse("file://")?; let file_scan_config = FileScanConfig::new(object_store_url, schema) .with_limit(limit) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index cfb49b023159..7b3d7ef23d8a 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -22,7 +22,7 @@ use std::sync::{Arc, Mutex}; use std::time::Duration; use datafusion::arrow::array::{UInt64Builder, UInt8Builder}; -use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::arrow::datatypes::{DataType, SchemaRef}; use datafusion::arrow::record_batch::RecordBatch; use datafusion::datasource::{provider_as_source, TableProvider, TableType}; use datafusion::error::Result; @@ -38,6 +38,8 @@ use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use tokio::time::timeout; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; /// This example demonstrates executing a simple query against a custom datasource #[tokio::main] @@ -162,10 +164,10 @@ impl TableProvider for CustomDataSource { self } - fn schema(&self) -> SchemaRef { - SchemaRef::new(Schema::new(vec![ - Field::new("id", DataType::UInt8, false), - Field::new("bank_account", DataType::UInt64, true), + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(LogicalSchema::new(vec![ + LogicalField::new("id", DataType::UInt8, false), + LogicalField::new("bank_account", DataType::UInt64, true), ])) } @@ -181,7 +183,8 @@ impl TableProvider for CustomDataSource { _filters: &[Expr], _limit: Option, ) -> Result> { - return self.create_physical_plan(projection, self.schema()).await; + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); + return self.create_physical_plan(projection, schema).await; } } diff --git a/datafusion-examples/examples/dataframe_subquery.rs b/datafusion-examples/examples/dataframe_subquery.rs index f3cdd048968b..7a7f3083742c 100644 --- a/datafusion-examples/examples/dataframe_subquery.rs +++ b/datafusion-examples/examples/dataframe_subquery.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::logical_type::LogicalType; use std::sync::Arc; +use arrow_schema::DataType; use datafusion::error::Result; use datafusion::functions_aggregate::average::avg; use datafusion::prelude::*; @@ -48,7 +48,7 @@ async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { scalar_subquery(Arc::new( ctx.table("t2") .await? - .filter(out_ref_col(LogicalType::Utf8, "t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? .aggregate(vec![], vec![avg(col("t2.c2"))])? .select(vec![avg(col("t2.c2"))])? .into_unoptimized_plan(), @@ -91,7 +91,7 @@ async fn where_exist_subquery(ctx: &SessionContext) -> Result<()> { .filter(exists(Arc::new( ctx.table("t2") .await? - .filter(out_ref_col(LogicalType::Utf8, "t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? .select(vec![col("t2.c2")])? .into_unoptimized_plan(), )))? diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index d53a1feff085..40533961e7ee 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -30,7 +30,6 @@ use datafusion::physical_expr::{analyze, AnalysisContext, ExprBoundaries}; use datafusion::prelude::*; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::schema::LogicalSchema; -use datafusion_common::logical_type::LogicalType; use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr::BinaryExpr; @@ -214,7 +213,7 @@ fn simplify_demo() -> Result<()> { // String --> Date simplification // `cast('2020-09-01' as date)` --> 18500 assert_eq!( - simplifier.simplify(lit("2020-09-01").cast_to(&LogicalType::Date32, &schema)?)?, + simplifier.simplify(lit("2020-09-01").cast_to(&DataType::Date32.into(), &schema)?)?, lit(ScalarValue::Date32(Some(18506))) ); @@ -296,14 +295,14 @@ fn expression_type_demo() -> Result<()> { // a schema. In this case we create a schema where the column `c` is of // type Utf8 (a String / VARCHAR) let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("c", LogicalType::Utf8, true)].into(), + vec![LogicalField::new("c", DataType::Utf8, true)].into(), HashMap::new(), )?; assert_eq!("Utf8", format!("{}", expr.get_type(&schema).unwrap())); // Using a schema where the column `foo` is of type Int32 let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("c", LogicalType::Int32, true)].into(), + vec![LogicalField::new("c", DataType::Int32, true)].into(), HashMap::new(), )?; assert_eq!("Int32", format!("{}", expr.get_type(&schema).unwrap())); @@ -313,8 +312,8 @@ fn expression_type_demo() -> Result<()> { let expr = col("c1") + col("c2"); let schema = DFSchema::from_unqualified_fields( vec![ - LogicalField::new("c1", LogicalType::Int32, true), - LogicalField::new("c2", LogicalType::Float32, true), + LogicalField::new("c1", DataType::Int32, true), + LogicalField::new("c2", DataType::Float32, true), ] .into(), HashMap::new(), diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index bf2a44e7063e..a19f36fd6ac7 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -22,7 +22,7 @@ use datafusion::error::Result; use datafusion::execution::context::{ FunctionFactory, RegisterFunction, SessionContext, SessionState, }; -use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{exec_err, internal_err, DataFusionError}; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; @@ -218,13 +218,14 @@ impl TryFrom for ScalarFunctionWrapper { return_type: definition .return_type .expect("Return type has to be defined!") - .physical_type(), + .physical() + .clone(), signature: Signature::exact( definition .args .unwrap_or_default() .into_iter() - .map(|a| a.data_type.physical_type()) + .map(|a| a.data_type.physical().clone()) .collect(), definition .params diff --git a/datafusion-examples/examples/logical_type.rs b/datafusion-examples/examples/logical_type.rs new file mode 100644 index 000000000000..8a55e8035009 --- /dev/null +++ b/datafusion-examples/examples/logical_type.rs @@ -0,0 +1,3 @@ +fn main() { + +} \ No newline at end of file diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 668eda047444..49870095caf4 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -54,6 +54,7 @@ use std::sync::{ }; use tempfile::TempDir; use url::Url; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// This example demonstrates building a secondary index over multiple Parquet /// files and using that index during query to skip ("prune") files that do not @@ -212,8 +213,8 @@ impl TableProvider for IndexTableProvider { self } - fn schema(&self) -> SchemaRef { - self.index.schema().clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.index.schema().into()) } fn table_type(&self) -> TableType { @@ -243,7 +244,8 @@ impl TableProvider for IndexTableProvider { let files = self.index.get_files(predicate.clone())?; let object_store_url = ObjectStoreUrl::parse("file://")?; - let mut file_scan_config = FileScanConfig::new(object_store_url, self.schema()) + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); + let mut file_scan_config = FileScanConfig::new(object_store_url, schema) .with_projection(projection.cloned()) .with_limit(limit); diff --git a/datafusion-examples/examples/rewrite_expr.rs b/datafusion-examples/examples/rewrite_expr.rs index ed748288ff3d..e9e9014e78f0 100644 --- a/datafusion-examples/examples/rewrite_expr.rs +++ b/datafusion-examples/examples/rewrite_expr.rs @@ -17,7 +17,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ @@ -32,6 +32,7 @@ use datafusion_sql::sqlparser::parser::Parser; use datafusion_sql::TableReference; use std::any::Any; use std::sync::Arc; +use datafusion_common::logical_type::schema::LogicalSchemaRef; pub fn main() -> Result<()> { // produce a logical plan using the datafusion-sql crate @@ -212,7 +213,7 @@ impl ContextProvider for MyContextProvider { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -246,7 +247,7 @@ impl TableSource for MyTableSource { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.as_ref().clone().into()) } } diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index c68c21fab169..8776cc7599d1 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -35,7 +35,7 @@ use std::fs::File; use std::io::Seek; use std::path::Path; use std::sync::Arc; - +use datafusion_common::logical_type::schema::LogicalSchemaRef; // To define your own table function, you only need to do the following 3 things: // 1. Implement your own [`TableProvider`] // 2. Implement your own [`TableFunctionImpl`] and return your [`TableProvider`] @@ -85,8 +85,8 @@ impl TableProvider for LocalCsvTable { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { @@ -121,7 +121,7 @@ impl TableProvider for LocalCsvTable { }; Ok(Arc::new(MemoryExec::try_new( &[batches], - TableProvider::schema(self), + self.schema.clone(), projection.cloned(), )?)) } diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 58c43dab5675..aa3bee9a70f1 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -31,10 +31,9 @@ use crate::{ use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; -use crate::logical_type::extension::ExtensionType; use crate::logical_type::field::{LogicalField, LogicalFieldRef}; use crate::logical_type::fields::LogicalFields; -use crate::logical_type::LogicalType; +use crate::logical_type::{TypeRelation, ExtensionType}; use crate::logical_type::schema::{LogicalSchema, LogicalSchemaBuilder, LogicalSchemaRef}; /// A reference-counted reference to a [DFSchema]. @@ -165,7 +164,7 @@ impl DFSchema { /// Create a new `DFSchema` from a list of Arrow [Field]s #[allow(deprecated)] pub fn from_unqualified_fields( - fields: Fields, + fields: LogicalFields, metadata: HashMap, ) -> Result { Self::from_unqualifed_fields(fields, metadata) @@ -675,7 +674,7 @@ impl DFSchema { self_fields.zip(other_fields).all(|((q1, f1), (q2, f2))| { q1 == q2 && f1.name() == f2.name() - && Self::datatype_is_semantically_equal(&f1.data_type().physical_type(), &f2.data_type().physical_type()) + && Self::datatype_is_semantically_equal(&f1.data_type().physical(), &f2.data_type().physical()) }) } @@ -683,7 +682,7 @@ impl DFSchema { /// than datatype_is_semantically_equal in that a Dictionary type is logically /// equal to a plain V type, but not semantically equal. Dictionary is also /// logically equal to Dictionary. - pub fn datatype_is_logically_equal(dt1: &LogicalType, dt2: &LogicalType) -> bool { + pub fn datatype_is_logically_equal(dt1: &TypeRelation, dt2: &TypeRelation) -> bool { dt1 == dt2 } @@ -916,13 +915,13 @@ pub trait ExprSchema: std::fmt::Debug { fn nullable(&self, col: &Column) -> Result; /// What is the datatype of this column? - fn data_type(&self, col: &Column) -> Result<&LogicalType>; + fn data_type(&self, col: &Column) -> Result<&TypeRelation>; /// Returns the column's optional metadata. fn metadata(&self, col: &Column) -> Result<&HashMap>; /// Return the coulmn's datatype and nullability - fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)>; + fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)>; } // Implement `ExprSchema` for `Arc` @@ -931,7 +930,7 @@ impl + std::fmt::Debug> ExprSchema for P { self.as_ref().nullable(col) } - fn data_type(&self, col: &Column) -> Result<&LogicalType> { + fn data_type(&self, col: &Column) -> Result<&TypeRelation> { self.as_ref().data_type(col) } @@ -939,7 +938,7 @@ impl + std::fmt::Debug> ExprSchema for P { ExprSchema::metadata(self.as_ref(), col) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)> { self.as_ref().data_type_and_nullable(col) } } @@ -949,7 +948,7 @@ impl ExprSchema for DFSchema { Ok(self.field_from_column(col)?.is_nullable()) } - fn data_type(&self, col: &Column) -> Result<&LogicalType> { + fn data_type(&self, col: &Column) -> Result<&TypeRelation> { Ok(self.field_from_column(col)?.data_type()) } @@ -957,7 +956,7 @@ impl ExprSchema for DFSchema { Ok(self.field_from_column(col)?.metadata()) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)> { let field = self.field_from_column(col)?; Ok((field.data_type(), field.is_nullable())) } @@ -1080,8 +1079,8 @@ mod tests { let schema = DFSchema::from_field_specific_qualified_schema( vec![Some("t1".into()), None], &Arc::new(LogicalSchema::new(vec![ - LogicalField::new("c0", LogicalType::Boolean, true), - LogicalField::new("c1", LogicalType::Boolean, true), + LogicalField::new("c0", DataType::Boolean, true), + LogicalField::new("c1", DataType::Boolean, true), ])), )?; assert_eq!("fields:[t1.c0, c1], metadata:{}", schema.to_string()); @@ -1333,8 +1332,8 @@ mod tests { fn test_schema_2() -> LogicalSchema { LogicalSchema::new(vec![ - LogicalField::new("c100", LogicalType::Boolean, true), - LogicalField::new("c101", LogicalType::Boolean, true), + LogicalField::new("c100", DataType::Boolean, true), + LogicalField::new("c101", DataType::Boolean, true), ]) } diff --git a/datafusion/common/src/logical_type/extension.rs b/datafusion/common/src/logical_type/extension.rs deleted file mode 100644 index 0332de49f19e..000000000000 --- a/datafusion/common/src/logical_type/extension.rs +++ /dev/null @@ -1,289 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::sync::Arc; - -use arrow_schema::{DataType, FieldRef, IntervalUnit, TimeUnit}; - -use crate::logical_type::type_signature::TypeSignature; -use crate::logical_type::LogicalType; - -pub type ExtensionTypeRef = Arc; - -pub trait ExtensionType: std::fmt::Debug { - fn display_name(&self) -> &str; - fn type_signature(&self) -> TypeSignature; - fn physical_type(&self) -> DataType; - - fn is_comparable(&self) -> bool; - fn is_orderable(&self) -> bool; - fn is_numeric(&self) -> bool; - fn is_floating(&self) -> bool; -} - -impl ExtensionType for LogicalType { - fn display_name(&self) -> &str { - use crate::logical_type::LogicalType::*; - match self { - Null => "Null", - Boolean => "Boolean", - Int8 => "Int8", - Int16 => "Int16", - Int32 => "Int32", - Int64 => "Int64", - UInt8 => "Uint8", - UInt16 => "Uint16", - UInt32 => "Uint32", - UInt64 => "Uint64", - Float16 => "Float16", - Float32 => "Float16", - Float64 => "Float64", - Date32 => "Date32", - Date64 => "Date64", - Time32(_) => "Time32", - Time64(_) => "Time64", - Timestamp(_, _) => "Timestamp", - Duration(_) => "Duration", - Interval(_) => "Interval", - Binary => "Binary", - FixedSizeBinary(_) => "FixedSizeBinary", - LargeBinary => "LargeBinary", - Utf8 => "Utf8", - LargeUtf8 => "LargeUtf8", - List(_) => "List", - FixedSizeList(_, _) => "FixedSizeList", - LargeList(_) => "LargeList", - Struct(_) => "Struct", - Map(_, _) => "Map", - Decimal128(_, _) => "Decimal128", - Decimal256(_, _) => "Decimal256", - Extension(ext) => ext.display_name(), - } - } - - fn type_signature(&self) -> TypeSignature { - use crate::logical_type::LogicalType::*; - fn time_unit_to_param(tu: &TimeUnit) -> &'static str { - match tu { - TimeUnit::Second => "second", - TimeUnit::Millisecond => "millisecond", - TimeUnit::Microsecond => "microsecond", - TimeUnit::Nanosecond => "nanosecond", - } - } - - match self { - Boolean => TypeSignature::new("boolean"), - Int32 => TypeSignature::new("int32"), - Int64 => TypeSignature::new("int64"), - UInt64 => TypeSignature::new("uint64"), - Float32 => TypeSignature::new("float32"), - Float64 => TypeSignature::new("float64"), - Timestamp(tu, zone) => { - let params = if let Some(zone) = zone { - vec![time_unit_to_param(tu).into(), zone.as_ref().into()] - } else { - vec![time_unit_to_param(tu).into()] - }; - - TypeSignature::new_with_params("timestamp", params) - } - Binary => TypeSignature::new("binary"), - Utf8 => TypeSignature::new("string"), - Struct(fields) => { - let params = fields.iter().map(|f| f.name().into()).collect(); - TypeSignature::new_with_params("struct", params) - } - Extension(ext) => ext.type_signature(), - Null => TypeSignature::new("null"), - Int8 => TypeSignature::new("int8"), - Int16 => TypeSignature::new("int16"), - UInt8 => TypeSignature::new("uint8"), - UInt16 => TypeSignature::new("uint16"), - UInt32 => TypeSignature::new("uint32"), - Float16 => TypeSignature::new("float16"), - Date32 => TypeSignature::new("date_32"), - Date64 => TypeSignature::new("date_64"), - Time32(tu) => TypeSignature::new_with_params( - "time_32", - vec![time_unit_to_param(tu).into()], - ), - Time64(tu) => TypeSignature::new_with_params( - "time_64", - vec![time_unit_to_param(tu).into()], - ), - Duration(tu) => TypeSignature::new_with_params( - "duration", - vec![time_unit_to_param(tu).into()], - ), - Interval(iu) => { - let iu = match iu { - IntervalUnit::YearMonth => "year_month", - IntervalUnit::DayTime => "day_time", - IntervalUnit::MonthDayNano => "month_day_nano", - }; - TypeSignature::new_with_params("interval", vec![iu.into()]) - } - FixedSizeBinary(size) => TypeSignature::new_with_params( - "fixed_size_binary", - vec![size.to_string().into()], - ), - LargeBinary => TypeSignature::new("large_binary"), - LargeUtf8 => TypeSignature::new("large_utf_8"), - List(f) => TypeSignature::new_with_params( - "list", - vec![f.data_type().display_name().into()], - ), - FixedSizeList(f, size) => TypeSignature::new_with_params( - "fixed_size_list", - vec![f.data_type().display_name().into(), size.to_string().into()], - ), - LargeList(f) => TypeSignature::new_with_params( - "large_list", - vec![f.data_type().display_name().into()], - ), - Map(f, b) => TypeSignature::new_with_params( - "map", - vec![f.data_type().display_name().into(), b.to_string().into()], - ), - Decimal128(a, b) => TypeSignature::new_with_params( - "decimal_128", - vec![a.to_string().into(), b.to_string().into()], - ), - Decimal256(a, b) => TypeSignature::new_with_params( - "decimal_256", - vec![a.to_string().into(), b.to_string().into()], - ), - } - } - - fn physical_type(&self) -> DataType { - use crate::logical_type::LogicalType::*; - match self { - Boolean => DataType::Boolean, - Int32 => DataType::Int32, - Int64 => DataType::Int64, - UInt64 => DataType::UInt64, - Float32 => DataType::Float32, - Float64 => DataType::Float64, - Timestamp(tu, zone) => DataType::Timestamp(tu.clone(), zone.clone()), - Binary => DataType::Binary, - Utf8 => DataType::Utf8, - Struct(fields) => { - let fields = fields - .iter() - .map(|f| FieldRef::new(f.as_ref().clone().into())) - .collect::>(); - DataType::Struct(fields.into()) - } - Extension(ext) => ext.physical_type(), - Null => DataType::Null, - Int8 => DataType::Int8, - Int16 => DataType::Int16, - UInt8 => DataType::UInt8, - UInt16 => DataType::UInt16, - UInt32 => DataType::UInt32, - Float16 => DataType::Float16, - Date32 => DataType::Date32, - Date64 => DataType::Date64, - Time32(tu) => DataType::Time32(tu.to_owned()), - Time64(tu) => DataType::Time64(tu.to_owned()), - Duration(tu) => DataType::Duration(tu.to_owned()), - Interval(iu) => DataType::Interval(iu.to_owned()), - FixedSizeBinary(size) => DataType::FixedSizeBinary(size.to_owned()), - LargeBinary => DataType::LargeBinary, - LargeUtf8 => DataType::LargeUtf8, - List(f) => DataType::List(FieldRef::new(f.as_ref().clone().into())), - FixedSizeList(f, size) => DataType::FixedSizeList(FieldRef::new(f.as_ref().clone().into()), size.to_owned()), - LargeList(f) => DataType::LargeList(FieldRef::new(f.as_ref().clone().into())), - Map(f, b) => DataType::Map(FieldRef::new(f.as_ref().clone().into()), b.to_owned()), - Decimal128(a, b) => DataType::Decimal128(a.to_owned(), b.to_owned()), - Decimal256(a, b) => DataType::Decimal256(a.to_owned(), b.to_owned()), - } - } - - fn is_comparable(&self) -> bool { - use crate::logical_type::LogicalType::*; - match self { - Null - | Boolean - | Int8 - | Int16 - | Int32 - | Int64 - | UInt8 - | UInt16 - | UInt32 - | UInt64 - | Float16 - | Float32 - | Float64 - | Date32 - | Date64 - | Time32(_) - | Time64(_) - | Timestamp(_, _) - | Duration(_) - | Interval(_) - | Binary - | FixedSizeBinary(_) - | LargeBinary - | Utf8 - | LargeUtf8 - | Decimal128(_, _) - | Decimal256(_, _) => true, - Extension(ext) => ext.is_comparable(), - _ => false, - } - } - - fn is_orderable(&self) -> bool { - todo!() - } - - #[inline] - fn is_numeric(&self) -> bool { - use crate::logical_type::LogicalType::*; - match self { - UInt8 - | UInt16 - | UInt32 - | UInt64 - | Int8 - | Int16 - | Int32 - | Int64 - | Float16 - | Float32 - | Float64 - | Decimal128(_, _) - | Decimal256(_, _) => true, - Extension(t) => t.is_numeric(), - _ => false, - } - } - - #[inline] - fn is_floating(&self) -> bool { - use crate::logical_type::LogicalType::*; - match self { - Float16 | Float32 | Float64 => true, - Extension(t) => t.is_floating(), - _ => false, - } - } -} diff --git a/datafusion/common/src/logical_type/field.rs b/datafusion/common/src/logical_type/field.rs index 3cd6f73fe374..720370a9dbac 100644 --- a/datafusion/common/src/logical_type/field.rs +++ b/datafusion/common/src/logical_type/field.rs @@ -1,18 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::collections::HashMap; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use arrow_schema::{DataType, Field}; -use crate::logical_type::extension::ExtensionType; -use crate::logical_type::fields::LogicalFields; -use crate::logical_type::LogicalType; -use crate::logical_type::type_signature::TypeSignature; + +use arrow_schema::{Field, FieldRef}; + +use super::{ExtensionType, TypeRelation}; pub type LogicalFieldRef = Arc; #[derive(Debug, Clone)] pub struct LogicalField { name: String, - data_type: LogicalType, + data_type: TypeRelation, nullable: bool, metadata: HashMap, } @@ -34,9 +50,25 @@ impl From for LogicalField { } } +impl From<&FieldRef> for LogicalField { + fn from(value: &FieldRef) -> Self { + Self::from(value.as_ref()) + } +} + +impl From for LogicalField { + fn from(value: FieldRef) -> Self { + Self::from(value.as_ref()) + } +} + impl Into for LogicalField { fn into(self) -> Field { - Field::new(self.name, self.data_type.physical_type(), self.nullable).with_metadata(self.metadata) + Field::new( + self.name, + self.data_type.physical().clone(), + self.nullable + ).with_metadata(self.metadata) } } @@ -67,59 +99,25 @@ impl Hash for LogicalField { } } -impl ExtensionType for LogicalField { - fn display_name(&self) -> &str { - &self.name - } - - fn type_signature(&self) -> TypeSignature { - TypeSignature::new(self.name()) - } - - fn physical_type(&self) -> DataType { - self.data_type.physical_type() - } - - fn is_comparable(&self) -> bool { - self.data_type.is_comparable() - } - - fn is_orderable(&self) -> bool { - self.data_type.is_orderable() - } - - fn is_numeric(&self) -> bool { - self.data_type.is_numeric() - } - - fn is_floating(&self) -> bool { - self.data_type.is_floating() - } -} - impl LogicalField { - pub fn new(name: impl Into, data_type: LogicalType, nullable: bool) -> Self { + pub fn new(name: impl Into, data_type: impl Into, nullable: bool) -> Self { LogicalField { name: name.into(), - data_type, + data_type: data_type.into(), nullable, metadata: HashMap::default(), } } - pub fn new_list_field(data_type: LogicalType, nullable: bool) -> Self { + pub fn new_list_field(data_type: impl Into, nullable: bool) -> Self { Self::new("item", data_type, nullable) } - pub fn new_struct(name: impl Into, fields: impl Into, nullable: bool) -> Self { - Self::new(name, LogicalType::Struct(fields.into()), nullable) - } - pub fn name(&self) -> &str { &self.name } - pub fn data_type(&self) -> &LogicalType { + pub fn data_type(&self) -> &TypeRelation { &self.data_type } @@ -150,7 +148,7 @@ impl LogicalField { } #[inline] - pub fn with_data_type(mut self, data_type: LogicalType) -> Self { + pub fn with_data_type(mut self, data_type: TypeRelation) -> Self { self.data_type = data_type; self } diff --git a/datafusion/common/src/logical_type/fields.rs b/datafusion/common/src/logical_type/fields.rs index 456835bc8d9b..99b9f7904a97 100644 --- a/datafusion/common/src/logical_type/fields.rs +++ b/datafusion/common/src/logical_type/fields.rs @@ -1,7 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::ops::Deref; use std::sync::Arc; -use arrow_schema::{Field, Fields}; -use crate::logical_type::field::{LogicalField, LogicalFieldRef}; + +use arrow_schema::{Field, Fields, UnionFields}; + +use super::field::{LogicalField, LogicalFieldRef}; #[derive(Clone, Eq, PartialEq, Hash)] pub struct LogicalFields(Arc<[LogicalFieldRef]>); @@ -12,9 +31,15 @@ impl std::fmt::Debug for LogicalFields { } } +impl From<&Fields> for LogicalFields { + fn from(value: &Fields) -> Self { + Self(value.iter().map(|v| LogicalFieldRef::new(v.into())).collect()) + } +} + impl From for LogicalFields { fn from(value: Fields) -> Self { - Self(value.into_iter().map(|v| LogicalFieldRef::new(LogicalField::from(v.as_ref()))).collect()) + Self::from(&value) } } @@ -92,3 +117,30 @@ impl LogicalFields { Self(Arc::new([])) } } + +#[derive(Clone, Eq, PartialEq, Hash)] +pub struct LogicalUnionFields(Arc<[(i8, LogicalFieldRef)]>); + +impl std::fmt::Debug for LogicalUnionFields { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.0.as_ref().fmt(f) + } +} + +impl FromIterator<(i8, LogicalFieldRef)> for LogicalUnionFields { + fn from_iter>(iter: T) -> Self { + Self(iter.into_iter().collect()) + } +} + +impl From<&UnionFields> for LogicalUnionFields { + fn from(value: &UnionFields) -> Self { + Self::from_iter(value.iter().map(|(i, f)| (i, LogicalFieldRef::new(f.into())))) + } +} + +impl From for LogicalUnionFields { + fn from(value: UnionFields) -> Self { + Self::from(&value) + } +} diff --git a/datafusion/common/src/logical_type/mod.rs b/datafusion/common/src/logical_type/mod.rs index fd464411a4ef..1629aa705c6b 100644 --- a/datafusion/common/src/logical_type/mod.rs +++ b/datafusion/common/src/logical_type/mod.rs @@ -17,85 +17,136 @@ use std::{fmt::Display, sync::Arc}; -use arrow_schema::{DataType, IntervalUnit, TimeUnit}; +use arrow_schema::DataType; -use crate::logical_type::extension::{ExtensionType, ExtensionTypeRef}; -use crate::logical_type::field::{LogicalField, LogicalFieldRef}; -use crate::logical_type::fields::LogicalFields; +use field::{LogicalField, LogicalFieldRef}; +use fields::LogicalFields; +use signature::LogicalType; -pub mod type_signature; -pub mod extension; -pub mod registry; -pub mod schema; pub mod field; pub mod fields; +pub mod schema; +pub mod signature; + +pub type ExtensionTypeRef = Arc; + +pub trait ExtensionType: std::fmt::Debug { + fn logical(&self) -> &LogicalType; + fn physical(&self) -> &DataType; +} #[derive(Clone, Debug)] -pub enum LogicalType { - Null, - Boolean, - Int8, - Int16, - Int32, - Int64, - UInt8, - UInt16, - UInt32, - UInt64, - Float16, - Float32, - Float64, - Date32, - Date64, - Time32(TimeUnit), - Time64(TimeUnit), - Timestamp(TimeUnit, Option>), - Duration(TimeUnit), - Interval(IntervalUnit), - Binary, - FixedSizeBinary(i32), - LargeBinary, - Utf8, - LargeUtf8, - List(LogicalFieldRef), - FixedSizeList(LogicalFieldRef, i32), - LargeList(LogicalFieldRef), - Struct(LogicalFields), - Map(LogicalFieldRef, bool), - Decimal128(u8, i8), - Decimal256(u8, i8), - Extension(ExtensionTypeRef), - // TODO: tbd union +pub struct TypeRelation(ExtensionTypeRef); + +impl TypeRelation { + pub fn new_list(inner: TypeRelation, nullable: bool) -> Self { + Self(Arc::new(NativeType::new_list(inner, nullable))) + } + + pub fn new_struct(fields: LogicalFields) -> Self { + Self(Arc::new(NativeType::new_struct(fields))) + } +} + +pub type NativeTypeRef = Arc; + +#[derive(Clone, Debug)] +pub struct NativeType { + logical: LogicalType, + physical: DataType, +} + +impl ExtensionType for NativeType { + fn logical(&self) -> &LogicalType { + &self.logical + } + + fn physical(&self) -> &DataType { + &self.physical + } +} + +impl NativeType { + pub fn new_list(inner: TypeRelation, nullable: bool) -> Self { + Self { + physical: DataType::new_list(inner.physical().clone(), nullable), + logical: LogicalType::List(LogicalFieldRef::new(LogicalField::new_list_field(inner, nullable))), + } + } + + pub fn new_struct(fields: LogicalFields) -> Self { + Self { + physical: DataType::Struct(fields.clone().into()), + logical: LogicalType::Struct(fields), + } + } +} + +impl ExtensionType for TypeRelation { + fn logical(&self) -> &LogicalType { + self.0.logical() + } + + fn physical(&self) -> &DataType { + self.0.physical() + } } -impl PartialEq for LogicalType { +impl PartialEq for TypeRelation { fn eq(&self, other: &Self) -> bool { - self.type_signature() == other.type_signature() + self.logical() == other.logical() } } -impl Eq for LogicalType {} +impl Eq for TypeRelation {} -impl std::hash::Hash for LogicalType { +impl std::hash::Hash for TypeRelation { fn hash(&self, state: &mut H) { - self.type_signature().hash(state) + self.logical().hash(state) } } -impl Display for LogicalType { +impl Display for TypeRelation { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{self:?}") } } -impl From<&DataType> for LogicalType { +impl From for NativeType { + fn from(value: DataType) -> Self { + Self { + logical: (&value).into(), + physical: value, + } + } +} + +impl From for TypeRelation { + fn from(value: ExtensionTypeRef) -> Self { + Self(value) + } +} + +impl From<&DataType> for TypeRelation { fn from(value: &DataType) -> Self { value.clone().into() } } +impl From for TypeRelation { + fn from(value: DataType) -> Self { + Self(NativeTypeRef::new(value.into())) + } +} + impl From for LogicalType { fn from(value: DataType) -> Self { + (&value).into() + } +} + +impl From<&DataType> for LogicalType { + fn from(value: &DataType) -> Self { match value { DataType::Null => LogicalType::Null, DataType::Boolean => LogicalType::Boolean, @@ -110,47 +161,42 @@ impl From for LogicalType { DataType::Float16 => LogicalType::Float16, DataType::Float32 => LogicalType::Float32, DataType::Float64 => LogicalType::Float64, - DataType::Timestamp(tu, z) => LogicalType::Timestamp(tu, z), - DataType::Date32 => LogicalType::Date32, - DataType::Date64 => LogicalType::Date64, - DataType::Time32(tu) => LogicalType::Time32(tu), - DataType::Time64(tu) => LogicalType::Time64(tu), - DataType::Duration(tu) => LogicalType::Duration(tu), - DataType::Interval(iu) => LogicalType::Interval(iu), - DataType::Binary | DataType::BinaryView => LogicalType::Binary, - DataType::FixedSizeBinary(len) => LogicalType::FixedSizeBinary(len), - DataType::LargeBinary => LogicalType::LargeBinary, - DataType::Utf8 | DataType::Utf8View => LogicalType::Utf8, - DataType::LargeUtf8 => LogicalType::LargeUtf8, - DataType::List(f) | DataType::ListView(f) => LogicalType::List(LogicalFieldRef::new(f.as_ref().into())), - DataType::FixedSizeList(f, len) => LogicalType::FixedSizeList(LogicalFieldRef::new(f.as_ref().into()), len), - DataType::LargeList(f) | DataType::LargeListView(f) => LogicalType::LargeList(LogicalFieldRef::new(f.as_ref().into())), - DataType::Struct(fields) => LogicalType::Struct(fields.into()), - DataType::Dictionary(_, dt) => dt.as_ref().into(), - DataType::Decimal128(p, s) => LogicalType::Decimal128(p, s), - DataType::Decimal256(p, s) => LogicalType::Decimal256(p, s), - DataType::Map(f, sorted) => LogicalType::Map(LogicalFieldRef::new(f.as_ref().into()), sorted), + DataType::Timestamp(tu, tz) => { + LogicalType::Timestamp(tu.clone(), tz.clone()) + } + DataType::Date32 | DataType::Date64 => LogicalType::Date, + DataType::Time32(tu) => LogicalType::Time32(tu.clone()), + DataType::Time64(tu) => LogicalType::Time64(tu.clone()), + DataType::Duration(tu) => LogicalType::Duration(tu.clone()), + DataType::Interval(iu) => LogicalType::Interval(iu.clone()), + DataType::Binary + | DataType::FixedSizeBinary(_) + | DataType::LargeBinary + | DataType::BinaryView => LogicalType::Binary, + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { + LogicalType::Utf8 + } + DataType::List(f) + | DataType::ListView(f) + | DataType::FixedSizeList(f, _) + | DataType::LargeList(f) + | DataType::LargeListView(f) => { + LogicalType::List(LogicalFieldRef::new(f.as_ref().clone().into())) + } + DataType::Struct(f) => LogicalType::Struct(f.clone().into()), + DataType::Dictionary(_, t) => t.as_ref().into(), + DataType::Decimal128(precision, scale) => { + LogicalType::Decimal128(precision.clone(), scale.clone()) + } + DataType::Decimal256(precision, scale) => { + LogicalType::Decimal256(precision.clone(), scale.clone()) + } + DataType::Map(f, sorted) => LogicalType::Map( + LogicalFieldRef::new(f.as_ref().clone().into()), + sorted.clone(), + ), DataType::RunEndEncoded(_, f) => f.data_type().into(), - DataType::Union(_, _) => unimplemented!(), // TODO: tbd union + DataType::Union(f, _) => LogicalType::Union(f.into()), } } } - -impl LogicalType { - - pub fn new_list(data_type: LogicalType, nullable: bool) -> Self { - LogicalType::List(Arc::new(LogicalField::new_list_field(data_type, nullable))) - } - - pub fn new_large_list(data_type: LogicalType, nullable: bool) -> Self { - LogicalType::LargeList(Arc::new(LogicalField::new_list_field(data_type, nullable))) - } - - pub fn new_fixed_size_list(data_type: LogicalType, size: i32, nullable: bool) -> Self { - LogicalType::FixedSizeList(Arc::new(LogicalField::new_list_field(data_type, nullable)), size) - } - - pub fn is_floating(&self) -> bool { - matches!(self, Self::Float16 | Self::Float32 | Self::Float64) - } -} diff --git a/datafusion/common/src/logical_type/registry.rs b/datafusion/common/src/logical_type/registry.rs deleted file mode 100644 index 63d25bb23e91..000000000000 --- a/datafusion/common/src/logical_type/registry.rs +++ /dev/null @@ -1,51 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::collections::HashMap; - -use crate::error::_plan_datafusion_err; -use crate::logical_type::extension::ExtensionTypeRef; -use crate::logical_type::type_signature::TypeSignature; - -pub trait TypeRegistry { - fn register_data_type( - &mut self, - extension_type: ExtensionTypeRef, - ) -> crate::Result>; - - fn data_type(&self, signature: &TypeSignature) -> crate::Result; -} - - -#[derive(Default, Debug)] -pub struct MemoryTypeRegistry { - types: HashMap, -} - -impl TypeRegistry for MemoryTypeRegistry { - fn register_data_type(&mut self, extension_type: ExtensionTypeRef) -> crate::Result> { - Ok(self.types.insert(extension_type.type_signature(), extension_type)) - } - - fn data_type(&self, signature: &TypeSignature) -> crate::Result { - self.types - .get(signature) - .cloned() - .ok_or_else(|| _plan_datafusion_err!("Type with signature {signature:?} not found")) - } -} - diff --git a/datafusion/common/src/logical_type/schema.rs b/datafusion/common/src/logical_type/schema.rs index f0c2854b4e64..7e64d1d8df77 100644 --- a/datafusion/common/src/logical_type/schema.rs +++ b/datafusion/common/src/logical_type/schema.rs @@ -1,10 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::collections::HashMap; use std::sync::Arc; -use arrow_schema::Schema; +use arrow_schema::{Schema, SchemaRef}; -use crate::logical_type::field::{LogicalField, LogicalFieldRef}; -use crate::logical_type::fields::LogicalFields; +use super::field::{LogicalField, LogicalFieldRef}; +use super::fields::LogicalFields; #[derive(Debug, Default)] pub struct LogicalSchemaBuilder { @@ -130,6 +147,34 @@ impl From for LogicalSchema { } } +impl From<&Schema> for LogicalSchema { + fn from(value: &Schema) -> Self { + Self::from(value.clone()) + } +} + +impl From for LogicalSchema { + fn from(value: SchemaRef) -> Self { + Self::from(value.as_ref()) + } +} + +impl From<&SchemaRef> for LogicalSchema { + fn from(value: &SchemaRef) -> Self { + Self::from(value.as_ref()) + } +} + +impl Into for LogicalSchema { + fn into(self) -> Schema { + Schema { + fields: self.fields.into(), + metadata: self.metadata, + } + } +} + + impl LogicalSchema { pub fn new(fields: impl Into) -> Self { Self::new_with_metadata(fields, HashMap::new()) @@ -156,4 +201,8 @@ impl LogicalSchema { pub fn field(&self, i: usize) -> &LogicalFieldRef { &self.fields[i] } -} \ No newline at end of file + + pub fn fields(&self) -> &LogicalFields { + &self.fields + } +} diff --git a/datafusion/common/src/logical_type/signature.rs b/datafusion/common/src/logical_type/signature.rs new file mode 100644 index 000000000000..2d75d4ea0a24 --- /dev/null +++ b/datafusion/common/src/logical_type/signature.rs @@ -0,0 +1,135 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use core::fmt; +use std::sync::Arc; + +use arrow_schema::{IntervalUnit, TimeUnit}; + +use super::{field::LogicalFieldRef, fields::{LogicalFields, LogicalUnionFields}}; + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub enum LogicalType { + Null, + Int8, + Int16, + Int32, + Int64, + UInt8, + UInt16, + UInt32, + UInt64, + Boolean, + Float16, + Float32, + Float64, + Utf8, + Binary, + Date, + Time32(TimeUnit), + Time64(TimeUnit), + Timestamp(TimeUnit, Option>), + Duration(TimeUnit), + Interval(IntervalUnit), + List(LogicalFieldRef), + Struct(LogicalFields), + Map(LogicalFieldRef, bool), + Decimal128(u8, i8), + Decimal256(u8, i8), + Union(LogicalUnionFields) + // TODO: extension signatures? +} + +impl fmt::Display for LogicalType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{self:?}") + } +} + +impl LogicalType { + /// Returns true if the type is primitive: (numeric, temporal). + #[inline] + pub fn is_primitive(&self) -> bool { + self.is_numeric() || self.is_temporal() + } + + /// Returns true if this type is numeric: (UInt*, Int*, Float*, Decimal*). + #[inline] + pub fn is_numeric(&self) -> bool { + use LogicalType::*; + matches!( + self, + UInt8 + | UInt16 + | UInt32 + | UInt64 + | Int8 + | Int16 + | Int32 + | Int64 + | Float16 + | Float32 + | Float64 + | Decimal128(_, _) + | Decimal256(_, _) + ) + } + + /// Returns true if this type is temporal: (Date*, Time*, Duration, or Interval). + #[inline] + pub fn is_temporal(&self) -> bool { + use LogicalType::*; + matches!( + self, + Date | Timestamp(_, _) | Time32(_) | Time64(_) | Duration(_) | Interval(_) + ) + } + + /// Returns true if this type is floating: (Float*). + #[inline] + pub fn is_floating(&self) -> bool { + use LogicalType::*; + matches!(self, Float16 | Float32 | Float64) + } + + /// Returns true if this type is integer: (Int*, UInt*). + #[inline] + pub fn is_integer(&self) -> bool { + self.is_signed_integer() || self.is_unsigned_integer() + } + + /// Returns true if this type is signed integer: (Int*). + #[inline] + pub fn is_signed_integer(&self) -> bool { + use LogicalType::*; + matches!(self, Int8 | Int16 | Int32 | Int64) + } + + /// Returns true if this type is unsigned integer: (UInt*). + #[inline] + pub fn is_unsigned_integer(&self) -> bool { + use LogicalType::*; + matches!(self, UInt8 | UInt16 | UInt32 | UInt64) + } + + /// Returns true if this type is TypeSignature::Null. + #[inline] + pub fn is_null(&self) -> bool { + use LogicalType::*; + matches!(self, Null) + } +} diff --git a/datafusion/common/src/logical_type/type_signature.rs b/datafusion/common/src/logical_type/type_signature.rs deleted file mode 100644 index 4b46d040e5c4..000000000000 --- a/datafusion/common/src/logical_type/type_signature.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::sync::Arc; - -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct TypeSignature { - // **func_name**(p1, p2) - name: Arc, - // func_name(**p1**, **p2**) - params: Vec>, -} - -impl TypeSignature { - pub fn new(name: impl Into>) -> Self { - Self::new_with_params(name, vec![]) - } - - pub fn new_with_params( - name: impl Into>, - params: Vec>, - ) -> Self { - Self { - name: name.into(), - params, - } - } -} diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index e389d520d834..84684509abaa 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -18,7 +18,7 @@ use crate::error::{_plan_datafusion_err, _plan_err}; use crate::{Result, ScalarValue}; use std::collections::HashMap; -use crate::logical_type::LogicalType; +use crate::logical_type::TypeRelation; /// The parameter value corresponding to the placeholder #[derive(Debug, Clone)] @@ -31,7 +31,7 @@ pub enum ParamValues { impl ParamValues { /// Verify parameter list length and type - pub fn verify(&self, expect: &[LogicalType]) -> Result<()> { + pub fn verify(&self, expect: &[TypeRelation]) -> Result<()> { match self { ParamValues::List(list) => { // Verify if the number of params matches the number of values diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 67e959984b3d..5b9c4a223de6 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -54,11 +54,10 @@ use arrow::{ }, }; use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, ScalarBuffer}; -use arrow_schema::{FieldRef, UnionFields, UnionMode}; +use arrow_schema::{UnionFields, UnionMode}; use half::f16; pub use struct_builder::ScalarStructBuilder; -use crate::logical_type::LogicalType; /// A dynamically typed, nullable single value. /// @@ -3271,118 +3270,6 @@ impl TryFrom<&DataType> for ScalarValue { } } - -impl TryFrom for ScalarValue { - type Error = DataFusionError; - - /// Create a Null instance of ScalarValue for this datatype - fn try_from(datatype: LogicalType) -> Result { - (&datatype).try_into() - } -} - -impl TryFrom<&LogicalType> for ScalarValue { - type Error = DataFusionError; - - /// Create a Null instance of ScalarValue for this datatype - fn try_from(data_type: &LogicalType) -> Result { - Ok(match data_type { - LogicalType::Boolean => ScalarValue::Boolean(None), - LogicalType::Float16 => ScalarValue::Float16(None), - LogicalType::Float64 => ScalarValue::Float64(None), - LogicalType::Float32 => ScalarValue::Float32(None), - LogicalType::Int8 => ScalarValue::Int8(None), - LogicalType::Int16 => ScalarValue::Int16(None), - LogicalType::Int32 => ScalarValue::Int32(None), - LogicalType::Int64 => ScalarValue::Int64(None), - LogicalType::UInt8 => ScalarValue::UInt8(None), - LogicalType::UInt16 => ScalarValue::UInt16(None), - LogicalType::UInt32 => ScalarValue::UInt32(None), - LogicalType::UInt64 => ScalarValue::UInt64(None), - LogicalType::Decimal128(precision, scale) => { - ScalarValue::Decimal128(None, *precision, *scale) - } - LogicalType::Decimal256(precision, scale) => { - ScalarValue::Decimal256(None, *precision, *scale) - } - LogicalType::Utf8 => ScalarValue::Utf8(None), - LogicalType::LargeUtf8 => ScalarValue::LargeUtf8(None), - LogicalType::Binary => ScalarValue::Binary(None), - LogicalType::FixedSizeBinary(len) => ScalarValue::FixedSizeBinary(*len, None), - LogicalType::LargeBinary => ScalarValue::LargeBinary(None), - LogicalType::Date32 => ScalarValue::Date32(None), - LogicalType::Date64 => ScalarValue::Date64(None), - LogicalType::Time32(TimeUnit::Second) => ScalarValue::Time32Second(None), - LogicalType::Time32(TimeUnit::Millisecond) => { - ScalarValue::Time32Millisecond(None) - } - LogicalType::Time64(TimeUnit::Microsecond) => { - ScalarValue::Time64Microsecond(None) - } - LogicalType::Time64(TimeUnit::Nanosecond) => ScalarValue::Time64Nanosecond(None), - LogicalType::Timestamp(TimeUnit::Second, tz_opt) => { - ScalarValue::TimestampSecond(None, tz_opt.clone()) - } - LogicalType::Timestamp(TimeUnit::Millisecond, tz_opt) => { - ScalarValue::TimestampMillisecond(None, tz_opt.clone()) - } - LogicalType::Timestamp(TimeUnit::Microsecond, tz_opt) => { - ScalarValue::TimestampMicrosecond(None, tz_opt.clone()) - } - LogicalType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { - ScalarValue::TimestampNanosecond(None, tz_opt.clone()) - } - LogicalType::Interval(IntervalUnit::YearMonth) => { - ScalarValue::IntervalYearMonth(None) - } - LogicalType::Interval(IntervalUnit::DayTime) => { - ScalarValue::IntervalDayTime(None) - } - LogicalType::Interval(IntervalUnit::MonthDayNano) => { - ScalarValue::IntervalMonthDayNano(None) - } - LogicalType::Duration(TimeUnit::Second) => ScalarValue::DurationSecond(None), - LogicalType::Duration(TimeUnit::Millisecond) => { - ScalarValue::DurationMillisecond(None) - } - LogicalType::Duration(TimeUnit::Microsecond) => { - ScalarValue::DurationMicrosecond(None) - } - LogicalType::Duration(TimeUnit::Nanosecond) => { - ScalarValue::DurationNanosecond(None) - } - // `ScalaValue::List` contains single element `ListArray`. - LogicalType::List(field_ref) => ScalarValue::List(Arc::new( - GenericListArray::new_null(FieldRef::new(field_ref.as_ref().clone().into()), 1), - )), - // `ScalarValue::LargeList` contains single element `LargeListArray`. - LogicalType::LargeList(field_ref) => ScalarValue::LargeList(Arc::new( - GenericListArray::new_null(FieldRef::new(field_ref.as_ref().clone().into()), 1), - )), - // `ScalaValue::FixedSizeList` contains single element `FixedSizeList`. - LogicalType::FixedSizeList(field_ref, fixed_length) => { - ScalarValue::FixedSizeList(Arc::new(FixedSizeListArray::new_null( - FieldRef::new(field_ref.as_ref().clone().into()), - *fixed_length, - 1, - ))) - } - LogicalType::Struct(fields) => ScalarValue::Struct( - new_null_array(&DataType::Struct(fields.clone().into()), 1) - .as_struct() - .to_owned() - .into(), - ), - LogicalType::Null => ScalarValue::Null, - _ => { - return _not_impl_err!( - "Can't create a scalar from data_type \"{data_type:?}\"" - ); - } - }) - } -} - macro_rules! format_option { ($F:expr, $EXPR:expr) => {{ match $EXPR { diff --git a/datafusion/core/src/catalog/information_schema.rs b/datafusion/core/src/catalog/information_schema.rs index a9d4590a5e28..410d22a64f6d 100644 --- a/datafusion/core/src/catalog/information_schema.rs +++ b/datafusion/core/src/catalog/information_schema.rs @@ -186,7 +186,7 @@ impl InformationSchemaConfig { &schema_name, &table_name, field_position, - field, + &field.as_ref().clone().into(), ) } } diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index afcfe6615c0a..875c8cc907da 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -46,8 +46,9 @@ use crate::prelude::SessionContext; use arrow::array::{Array, ArrayRef, Int64Array, StringArray}; use arrow::compute::{cast, concat}; use arrow::datatypes::{DataType, Field}; -use arrow_schema::{Schema, SchemaRef}; +use arrow_schema::Schema; use datafusion_common::config::{CsvOptions, JsonOptions}; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::{ plan_err, Column, DFSchema, DataFusionError, ParamValues, SchemaError, UnnestOptions, }; @@ -58,8 +59,8 @@ use datafusion_expr::{ use datafusion_functions_aggregate::expr_fn::{avg, count, median, stddev, sum}; use async_trait::async_trait; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// Contains options that control how data is /// written out from a DataFrame @@ -613,7 +614,7 @@ impl DataFrame { //define describe column let mut describe_schemas = vec![Field::new("describe", DataType::Utf8, false)]; describe_schemas.extend(original_schema_fields.clone().map(|field| { - if field.data_type().is_numeric() { + if field.data_type().logical().is_numeric() { Field::new(field.name(), DataType::Float64, true) } else { Field::new(field.name(), DataType::Utf8, true) @@ -649,7 +650,7 @@ impl DataFrame { vec![], original_schema_fields .clone() - .filter(|f| f.data_type().is_numeric()) + .filter(|f| f.data_type().logical().is_numeric()) .map(|f| avg(col(f.name())).alias(f.name())) .collect::>(), ), @@ -658,7 +659,7 @@ impl DataFrame { vec![], original_schema_fields .clone() - .filter(|f| f.data_type().is_numeric()) + .filter(|f| f.data_type().logical().is_numeric()) .map(|f| stddev(col(f.name())).alias(f.name())) .collect::>(), ), @@ -668,7 +669,7 @@ impl DataFrame { original_schema_fields .clone() .filter(|f| { - !matches!(f.data_type(), LogicalType::Binary | LogicalType::Boolean) + !matches!(f.data_type().logical(), LogicalType::Binary | LogicalType::Boolean) }) .map(|f| min(col(f.name())).alias(f.name())) .collect::>(), @@ -679,7 +680,7 @@ impl DataFrame { original_schema_fields .clone() .filter(|f| { - !matches!(f.data_type(), LogicalType::Binary | LogicalType::Boolean) + !matches!(f.data_type().logical(), LogicalType::Binary | LogicalType::Boolean) }) .map(|f| max(col(f.name())).alias(f.name())) .collect::>(), @@ -689,7 +690,7 @@ impl DataFrame { vec![], original_schema_fields .clone() - .filter(|f| f.data_type().is_numeric()) + .filter(|f| f.data_type().logical().is_numeric()) .map(|f| median(col(f.name())).alias(f.name())) .collect::>(), ), @@ -714,7 +715,7 @@ impl DataFrame { { let column = batchs[0].column_by_name(field.name()).unwrap(); - if field.data_type().is_numeric() { + if field.data_type().logical().is_numeric() { cast(column, &DataType::Float64)? } else { cast(column, &DataType::Utf8)? @@ -1474,7 +1475,7 @@ impl DataFrame { /// /// The method supports case sensitive rename with wrapping column name into one of following symbols ( " or ' or ` ) /// - /// Alternatively setting Datafusion param `datafusion.sql_parser.enable_ident_normalization` to `false` will enable + /// Alternatively setting Datafusion param `datafusion.sql_parser.enable_ident_normalization` to `false` will enable /// case sensitive rename without need to wrap column name into special symbols /// /// # Example @@ -1649,9 +1650,8 @@ impl TableProvider for DataFrameTableProvider { Ok(vec![TableProviderFilterPushDown::Exact; filters.len()]) } - fn schema(&self) -> SchemaRef { - let schema: Schema = self.plan.schema().as_ref().into(); - Arc::new(schema) + fn schema(&self) -> LogicalSchemaRef { + self.plan.schema().inner().clone() } fn table_type(&self) -> TableType { @@ -1696,8 +1696,8 @@ mod tests { use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name}; use arrow::array::{self, Int32Array}; + use arrow_schema::SchemaRef; use datafusion_common::{Constraint, Constraints}; - use datafusion_common::logical_type::LogicalType; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::{ array_agg, cast, create_udf, expr, lit, BuiltInWindowFunction, @@ -2365,7 +2365,7 @@ mod tests { let field = df.schema().field(0); // There are two columns named 'c', one from the input of the aggregate and the other from the output. // Select should return the column from the output of the aggregate, which is a list. - assert!(matches!(field.data_type(), LogicalType::List(_))); + assert!(matches!(field.data_type().logical(), LogicalType::List(_))); Ok(()) } @@ -3172,7 +3172,7 @@ mod tests { .await? .select_columns(&["c2", "c3"])? .limit(0, Some(1))? - .with_column("sum", cast(col("c2") + col("c3"), LogicalType::Int64))?; + .with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?; let df_results = df.clone().collect().await?; df.clone().show().await?; @@ -3274,7 +3274,7 @@ mod tests { .await? .select_columns(&["c2", "c3"])? .limit(0, Some(1))? - .with_column("sum", cast(col("c2") + col("c3"), LogicalType::Int64))?; + .with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?; let cached_df = df.clone().cache().await?; diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index afc4536f068e..7610fb6e4fb3 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_physical_plan::work_table::WorkTableExec; use crate::{ @@ -67,8 +68,8 @@ impl TableProvider for CteWorkTable { None } - fn schema(&self) -> SchemaRef { - self.table_schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.table_schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 977e681d6641..434c36250ee8 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -22,8 +22,8 @@ use std::sync::Arc; use crate::datasource::TableProvider; -use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, Constraints}; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; /// DataFusion default table source, wrapping TableProvider. @@ -52,7 +52,7 @@ impl TableSource for DefaultTableSource { } /// Get a reference to the schema for this table - fn schema(&self) -> SchemaRef { + fn schema(&self) -> LogicalSchemaRef { self.table_provider.schema() } diff --git a/datafusion/core/src/datasource/empty.rs b/datafusion/core/src/datasource/empty.rs index 5100987520ee..62a800834e73 100644 --- a/datafusion/core/src/datasource/empty.rs +++ b/datafusion/core/src/datasource/empty.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use arrow::datatypes::*; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::project_schema; use crate::datasource::{TableProvider, TableType}; @@ -59,8 +60,8 @@ impl TableProvider for EmptyTable { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 9e3d0f03ee9e..6ea768963798 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -265,7 +265,7 @@ async fn prune_partitions( let df_schema = DFSchema::from_unqualified_fields( partition_cols .iter() - .map(|(n, d)| LogicalField::new(n, d.clone().into(), true)) + .map(|(n, d)| LogicalField::new(n, d.clone(), true)) .collect(), Default::default(), )?; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ec0ee09e9785..e6b3ef5a7f1c 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -41,10 +41,7 @@ use crate::{ use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; -use datafusion_common::{ - config_datafusion_err, internal_err, plan_err, project_schema, Constraints, - SchemaExt, ToDFSchema, -}; +use datafusion_common::{config_datafusion_err, internal_err, plan_err, project_schema, Constraints, ToDFSchema, SchemaExt}; use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; use datafusion_physical_expr::{ @@ -55,7 +52,7 @@ use async_trait::async_trait; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; /// Configuration for creating a [`ListingTable`] #[derive(Debug, Clone)] @@ -723,8 +720,8 @@ impl TableProvider for ListingTable { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.table_schema) + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.table_schema.clone().into()) } fn constraints(&self) -> Option<&Constraints> { @@ -747,7 +744,8 @@ impl TableProvider for ListingTable { // if no files need to be read, return an `EmptyExec` if partitioned_file_lists.is_empty() { - let projected_schema = project_schema(&self.schema(), projection)?; + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); + let projected_schema = project_schema(&schema, projection)?; return Ok(Arc::new(EmptyExec::new(projected_schema))); } @@ -857,9 +855,9 @@ impl TableProvider for ListingTable { input: Arc, overwrite: bool, ) -> Result> { + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); // Check that the schema of the plan matches the schema of this table. - if !self - .schema() + if !schema .logically_equivalent_names_and_types(&input.schema()) { return plan_err!( @@ -898,7 +896,7 @@ impl TableProvider for ListingTable { object_store_url: self.table_paths()[0].object_store(), table_paths: self.table_paths().clone(), file_groups, - output_schema: self.schema(), + output_schema: SchemaRef::new(self.schema().as_ref().clone().into()), table_partition_cols: self.options.table_partition_cols.clone(), overwrite, keep_partition_by_columns, @@ -981,9 +979,10 @@ impl ListingTable { .boxed() .buffered(ctx.config_options().execution.meta_fetch_concurrency); + let schema = SchemaRef::new(self.schema().as_ref().clone().into()); let (files, statistics) = get_statistics_with_limit( files, - self.schema(), + schema, limit, self.options.collect_stat, ) @@ -1252,8 +1251,9 @@ mod tests { .with_schema(file_schema); let table = ListingTable::try_new(config)?; + let table_schema = table.schema().as_ref().clone().into(); assert_eq!( - columns(&table.schema()), + columns(&table_schema), vec!["a".to_owned(), "p1".to_owned()] ); diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 3126e8dee5de..cc5023f1f6b8 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -21,7 +21,6 @@ use std::any::Any; use std::collections::HashMap; use std::fmt::{self, Debug}; use std::sync::Arc; - use crate::datasource::{TableProvider, TableType}; use crate::error::Result; use crate::execution::context::SessionState; @@ -47,6 +46,7 @@ use log::debug; use parking_lot::Mutex; use tokio::sync::RwLock; use tokio::task::JoinSet; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// Type alias for partition data pub type PartitionData = Arc>>; @@ -159,6 +159,7 @@ impl MemTable { } } + let schema = SchemaRef::new(schema.as_ref().clone().into()); let exec = MemoryExec::try_new(&data, schema.clone(), None)?; if let Some(num_partitions) = output_partitions { @@ -192,8 +193,8 @@ impl TableProvider for MemTable { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn constraints(&self) -> Option<&Constraints> { @@ -218,7 +219,7 @@ impl TableProvider for MemTable { } let mut exec = - MemoryExec::try_new(&partitions, self.schema(), projection.cloned())?; + MemoryExec::try_new(&partitions, SchemaRef::new(self.schema().as_ref().clone().into()), projection.cloned())?; let show_sizes = state.config_options().explain.show_sizes; exec = exec.with_show_sizes(show_sizes); @@ -267,8 +268,8 @@ impl TableProvider for MemTable { // Create a physical plan from the logical plan. // Check that the schema of the plan matches the schema of this table. - if !self - .schema() + let schema = SchemaRef::new(self.schema.as_ref().clone()); + if !schema .logically_equivalent_names_and_types(&input.schema()) { return plan_err!( diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index b73bbd8dad66..b7d9336c6c05 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -425,7 +425,6 @@ mod test { use parquet::file::reader::{FileReader, SerializedFileReader}; use rand::prelude::*; - use datafusion_common::logical_type::LogicalType; // We should ignore predicate that read non-primitive columns @@ -474,10 +473,10 @@ mod test { ]); // The parquet file with `file_schema` just has `bigint_col` and `float_col` column, and don't have the `int_col` - let expr = col("bigint_col").eq(cast(col("int_col"), LogicalType::Int64)); + let expr = col("bigint_col").eq(cast(col("int_col"), DataType::Int64)); let expr = logical2physical(&expr, &table_schema); let expected_candidate_expr = - col("bigint_col").eq(cast(lit(ScalarValue::Int32(None)), LogicalType::Int64)); + col("bigint_col").eq(cast(lit(ScalarValue::Int32(None)), DataType::Int64)); let expected_candidate_expr = logical2physical(&expected_candidate_expr, &table_schema); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 87d64c4a43ba..32fd471d9af2 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -412,7 +412,7 @@ mod tests { use crate::physical_plan::metrics::ExecutionPlanMetricsSet; use arrow::datatypes::{DataType, Field}; - use datafusion_common::{Result, logical_type::LogicalType::*}; + use datafusion_common::Result; use datafusion_expr::{cast, col, lit, Expr}; use datafusion_physical_expr::planner::logical2physical; @@ -818,9 +818,9 @@ mod tests { .with_scale(0) .with_precision(9); let schema_descr = get_test_schema_descr(vec![field]); - let expr = cast(col("c1"), Decimal128(11, 2)).gt(cast( + let expr = cast(col("c1"), DataType::Decimal128(11, 2)).gt(cast( lit(ScalarValue::Decimal128(Some(500), 5, 2)), - Decimal128(11, 2), + DataType::Decimal128(11, 2), )); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); @@ -935,7 +935,7 @@ mod tests { .with_byte_len(16); let schema_descr = get_test_schema_descr(vec![field]); // cast the type of c1 to decimal(28,3) - let left = cast(col("c1"), Decimal128(28, 3)); + let left = cast(col("c1"), DataType::Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); @@ -1009,7 +1009,7 @@ mod tests { .with_byte_len(16); let schema_descr = get_test_schema_descr(vec![field]); // cast the type of c1 to decimal(28,3) - let left = cast(col("c1"), Decimal128(28, 3)); + let left = cast(col("c1"), DataType::Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 7c58aded3108..0a8605e55cfa 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -22,10 +22,10 @@ use std::sync::Arc; use async_trait::async_trait; use datafusion_common::{not_impl_err, Constraints, Statistics}; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_expr::{CreateExternalTable, LogicalPlan}; pub use datafusion_expr::{TableProviderFilterPushDown, TableType}; -use crate::arrow::datatypes::SchemaRef; use crate::datasource::listing_table_factory::ListingTableFactory; use crate::datasource::stream::StreamTableFactory; use crate::error::Result; @@ -41,7 +41,7 @@ pub trait TableProvider: Sync + Send { fn as_any(&self) -> &dyn Any; /// Get a reference to the schema for this table - fn schema(&self) -> SchemaRef; + fn schema(&self) -> LogicalSchemaRef; /// Get a reference to the constraints of the table. /// Returns: diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 9cfdb7bb1168..bf078229130e 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -43,6 +43,7 @@ use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use async_trait::async_trait; use futures::StreamExt; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// A [`TableProviderFactory`] for [`StreamTable`] #[derive(Debug, Default)] @@ -308,8 +309,8 @@ impl TableProvider for StreamTable { self } - fn schema(&self) -> SchemaRef { - self.0.source.schema().clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.0.source.schema().clone().into()) } fn constraints(&self) -> Option<&Constraints> { diff --git a/datafusion/core/src/datasource/streaming.rs b/datafusion/core/src/datasource/streaming.rs index 0ba6f85ec3e2..f8aefed7bec8 100644 --- a/datafusion/core/src/datasource/streaming.rs +++ b/datafusion/core/src/datasource/streaming.rs @@ -26,7 +26,7 @@ use async_trait::async_trait; use datafusion_common::{plan_err, Result}; use datafusion_expr::{Expr, TableType}; use log::debug; - +use datafusion_common::logical_type::schema::LogicalSchemaRef; use crate::datasource::TableProvider; use crate::execution::context::SessionState; use crate::physical_plan::streaming::{PartitionStream, StreamingTableExec}; @@ -75,8 +75,8 @@ impl TableProvider for StreamingTable { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 3f024a6b4cb7..648b8411eeed 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -22,6 +22,7 @@ use std::{any::Any, sync::Arc}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; use datafusion_common::Column; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_expr::{LogicalPlanBuilder, TableProviderFilterPushDown}; use crate::{ @@ -82,8 +83,8 @@ impl TableProvider for ViewTable { Some(&self.logical_plan) } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.table_schema) + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.table_schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 793570d4c707..56b51e792bae 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -42,13 +42,14 @@ use crate::physical_optimizer::optimizer::PhysicalOptimizer; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; use crate::{functions, functions_aggregate}; -use arrow_schema::{SchemaRef}; +use arrow_schema::SchemaRef; use async_trait::async_trait; use chrono::{DateTime, Utc}; use datafusion_common::alias::AliasGenerator; use datafusion_common::config::{ConfigExtension, ConfigOptions, TableOptions}; use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; use datafusion_common::file_options::file_type::FileType; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::tree_node::TreeNode; use datafusion_common::{ config_err, not_impl_err, plan_datafusion_err, DFSchema, DataFusionError, @@ -84,7 +85,7 @@ use std::fmt::Debug; use std::sync::Arc; use url::Url; use uuid::Uuid; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; /// Execution context for registering data sources and executing queries. /// See [`SessionContext`] for a higher level API. @@ -1003,7 +1004,7 @@ impl<'a> ContextProvider for SessionContextProvider<'a> { self.state.window_functions().get(name).cloned() } - fn get_variable_type(&self, variable_names: &[String]) -> Option { + fn get_variable_type(&self, variable_names: &[String]) -> Option { if variable_names.is_empty() { return None; } @@ -1220,7 +1221,7 @@ impl<'a> SessionSimplifyProvider<'a> { impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> { fn is_boolean_type(&self, expr: &Expr) -> datafusion_common::Result { - Ok(expr.get_type(self.df_schema)? == LogicalType::Boolean) + Ok(expr.get_type(self.df_schema)?.logical() == &LogicalType::Boolean) } fn nullable(&self, expr: &Expr) -> datafusion_common::Result { @@ -1231,7 +1232,7 @@ impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> { self.state.execution_props() } - fn get_data_type(&self, expr: &Expr) -> datafusion_common::Result { + fn get_data_type(&self, expr: &Expr) -> datafusion_common::Result { expr.get_type(self.df_schema) } } diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 8624a80bfce3..e8f2f34abda0 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -1571,7 +1571,6 @@ mod tests { use datafusion_expr::expr::InList; use datafusion_expr::{cast, is_null, try_cast, Expr}; use datafusion_physical_expr::planner::logical2physical; - use datafusion_common::logical_type::LogicalType; #[derive(Debug, Default)] /// Mock statistic provider for tests @@ -2608,13 +2607,13 @@ mod tests { // test cast(c1 as int64) = 1 // test column on the left - let expr = cast(col("c1"), LogicalType::Int64).eq(lit(ScalarValue::Int64(Some(1)))); + let expr = cast(col("c1"), DataType::Int64).eq(lit(ScalarValue::Int64(Some(1)))); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right - let expr = lit(ScalarValue::Int64(Some(1))).eq(cast(col("c1"), LogicalType::Int64)); + let expr = lit(ScalarValue::Int64(Some(1))).eq(cast(col("c1"), DataType::Int64)); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); @@ -2626,14 +2625,14 @@ mod tests { // test column on the left let expr = - try_cast(col("c1"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(1)))); + try_cast(col("c1"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(1)))); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = - lit(ScalarValue::Int64(Some(1))).lt(try_cast(col("c1"), LogicalType::Int64)); + lit(ScalarValue::Int64(Some(1))).lt(try_cast(col("c1"), DataType::Int64)); let predicate_expr = test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); @@ -2646,7 +2645,7 @@ mod tests { let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); // test cast(c1 as int64) in int64(1, 2, 3) let expr = Expr::InList(InList::new( - Box::new(cast(col("c1"), LogicalType::Int64)), + Box::new(cast(col("c1"), DataType::Int64)), vec![ lit(ScalarValue::Int64(Some(1))), lit(ScalarValue::Int64(Some(2))), @@ -2671,7 +2670,7 @@ mod tests { assert_eq!(predicate_expr.to_string(), expected_expr); let expr = Expr::InList(InList::new( - Box::new(cast(col("c1"), LogicalType::Int64)), + Box::new(cast(col("c1"), DataType::Int64)), vec![ lit(ScalarValue::Int64(Some(1))), lit(ScalarValue::Int64(Some(2))), @@ -2725,7 +2724,7 @@ mod tests { prune_with_expr( // with cast column to other type - cast(col("s1"), LogicalType::Decimal128(14, 3)) + cast(col("s1"), DataType::Decimal128(14, 3)) .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))), &schema, &TestStatistics::new().with( @@ -2740,7 +2739,7 @@ mod tests { prune_with_expr( // with try cast column to other type - try_cast(col("s1"), LogicalType::Decimal128(14, 3)) + try_cast(col("s1"), DataType::Decimal128(14, 3)) .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))), &schema, &TestStatistics::new().with( @@ -2827,7 +2826,7 @@ mod tests { prune_with_expr( // filter with cast - cast(col("s2"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(5)))), + cast(col("s2"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(5)))), &schema, &statistics, &[false, true, true, true], @@ -3055,7 +3054,7 @@ mod tests { prune_with_expr( // cast(i as utf8) <= 0 - cast(col("i"), LogicalType::Utf8).lt_eq(lit("0")), + cast(col("i"), DataType::Utf8).lt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3063,7 +3062,7 @@ mod tests { prune_with_expr( // try_cast(i as utf8) <= 0 - try_cast(col("i"), LogicalType::Utf8).lt_eq(lit("0")), + try_cast(col("i"), DataType::Utf8).lt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3071,7 +3070,7 @@ mod tests { prune_with_expr( // cast(-i as utf8) >= 0 - cast(Expr::Negative(Box::new(col("i"))), LogicalType::Utf8).gt_eq(lit("0")), + cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3079,7 +3078,7 @@ mod tests { prune_with_expr( // try_cast(-i as utf8) >= 0 - try_cast(Expr::Negative(Box::new(col("i"))), LogicalType::Utf8).gt_eq(lit("0")), + try_cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")), &schema, &statistics, expected_ret, @@ -3120,14 +3119,14 @@ mod tests { let expected_ret = &[true, false, false, true, false]; prune_with_expr( - cast(col("i"), LogicalType::Int64).eq(lit(0i64)), + cast(col("i"), DataType::Int64).eq(lit(0i64)), &schema, &statistics, expected_ret, ); prune_with_expr( - try_cast(col("i"), LogicalType::Int64).eq(lit(0i64)), + try_cast(col("i"), DataType::Int64).eq(lit(0i64)), &schema, &statistics, expected_ret, @@ -3150,7 +3149,7 @@ mod tests { let expected_ret = &[true, true, true, true, true]; prune_with_expr( - cast(col("i"), LogicalType::Utf8).eq(lit("0")), + cast(col("i"), DataType::Utf8).eq(lit("0")), &schema, &statistics, expected_ret, @@ -3305,7 +3304,7 @@ mod tests { prune_with_expr( // i > int64(0) - col("i").gt(cast(lit(ScalarValue::Int64(Some(0))), LogicalType::Int32)), + col("i").gt(cast(lit(ScalarValue::Int64(Some(0))), DataType::Int32)), &schema, &statistics, expected_ret, @@ -3313,7 +3312,7 @@ mod tests { prune_with_expr( // cast(i as int64) > int64(0) - cast(col("i"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), + cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), &schema, &statistics, expected_ret, @@ -3321,7 +3320,7 @@ mod tests { prune_with_expr( // try_cast(i as int64) > int64(0) - try_cast(col("i"), LogicalType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), + try_cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), &schema, &statistics, expected_ret, @@ -3329,7 +3328,7 @@ mod tests { prune_with_expr( // `-cast(i as int64) < 0` convert to `cast(i as int64) > -0` - Expr::Negative(Box::new(cast(col("i"), LogicalType::Int64))) + Expr::Negative(Box::new(cast(col("i"), DataType::Int64))) .lt(lit(ScalarValue::Int64(Some(0)))), &schema, &statistics, @@ -3358,7 +3357,7 @@ mod tests { assert_eq!(result_right.to_string(), right_input.to_string()); // cast op lit - let left_input = cast(col("a"), LogicalType::Decimal128(20, 3)); + let left_input = cast(col("a"), DataType::Decimal128(20, 3)); let left_input = logical2physical(&left_input, &schema); let right_input = lit(ScalarValue::Decimal128(Some(12), 20, 3)); let right_input = logical2physical(&right_input, &schema); @@ -3373,7 +3372,7 @@ mod tests { assert_eq!(result_right.to_string(), right_input.to_string()); // try_cast op lit - let left_input = try_cast(col("a"), LogicalType::Int64); + let left_input = try_cast(col("a"), DataType::Int64); let left_input = logical2physical(&left_input, &schema); let right_input = lit(ScalarValue::Int64(Some(12))); let right_input = logical2physical(&right_input, &schema); @@ -3392,7 +3391,7 @@ mod tests { // this cast is not supported let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); let df_schema = DFSchema::try_from(schema.clone()).unwrap(); - let left_input = cast(col("a"), LogicalType::Int64); + let left_input = cast(col("a"), DataType::Int64); let left_input = logical2physical(&left_input, &schema); let right_input = lit(ScalarValue::Int64(Some(12))); let right_input = logical2physical(&right_input, &schema); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 164fa2ed088a..47efaac858ac 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2244,7 +2244,6 @@ mod tests { use arrow::datatypes::{DataType, Field, Int32Type}; use datafusion_common::{assert_contains, DFSchemaRef, TableReference}; use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::LogicalType; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{col, lit, LogicalPlanBuilder, UserDefinedLogicalNodeCore}; @@ -2765,7 +2764,7 @@ mod tests { Self { schema: DFSchemaRef::new( DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", LogicalType::Int32, false)].into(), + vec![LogicalField::new("a", DataType::Int32, false)].into(), HashMap::new(), ) .unwrap(), diff --git a/datafusion/core/src/test/variable.rs b/datafusion/core/src/test/variable.rs index 7cf5a7f3ec2b..ab0de4a11e24 100644 --- a/datafusion/core/src/test/variable.rs +++ b/datafusion/core/src/test/variable.rs @@ -20,7 +20,8 @@ use crate::error::Result; use crate::scalar::ScalarValue; use crate::variable::VarProvider; -use datafusion_common::logical_type::LogicalType; +use arrow_schema::DataType; +use datafusion_common::logical_type::TypeRelation; /// System variable #[derive(Default, Debug)] @@ -40,8 +41,8 @@ impl VarProvider for SystemVar { Ok(ScalarValue::from(s)) } - fn get_type(&self, _: &[String]) -> Option { - Some(LogicalType::Utf8) + fn get_type(&self, _: &[String]) -> Option { + Some(DataType::Utf8.into()) } } @@ -67,11 +68,11 @@ impl VarProvider for UserDefinedVar { } } - fn get_type(&self, var_names: &[String]) -> Option { + fn get_type(&self, var_names: &[String]) -> Option { if var_names[0] != "@integer" { - Some(LogicalType::Utf8) + Some(DataType::Utf8.into()) } else { - Some(LogicalType::Int32) + Some(DataType::Int32.into()) } } } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 059fa8fc6da7..7bef5aad85a1 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -51,7 +51,7 @@ use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use futures::Stream; use tempfile::TempDir; - +use datafusion_common::logical_type::schema::LogicalSchemaRef; // backwards compatibility #[cfg(feature = "parquet")] pub use datafusion_common::test_util::parquet_test_data; @@ -203,8 +203,8 @@ impl TableProvider for TestTableProvider { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index eebc946ccb68..b79b655bd2fd 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -44,6 +44,7 @@ use datafusion_physical_plan::{ExecutionMode, PlanProperties}; use async_trait::async_trait; use futures::stream::Stream; +use datafusion_common::logical_type::schema::LogicalSchemaRef; mod provider_filter_pushdown; mod statistics; @@ -80,6 +81,7 @@ struct CustomExecutionPlan { impl CustomExecutionPlan { fn new(projection: Option>) -> Self { let schema = TEST_CUSTOM_SCHEMA_REF!(); + let schema = SchemaRef::new(schema.as_ref().clone().into()); let schema = project_schema(&schema, projection.as_ref()).expect("projected schema"); let cache = Self::compute_properties(schema); @@ -202,8 +204,8 @@ impl TableProvider for CustomTableProvider { self } - fn schema(&self) -> SchemaRef { - TEST_CUSTOM_SCHEMA_REF!() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(TEST_CUSTOM_SCHEMA_REF!().as_ref().clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index b5506b7c12f6..d29225cc9f15 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -39,6 +39,7 @@ use datafusion_functions_aggregate::expr_fn::count; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; fn create_batch(value: i32, num_rows: usize) -> Result { let mut builder = Int32Builder::with_capacity(num_rows); @@ -152,8 +153,8 @@ impl TableProvider for CustomProvider { self } - fn schema(&self) -> SchemaRef { - self.zero_batch.schema() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.zero_batch.schema().clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 2d42b03bfed8..aa7e1089d8f9 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -36,6 +36,7 @@ use datafusion_common::{project_schema, stats::Precision}; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// This is a testing structure for statistics /// It will act both as a table provider and execution plan @@ -79,8 +80,8 @@ impl TableProvider for StatisticsValidation { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index b1d615405cac..013b8c9b578d 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -31,7 +31,6 @@ use datafusion::prelude::*; use datafusion::assert_batches_eq; use datafusion_common::{DFSchema, ScalarValue}; -use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::Alias; use datafusion_expr::ExprSchemable; use datafusion_functions_aggregate::expr_fn::{approx_median, approx_percentile_cont}; @@ -377,7 +376,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { // the arg2 parameter is a complex expr, but it can be evaluated to the literal value let alias_expr = Expr::Alias(Alias::new( - cast(lit(0.5), LogicalType::Float32), + cast(lit(0.5), DataType::Float32), None::<&str>, "arg_2".to_string(), )); @@ -950,7 +949,7 @@ async fn test_fn_substr() -> Result<()> { #[tokio::test] async fn test_cast() -> Result<()> { - let expr = cast(col("b"), LogicalType::Float64); + let expr = cast(col("b"), DataType::Float64); let expected = [ "+--------+", "| test.b |", @@ -1053,7 +1052,7 @@ async fn test_fn_decode() -> Result<()> { let expr = decode(encode(col("a"), lit("hex")), lit("hex")) // need to cast to utf8 otherwise the default display of binary array is hex // so it looks like nothing is done - .cast_to(&LogicalType::Utf8, &df_schema)?; + .cast_to(&DataType::Utf8.into(), &df_schema)?; let expected = [ "+------------------------------------------------+", diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index d7228dcf3689..225d3023f520 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -47,7 +47,7 @@ use datafusion::prelude::{CsvReadOptions, ParquetReadOptions}; use datafusion::test_util::{parquet_test_data, populate_csv_partitions}; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; use datafusion_common::{assert_contains, DataFusionError, ScalarValue, UnnestOptions}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_expr::expr::{GroupingSet, Sort}; @@ -257,7 +257,7 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { scalar_subquery(Arc::new( ctx.table("t2") .await? - .filter(out_ref_col(LogicalType::UInt32, "t1.a").eq(col("t2.a")))? + .filter(out_ref_col(DataType::UInt32, "t1.a").eq(col("t2.a")))? .aggregate(vec![], vec![count(wildcard())])? .select(vec![col(count(wildcard()).to_string())])? .into_unoptimized_plan(), @@ -747,8 +747,8 @@ async fn join_with_alias_filter() -> Result<()> { // filter: t1.a + CAST(Int64(1), UInt32) = t2.a + CAST(Int64(2), UInt32) as t1.a + 1 = t2.a + 2 let filter = Expr::eq( - col("t1.a") + lit(3i64).cast_to(&LogicalType::UInt32, &t1_schema)?, - col("t2.a") + lit(1i32).cast_to(&LogicalType::UInt32, &t2_schema)?, + col("t1.a") + lit(3i64).cast_to(&DataType::UInt32.into(), &t1_schema)?, + col("t2.a") + lit(1i32).cast_to(&DataType::UInt32.into(), &t2_schema)?, ) .alias("t1.b + 1 = t2.a + 2"); @@ -1623,7 +1623,7 @@ async fn consecutive_projection_same_schema() -> Result<()> { // Add `t` column full of nulls let df = df - .with_column("t", cast(Expr::Literal(ScalarValue::Null), LogicalType::Int32)) + .with_column("t", cast(Expr::Literal(ScalarValue::Null), DataType::Int32)) .unwrap(); df.clone().show().await.unwrap(); @@ -1926,8 +1926,8 @@ impl VarProvider for HardcodedIntProvider { Ok(ScalarValue::Int64(Some(1234))) } - fn get_type(&self, _: &[String]) -> Option { - Some(LogicalType::Int64) + fn get_type(&self, _: &[String]) -> Option { + Some(DataType::Int64.into()) } } diff --git a/datafusion/core/tests/expr_api/parse_sql_expr.rs b/datafusion/core/tests/expr_api/parse_sql_expr.rs index e082aaed322b..4cd617dd6989 100644 --- a/datafusion/core/tests/expr_api/parse_sql_expr.rs +++ b/datafusion/core/tests/expr_api/parse_sql_expr.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. +use arrow_schema::DataType; use datafusion::prelude::{CsvReadOptions, SessionContext}; use datafusion_common::{DFSchemaRef, Result, ToDFSchema}; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::Expr; use datafusion_sql::unparser::Unparser; @@ -30,9 +30,9 @@ use datafusion_sql::unparser::Unparser; /// s: Float32 fn schema() -> DFSchemaRef { LogicalSchema::new(vec![ - LogicalField::new("a", LogicalType::Int32, true), - LogicalField::new("b", LogicalType::Int32, false), - LogicalField::new("c", LogicalType::Float32, false), + LogicalField::new("a", DataType::Int32, true), + LogicalField::new("b", DataType::Int32, false), + LogicalField::new("c", DataType::Float32, false), ]) .to_dfschema_ref() .unwrap() diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 38e1e8cd1551..73de1cc2cfdb 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -23,6 +23,7 @@ use arrow_buffer::IntervalDayTime; use chrono::{DateTime, TimeZone, Utc}; use datafusion::{error::Result, execution::context::ExecutionProps, prelude::*}; use datafusion_common::cast::as_int32_array; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::ScalarValue; use datafusion_common::{DFSchemaRef, ToDFSchema}; use datafusion_expr::expr::ScalarFunction; @@ -38,7 +39,7 @@ use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyExpress use datafusion_optimizer::{OptimizerContext, OptimizerRule}; use std::sync::Arc; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_common::logical_type::schema::LogicalSchema; /// In order to simplify expressions, DataFusion must have information @@ -58,7 +59,7 @@ struct MyInfo { impl SimplifyInfo for MyInfo { fn is_boolean_type(&self, expr: &Expr) -> Result { Ok(matches!( - expr.get_type(self.schema.as_ref())?, + expr.get_type(self.schema.as_ref())?.logical(), LogicalType::Boolean )) } @@ -71,7 +72,7 @@ impl SimplifyInfo for MyInfo { &self.execution_props } - fn get_data_type(&self, expr: &Expr) -> Result { + fn get_data_type(&self, expr: &Expr) -> Result { expr.get_type(self.schema.as_ref()) } } @@ -92,9 +93,9 @@ impl From for MyInfo { /// s: Utf8 fn schema() -> DFSchemaRef { LogicalSchema::new(vec![ - LogicalField::new("a", LogicalType::Int32, true), - LogicalField::new("b", LogicalType::Int32, false), - LogicalField::new("s", LogicalType::Utf8, false), + LogicalField::new("a", DataType::Int32, true), + LogicalField::new("b", DataType::Int32, false), + LogicalField::new("s", DataType::Utf8, false), ]) .to_dfschema_ref() .unwrap() @@ -193,7 +194,7 @@ fn make_udf_add(volatility: Volatility) -> Arc { } fn cast_to_int64_expr(expr: Expr) -> Expr { - Expr::Cast(Cast::new(expr.into(), LogicalType::Int64)) + Expr::Cast(Cast::new(expr.into(), DataType::Int64)) } fn to_timestamp_expr(arg: impl Into) -> Expr { @@ -284,7 +285,7 @@ fn select_date_plus_interval() -> Result<()> { let schema = table_scan.schema(); let date_plus_interval_expr = to_timestamp_expr(ts_string) - .cast_to(&LogicalType::Date32, schema)? + .cast_to(&DataType::Date32.into(), schema)? + Expr::Literal(ScalarValue::IntervalDayTime(Some(IntervalDayTime { days: 123, milliseconds: 0, @@ -487,14 +488,14 @@ fn multiple_now() -> Result<()> { fn expr_test_schema() -> DFSchemaRef { LogicalSchema::new(vec![ - LogicalField::new("c1", LogicalType::Utf8, true), - LogicalField::new("c2", LogicalType::Boolean, true), - LogicalField::new("c3", LogicalType::Int64, true), - LogicalField::new("c4", LogicalType::UInt32, true), - LogicalField::new("c1_non_null", LogicalType::Utf8, false), - LogicalField::new("c2_non_null", LogicalType::Boolean, false), - LogicalField::new("c3_non_null", LogicalType::Int64, false), - LogicalField::new("c4_non_null", LogicalType::UInt32, false), + LogicalField::new("c1", DataType::Utf8, true), + LogicalField::new("c2", DataType::Boolean, true), + LogicalField::new("c3", DataType::Int64, true), + LogicalField::new("c4", DataType::UInt32, true), + LogicalField::new("c1_non_null", DataType::Utf8, false), + LogicalField::new("c2_non_null", DataType::Boolean, false), + LogicalField::new("c3_non_null", DataType::Int64, false), + LogicalField::new("c4_non_null", DataType::UInt32, false), ]) .to_dfschema_ref() .unwrap() @@ -691,8 +692,8 @@ fn test_simplify_concat() { #[test] fn test_simplify_cycles() { // cast(now() as int64) < cast(to_timestamp(0) as int64) + i64::MAX - let expr = cast(now(), LogicalType::Int64) - .lt(cast(to_timestamp(vec![lit(0)]), LogicalType::Int64) + lit(i64::MAX)); + let expr = cast(now(), DataType::Int64) + .lt(cast(to_timestamp(vec![lit(0)]), DataType::Int64) + lit(i64::MAX)); let expected = lit(true); test_simplify_with_cycle_count(expr, expected, 3); } diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index f61ee5d9ab98..1c7bf0391ade 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -43,6 +43,7 @@ use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use datafusion_common::{assert_contains, Result}; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_execution::TaskContext; use test_utils::AccessLogGenerator; @@ -732,8 +733,8 @@ impl TableProvider for SortedTableProvider { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { @@ -748,7 +749,7 @@ impl TableProvider for SortedTableProvider { _limit: Option, ) -> Result> { let mem_exec = - MemoryExec::try_new(&self.batches, self.schema(), projection.cloned())? + MemoryExec::try_new(&self.batches, self.schema.clone(), projection.cloned())? .with_sort_information(self.sort_information.clone()); Ok(Arc::new(mem_exec)) diff --git a/datafusion/core/tests/optimizer_integration.rs b/datafusion/core/tests/optimizer_integration.rs index 40c91c4bd4ff..0573ce428adc 100644 --- a/datafusion/core/tests/optimizer_integration.rs +++ b/datafusion/core/tests/optimizer_integration.rs @@ -26,7 +26,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use arrow_schema::{Fields, SchemaBuilder}; use datafusion_common::config::ConfigOptions; use datafusion_common::logical_type::schema::LogicalSchemaRef; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{plan_err, DFSchema, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::{Interval, NullableInterval}; @@ -205,7 +205,7 @@ impl ContextProvider for MyContextProvider { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -239,8 +239,8 @@ impl TableSource for MyTableSource { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } } diff --git a/datafusion/core/tests/sql/create_drop.rs b/datafusion/core/tests/sql/create_drop.rs index 2174009b8557..9ab3098efb49 100644 --- a/datafusion/core/tests/sql/create_drop.rs +++ b/datafusion/core/tests/sql/create_drop.rs @@ -18,7 +18,7 @@ use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::test_util::TestTableFactory; - +use datafusion_common::logical_type::ExtensionType; use super::*; #[tokio::test] @@ -67,9 +67,9 @@ async fn create_external_table_with_ddl() -> Result<()> { assert_eq!(3, table_schema.fields().len()); - assert_eq!(&DataType::Int32, table_schema.field(0).data_type()); - assert_eq!(&DataType::Utf8, table_schema.field(1).data_type()); - assert_eq!(&DataType::Boolean, table_schema.field(2).data_type()); + assert_eq!(&DataType::Int32, table_schema.field(0).data_type().physical()); + assert_eq!(&DataType::Utf8, table_schema.field(1).data_type().physical()); + assert_eq!(&DataType::Boolean, table_schema.field(2).data_type().physical()); Ok(()) } diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 2603290442b6..886124312d20 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -39,8 +39,7 @@ use datafusion_expr::{ use datafusion_functions_array::range::range_udf; use parking_lot::Mutex; use sqlparser::ast::Ident; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::ExtensionType; /// test that casting happens on udfs. /// c11 is f32, but `custom_sqrt` requires f64. Casting happens but the logical plan and @@ -520,14 +519,14 @@ impl ScalarUDFImpl for CastToI64UDF { // SimplifyInfo so we have to replicate some of the casting logic here. let source_type = info.get_data_type(&arg)?; - let new_expr = if source_type == LogicalType::Int64 { + let new_expr = if source_type.physical() == &DataType::Int64 { // the argument's data type is already the correct type arg } else { // need to use an actual cast to get the correct type Expr::Cast(datafusion_expr::Cast { expr: Box::new(arg), - data_type: LogicalType::Int64, + data_type: DataType::Int64.into(), }) }; // return the newly written argument to DataFusion @@ -647,7 +646,7 @@ impl ScalarUDFImpl for TakeUDF { ); }; - arg_exprs.get(take_idx).unwrap().get_type(schema).map(|t| t.physical_type()) + arg_exprs.get(take_idx).unwrap().get_type(schema).map(|t| t.physical().clone()) } // The actual implementation @@ -689,8 +688,8 @@ async fn verify_udf_return_type() -> Result<()> { // The output schema should be // * type of column smallint_col (int32) // * type of column double_col (float64) - assert_eq!(schema.field(0).data_type(), &LogicalType::Int32); - assert_eq!(schema.field(1).data_type(), &LogicalType::Float64); + assert_eq!(schema.field(0).data_type().physical(), &DataType::Int32); + assert_eq!(schema.field(1).data_type().physical(), &DataType::Float64); let expected = [ "+-------+-------+", @@ -838,7 +837,7 @@ impl TryFrom for ScalarFunctionWrapper { return_type: definition .return_type .expect("Return type has to be defined!") - .physical_type(), + .physical().clone(), // TODO(@notfilippo): avoid conversion to physical type signature: Signature::exact( definition @@ -846,7 +845,7 @@ impl TryFrom for ScalarFunctionWrapper { .unwrap_or_default() .into_iter() // TODO(@notfilippo): avoid conversion to physical type - .map(|a| a.data_type.physical_type()) + .map(|a| a.data_type.physical().clone()) .collect(), definition .params @@ -995,10 +994,10 @@ async fn create_scalar_function_from_sql_statement_postgres_syntax() -> Result<( value: "name".into(), quote_style: None, }), - data_type: LogicalType::Utf8, + data_type: DataType::Utf8.into(), default_expr: None, }]), - return_type: Some(LogicalType::Int32), + return_type: Some(DataType::Int32.into()), params: CreateFunctionBody { language: Some(Ident { value: "plrust".into(), diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index 1e8d30cab638..2052ea90be6a 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -35,6 +35,7 @@ use std::fs::File; use std::io::Seek; use std::path::Path; use std::sync::Arc; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// test simple udtf with define read_csv with parameters #[tokio::test] @@ -117,8 +118,8 @@ impl TableProvider for SimpleCsvTable { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { @@ -154,7 +155,7 @@ impl TableProvider for SimpleCsvTable { }; Ok(Arc::new(MemoryExec::try_new( &[batches], - TableProvider::schema(self), + self.schema.clone(), projection.cloned(), )?)) } diff --git a/datafusion/expr/src/conditional_expressions.rs b/datafusion/expr/src/conditional_expressions.rs index 176a2002501a..4a96442bcc4c 100644 --- a/datafusion/expr/src/conditional_expressions.rs +++ b/datafusion/expr/src/conditional_expressions.rs @@ -18,9 +18,10 @@ //! Conditional expressions use crate::expr::Case; use crate::{expr_schema::ExprSchemable, Expr}; +use arrow::datatypes::DataType; use datafusion_common::{plan_err, DFSchema, Result}; use std::collections::HashSet; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; /// Helper struct for building [Expr::Case] pub struct CaseBuilder { @@ -70,18 +71,18 @@ impl CaseBuilder { then_expr.push(e.as_ref().to_owned()); } - let then_types: Vec = then_expr + let then_types: Vec = then_expr .iter() .map(|e| match e { Expr::Literal(_) => e.get_type(&DFSchema::empty()), - _ => Ok(LogicalType::Null), + _ => Ok(DataType::Null.into()), }) .collect::>>()?; - if then_types.contains(&LogicalType::Null) { + if then_types.contains(&DataType::Null.into()) { // cannot verify types until execution type } else { - let unique_types: HashSet<&LogicalType> = then_types.iter().collect(); + let unique_types: HashSet<&TypeRelation> = then_types.iter().collect(); if unique_types.len() != 1 { return plan_err!( "CASE expression 'then' values had multiple data types: {unique_types:?}" diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 0c084c8cadb7..3893ee750f59 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -42,7 +42,7 @@ use datafusion_common::{ }; use sqlparser::ast::NullTreatment; use datafusion_common::logical_type::field::LogicalFieldRef; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; /// Represents logical expressions such as `A + 1`, or `CAST(c1 AS int)`. /// @@ -225,7 +225,7 @@ pub enum Expr { /// A named reference to a qualified filed in a schema. Column(Column), /// A named reference to a variable in a registry. - ScalarVariable(LogicalType, Vec), + ScalarVariable(TypeRelation, Vec), /// A constant value. Literal(ScalarValue), /// A binary expression such as "age > 21" @@ -319,7 +319,7 @@ pub enum Expr { Placeholder(Placeholder), /// A place holder which hold a reference to a qualified field /// in the outer query, used for correlated sub queries. - OuterReferenceColumn(LogicalType, Column), + OuterReferenceColumn(TypeRelation, Column), /// Unnest expression Unnest(Unnest), } @@ -561,13 +561,13 @@ pub struct Cast { /// The expression being cast pub expr: Box, /// The `LogicalType` the expression will yield - pub data_type: LogicalType, + pub data_type: TypeRelation, } impl Cast { /// Create a new Cast expression - pub fn new(expr: Box, data_type: LogicalType) -> Self { - Self { expr, data_type } + pub fn new(expr: Box, data_type: impl Into) -> Self { + Self { expr, data_type: data_type.into() } } } @@ -577,13 +577,13 @@ pub struct TryCast { /// The expression being cast pub expr: Box, /// The `LogicalType` the expression will yield - pub data_type: LogicalType, + pub data_type: TypeRelation, } impl TryCast { /// Create a new TryCast expression - pub fn new(expr: Box, data_type: LogicalType) -> Self { - Self { expr, data_type } + pub fn new(expr: Box, data_type: impl Into) -> Self { + Self { expr, data_type: data_type.into() } } } @@ -928,12 +928,12 @@ pub struct Placeholder { /// The identifier of the parameter, including the leading `$` (e.g, `"$1"` or `"$foo"`) pub id: String, /// The type the parameter will be filled in with - pub data_type: Option, + pub data_type: Option, } impl Placeholder { /// Create a new Placeholder expression - pub fn new(id: String, data_type: Option) -> Self { + pub fn new(id: String, data_type: Option) -> Self { Self { id, data_type } } } @@ -2224,7 +2224,7 @@ mod test { fn format_cast() -> Result<()> { let expr = Expr::Cast(Cast { expr: Box::new(Expr::Literal(ScalarValue::Float32(Some(1.23)))), - data_type: LogicalType::Utf8, + data_type: DataType::Utf8.into(), }); let expected_canonical = "CAST(Float32(1.23) AS Utf8)"; assert_eq!(expected_canonical, expr.canonical_name()); @@ -2257,7 +2257,7 @@ mod test { fn test_collect_expr() -> Result<()> { // single column { - let expr = &Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Float64)); + let expr = &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)); let columns = expr.column_refs(); assert_eq!(1, columns.len()); assert!(columns.contains(&Column::from_name("a"))); diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 2db73e5c413b..0821b475b183 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -40,7 +40,7 @@ use std::any::Any; use std::fmt::Debug; use std::ops::Not; use std::sync::Arc; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; /// Create a column expression based on a qualified or unqualified column name. Will /// normalize unquoted identifiers according to SQL rules (identifiers will become lowercase). @@ -63,8 +63,8 @@ pub fn col(ident: impl Into) -> Expr { /// Create an out reference column which hold a reference that has been resolved to a field /// outside of the current plan. -pub fn out_ref_col(dt: LogicalType, ident: impl Into) -> Expr { - Expr::OuterReferenceColumn(dt, ident.into()) +pub fn out_ref_col(dt: impl Into, ident: impl Into) -> Expr { + Expr::OuterReferenceColumn(dt.into(), ident.into()) } /// Create an unqualified column expression from the provided name, without normalizing @@ -309,13 +309,13 @@ pub fn rollup(exprs: Vec) -> Expr { } /// Create a cast expression -pub fn cast(expr: Expr, data_type: LogicalType) -> Expr { - Expr::Cast(Cast::new(Box::new(expr), data_type)) +pub fn cast(expr: Expr, data_type: impl Into) -> Expr { + Expr::Cast(Cast::new(Box::new(expr), data_type.into())) } /// Create a try cast expression -pub fn try_cast(expr: Expr, data_type: LogicalType) -> Expr { - Expr::TryCast(TryCast::new(Box::new(expr), data_type)) +pub fn try_cast(expr: Expr, data_type: impl Into) -> Expr { + Expr::TryCast(TryCast::new(Box::new(expr), data_type.into())) } /// Create is null expression diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index ea3d6f353aa9..217477fa1010 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -286,8 +286,8 @@ mod test { use super::*; use crate::expr::Sort; use crate::{col, lit, Cast}; + use arrow::datatypes::DataType; use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::LogicalType; use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_common::ScalarValue; @@ -409,7 +409,7 @@ mod test { ) -> DFSchema { let fields = fields .iter() - .map(|f| Arc::new(LogicalField::new(f.to_string(), LogicalType::Int8, false))) + .map(|f| Arc::new(LogicalField::new(f.to_string(), DataType::Int8, false))) .collect::>(); let schema = Arc::new(LogicalSchema::new(fields)); DFSchema::from_field_specific_qualified_schema(qualifiers, &schema.into()).unwrap() @@ -442,7 +442,7 @@ mod test { // cast data types test_rewrite( col("a"), - Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Int32)), + Expr::Cast(Cast::new(Box::new(col("a")), DataType::Int32)), ); // change literal type from i32 to i64 diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index 8084995bdf63..dbf7236e8a85 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -154,7 +154,6 @@ mod test { use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::logical_type::LogicalType; use crate::{ cast, col, lit, logical_plan::builder::LogicalTableSource, min, test::function_stub::avg, try_cast, LogicalPlanBuilder, @@ -270,13 +269,13 @@ mod test { let cases = vec![ TestCase { desc: "Cast is preserved by rewrite_sort_cols_by_aggs", - input: sort(cast(col("c2"), LogicalType::Int64)), - expected: sort(cast(col("c2").alias("c2"), LogicalType::Int64)), + input: sort(cast(col("c2"), DataType::Int64)), + expected: sort(cast(col("c2").alias("c2"), DataType::Int64)), }, TestCase { desc: "TryCast is preserved by rewrite_sort_cols_by_aggs", - input: sort(try_cast(col("c2"), LogicalType::Int64)), - expected: sort(try_cast(col("c2").alias("c2"), LogicalType::Int64)), + input: sort(try_cast(col("c2"), DataType::Int64)), + expected: sort(try_cast(col("c2").alias("c2"), DataType::Int64)), }, ]; diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 90204ac0dcd0..e7f31ca6e584 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -26,20 +26,21 @@ use crate::type_coercion::functions::{ }; use crate::{utils, LogicalPlan, Projection, Subquery, WindowFunctionDefinition}; use arrow::compute::can_cast_types; +use arrow::datatypes::DataType; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::{ internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, ExprSchema, Result, TableReference, }; use std::collections::HashMap; use std::sync::Arc; -use datafusion_common::logical_type::extension::ExtensionType; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; /// trait to allow expr to typable with respect to a schema pub trait ExprSchemable { /// given a schema, return the type of the expr - fn get_type(&self, schema: &dyn ExprSchema) -> Result; + fn get_type(&self, schema: &dyn ExprSchema) -> Result; /// given a schema, return the nullability of the expr fn nullable(&self, input_schema: &dyn ExprSchema) -> Result; @@ -54,11 +55,11 @@ pub trait ExprSchemable { ) -> Result<(Option, Arc)>; /// cast to a type with respect to a schema - fn cast_to(self, cast_to_type: &LogicalType, schema: &dyn ExprSchema) -> Result; + fn cast_to(self, cast_to_type: &TypeRelation, schema: &dyn ExprSchema) -> Result; /// given a schema, return the type and nullability of the expr fn data_type_and_nullable(&self, schema: &dyn ExprSchema) - -> Result<(LogicalType, bool)>; + -> Result<(TypeRelation, bool)>; } impl ExprSchemable for Expr { @@ -100,7 +101,7 @@ impl ExprSchemable for Expr { /// expression refers to a column that does not exist in the /// schema, or when the expression is incorrectly typed /// (e.g. `[utf8] + [bool]`). - fn get_type(&self, schema: &dyn ExprSchema) -> Result { + fn get_type(&self, schema: &dyn ExprSchema) -> Result { match self { Expr::Alias(Alias { expr, name, .. }) => match &**expr { Expr::Placeholder(Placeholder { data_type, .. }) => match &data_type { @@ -120,8 +121,8 @@ impl ExprSchemable for Expr { Expr::Unnest(Unnest { expr }) => { let arg_data_type = expr.get_type(schema)?; // Unnest's output type is the inner type of the list - match arg_data_type{ - LogicalType::List(field) | LogicalType::LargeList(field) | LogicalType::FixedSizeList(field, _) =>{ + match arg_data_type.logical() { + LogicalType::List(field) =>{ Ok(field.data_type().clone()) } LogicalType::Struct(_) => { @@ -144,7 +145,7 @@ impl ExprSchemable for Expr { // TODO(@notfilippo): not convert to DataType let arg_data_types = arg_data_types .into_iter() - .map(|e| e.physical_type()) + .map(|e| e.physical().clone()) .collect::>(); // verify that function is invoked with correct number and type of arguments as defined in `TypeSignature` @@ -172,7 +173,7 @@ impl ExprSchemable for Expr { // TODO(@notfilippo): not convert to DataType let data_types = data_types .into_iter() - .map(|e| e.physical_type()) + .map(|e| e.physical().clone()) .collect::>(); let nullability = args .iter() @@ -210,7 +211,7 @@ impl ExprSchemable for Expr { // TODO(@notfilippo): not convert to DataType let data_types = data_types .into_iter() - .map(|e| e.physical_type()) + .map(|e| e.physical().clone()) .collect::>(); match func_def { AggregateFunctionDefinition::BuiltIn(fun) => { @@ -244,7 +245,7 @@ impl ExprSchemable for Expr { | Expr::IsUnknown(_) | Expr::IsNotTrue(_) | Expr::IsNotFalse(_) - | Expr::IsNotUnknown(_) => Ok(LogicalType::Boolean), + | Expr::IsNotUnknown(_) => Ok(DataType::Boolean.into()), Expr::ScalarSubquery(subquery) => { Ok(subquery.subquery.schema().field(0).data_type().clone()) } @@ -253,8 +254,8 @@ impl ExprSchemable for Expr { ref right, ref op, // TODO(@notfilippo): do not convert to physical type - }) => Ok(get_result_type(&left.get_type(schema)?.physical_type(), op, &right.get_type(schema)?.physical_type())?.into()), - Expr::Like { .. } | Expr::SimilarTo { .. } => Ok(LogicalType::Boolean), + }) => Ok(get_result_type(&left.get_type(schema)?.physical(), op, &right.get_type(schema)?.physical())?.into()), + Expr::Like { .. } | Expr::SimilarTo { .. } => Ok(DataType::Boolean.into()), Expr::Placeholder(Placeholder { data_type, .. }) => { data_type.clone().ok_or_else(|| { plan_datafusion_err!("Placeholder type could not be resolved. Make sure that the placeholder is bound to a concrete type, e.g. by providing parameter values.") @@ -264,12 +265,12 @@ impl ExprSchemable for Expr { // Wildcard do not really have a type and do not appear in projections match qualifier { Some(_) => internal_err!("QualifiedWildcard expressions are not valid in a logical query plan"), - None => Ok(LogicalType::Null) + None => Ok(DataType::Null.into()) } } Expr::GroupingSet(_) => { // grouping sets do not really have a type and do not appear in projections - Ok(LogicalType::Null) + Ok(DataType::Null.into()) } } } @@ -412,7 +413,7 @@ impl ExprSchemable for Expr { fn data_type_and_nullable( &self, schema: &dyn ExprSchema, - ) -> Result<(LogicalType, bool)> { + ) -> Result<(TypeRelation, bool)> { match self { Expr::Alias(Alias { expr, name, .. }) => match &**expr { Expr::Placeholder(Placeholder { data_type, .. }) => match &data_type { @@ -440,7 +441,7 @@ impl ExprSchemable for Expr { | Expr::IsNotTrue(_) | Expr::IsNotFalse(_) | Expr::IsNotUnknown(_) - | Expr::Exists { .. } => Ok((LogicalType::Boolean, false)), + | Expr::Exists { .. } => Ok((DataType::Boolean.into(), false)), Expr::ScalarSubquery(subquery) => Ok(( subquery.subquery.schema().field(0).data_type().clone(), subquery.subquery.schema().field(0).is_nullable(), @@ -453,7 +454,7 @@ impl ExprSchemable for Expr { let left = left.data_type_and_nullable(schema)?; let right = right.data_type_and_nullable(schema)?; // TODO(@notfilippo): do not convert to physical type - Ok((get_result_type(&left.0.physical_type(), op, &right.0.physical_type())?.into(), left.1 || right.1)) + Ok((get_result_type(&left.0.physical(), op, &right.0.physical())?.into(), left.1 || right.1)) } _ => Ok((self.get_type(schema)?, self.nullable(schema)?)), } @@ -504,7 +505,7 @@ impl ExprSchemable for Expr { /// /// This function errors when it is impossible to cast the /// expression to the target [arrow::datatypes::DataType]. - fn cast_to(self, cast_to_type: &LogicalType, schema: &dyn ExprSchema) -> Result { + fn cast_to(self, cast_to_type: &TypeRelation, schema: &dyn ExprSchema) -> Result { let this_type = self.get_type(schema)?; if this_type == *cast_to_type { return Ok(self); @@ -515,7 +516,7 @@ impl ExprSchemable for Expr { // type of the expression? // TODO(@notfilippo): The basis for whether cast can be executed should be the logical type - if can_cast_types(&this_type.physical_type(), &cast_to_type.physical_type()) { + if can_cast_types(&this_type.physical(), &cast_to_type.physical()) { match self { Expr::ScalarSubquery(subquery) => { Ok(Expr::ScalarSubquery(cast_subquery(subquery, cast_to_type)?)) @@ -529,7 +530,7 @@ impl ExprSchemable for Expr { } /// cast subquery in InSubquery/ScalarSubquery to a given type. -pub fn cast_subquery(subquery: Subquery, cast_to_type: &LogicalType) -> Result { +pub fn cast_subquery(subquery: Subquery, cast_to_type: &TypeRelation) -> Result { if subquery.subquery.schema().field(0).data_type() == cast_to_type { return Ok(subquery); } @@ -596,7 +597,7 @@ mod tests { fn test_between_nullability() { let get_schema = |nullable| { MockExprSchema::new() - .with_data_type(LogicalType::Int32) + .with_data_type(DataType::Int32.into()) .with_nullable(nullable) }; @@ -620,7 +621,7 @@ mod tests { fn test_inlist_nullability() { let get_schema = |nullable| { MockExprSchema::new() - .with_data_type(LogicalType::Int32) + .with_data_type(DataType::Int32.into()) .with_nullable(nullable) }; @@ -645,7 +646,7 @@ mod tests { fn test_like_nullability() { let get_schema = |nullable| { MockExprSchema::new() - .with_data_type(LogicalType::Utf8) + .with_data_type(DataType::Utf8.into()) .with_nullable(nullable) }; @@ -661,9 +662,9 @@ mod tests { fn expr_schema_data_type() { let expr = col("foo"); assert_eq!( - LogicalType::Utf8, - expr.get_type(&MockExprSchema::new().with_data_type(LogicalType::Utf8)) - .unwrap() + &LogicalType::Utf8, + expr.get_type(&MockExprSchema::new().with_data_type(DataType::Utf8.into())) + .unwrap().logical() ); } @@ -673,7 +674,7 @@ mod tests { meta.insert("bar".to_string(), "buzz".to_string()); let expr = col("foo"); let schema = MockExprSchema::new() - .with_data_type(LogicalType::Int32) + .with_data_type(DataType::Int32.into()) .with_metadata(meta.clone()); // col and alias should be metadata-preserving @@ -684,14 +685,14 @@ mod tests { assert_eq!( HashMap::new(), expr.clone() - .cast_to(&LogicalType::Int64, &schema) + .cast_to(&DataType::Int64.into(), &schema) .unwrap() .metadata(&schema) .unwrap() ); let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("foo", LogicalType::Int32, true).with_metadata(meta.clone())] + vec![LogicalField::new("foo", DataType::Int32, true).with_metadata(meta.clone())] .into(), HashMap::new(), ) @@ -704,7 +705,7 @@ mod tests { #[derive(Debug)] struct MockExprSchema { nullable: bool, - data_type: LogicalType, + data_type: TypeRelation, error_on_nullable: bool, metadata: HashMap, } @@ -713,7 +714,7 @@ mod tests { fn new() -> Self { Self { nullable: false, - data_type: LogicalType::Null, + data_type: DataType::Null.into(), error_on_nullable: false, metadata: HashMap::new(), } @@ -724,7 +725,7 @@ mod tests { self } - fn with_data_type(mut self, data_type: LogicalType) -> Self { + fn with_data_type(mut self, data_type: TypeRelation) -> Self { self.data_type = data_type; self } @@ -749,7 +750,7 @@ mod tests { } } - fn data_type(&self, _col: &Column) -> Result<&LogicalType> { + fn data_type(&self, _col: &Column) -> Result<&TypeRelation> { Ok(&self.data_type) } @@ -757,7 +758,7 @@ mod tests { Ok(&self.metadata) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalType, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)> { Ok((self.data_type(col)?, self.nullable(col)?)) } } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 91f72a1386d8..cef839860a70 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -48,15 +48,15 @@ use crate::{ WriteOp, }; -use arrow::datatypes::{Schema, SchemaRef}; +use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::file_options::file_type::FileType; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::{get_target_functional_dependencies, internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, TableReference, UnnestOptions, ToDFSchema}; -use datafusion_common::logical_type::extension::ExtensionType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::LogicalType; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; /// Default table name for unnamed table pub const UNNAMED_TABLE: &str = "?table?"; @@ -183,18 +183,18 @@ impl LogicalPlanBuilder { } let empty_schema = DFSchema::empty(); - let mut field_types: Vec = Vec::with_capacity(n_cols); + let mut field_types: Vec = Vec::with_capacity(n_cols); for j in 0..n_cols { - let mut common_type: Option = None; + let mut common_type: Option = None; for (i, row) in values.iter().enumerate() { let value = &row[j]; let data_type = value.get_type(&empty_schema)?; - if data_type == LogicalType::Null { + if *data_type.logical() == LogicalType::Null { continue; } if let Some(prev_type) = common_type { // get common type of each column values. - let Some(new_type) = values_coercion(&data_type.physical_type(), &prev_type.physical_type()) else { + let Some(new_type) = values_coercion(&data_type.physical(), &prev_type.physical()) else { return plan_err!("Inconsistent data type across values list at row {i} column {j}. Was {prev_type} but found {data_type}"); }; common_type = Some(new_type.into()); @@ -202,13 +202,13 @@ impl LogicalPlanBuilder { common_type = Some(data_type.clone()); } } - field_types.push(common_type.unwrap_or(LogicalType::Utf8)); + field_types.push(common_type.unwrap_or(DataType::Utf8.into())); } // wrap cast if data type is not same as common type. for row in &mut values { for (j, field_type) in field_types.iter().enumerate() { if let Expr::Literal(ScalarValue::Null) = row[j] { - row[j] = Expr::Literal(ScalarValue::try_from(field_type.clone())?); + row[j] = Expr::Literal(ScalarValue::try_from(field_type.physical())?); } else { row[j] = std::mem::take(&mut row[j]).cast_to(field_type, &empty_schema)?; @@ -384,7 +384,7 @@ impl LogicalPlanBuilder { } /// Make a builder for a prepare logical plan from the builder's plan - pub fn prepare(self, name: String, data_types: Vec) -> Result { + pub fn prepare(self, name: String, data_types: Vec) -> Result { Ok(Self::from(LogicalPlan::Prepare(Prepare { name, data_types, @@ -1380,8 +1380,8 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result SchemaRef { - self.table_schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.table_schema.clone().into()) } fn supports_filters_pushdown( @@ -1604,14 +1604,12 @@ pub fn unnest(input: LogicalPlan, columns: Vec) -> Result { // - Struct(field1, field2) returns ["a.field1","a.field2"] pub fn get_unnested_columns( col_name: &String, - data_type: &LogicalType, + data_type: &TypeRelation, ) -> Result)>> { let mut qualified_columns = Vec::with_capacity(1); - match data_type { - LogicalType::List(field) - | LogicalType::FixedSizeList(field, _) - | LogicalType::LargeList(field) => { + match data_type.logical() { + LogicalType::List(field) => { let new_field = Arc::new(LogicalField::new( col_name.clone(), field.data_type().clone(), @@ -1671,10 +1669,8 @@ pub fn unnest_with_options( &column_to_unnest.name, original_field.data_type(), )?; - match original_field.data_type() { - LogicalType::List(_) - | LogicalType::FixedSizeList(_, _) - | LogicalType::LargeList(_) => list_columns.push(index), + match original_field.data_type().logical() { + LogicalType::List(_) => list_columns.push(index), LogicalType::Struct(_) => struct_columns.push(index), _ => { panic!( @@ -1723,6 +1719,7 @@ pub fn unnest_with_options( #[cfg(test)] mod tests { use arrow::datatypes::{DataType, Field, Fields}; + use datafusion_common::logical_type::ExtensionType; use super::*; use crate::logical_plan::StringifiedPlan; use crate::{col, expr, expr_fn::exists, in_subquery, lit, scalar_subquery}; @@ -2112,7 +2109,7 @@ mod tests { // Check unnested field is a scalar let field = plan.schema().field_with_name(None, "strings").unwrap(); - assert_eq!(&LogicalType::Utf8, field.data_type()); + assert_eq!(&LogicalType::Utf8, field.data_type().logical()); // Unnesting the singular struct column result into 2 new columns for each subfield let plan = nested_table_scan("test_table")? @@ -2130,7 +2127,7 @@ mod tests { .schema() .field_with_name(None, &format!("struct_singular.{}", field_name)) .unwrap(); - assert_eq!(&LogicalType::UInt32, field.data_type()); + assert_eq!(&LogicalType::UInt32, field.data_type().logical()); } // Unnesting multiple fields in separate plans @@ -2149,7 +2146,7 @@ mod tests { // Check unnested struct list field should be a struct. let field = plan.schema().field_with_name(None, "structs").unwrap(); - assert!(matches!(field.data_type(), LogicalType::Struct(_))); + assert!(matches!(field.data_type().logical(), LogicalType::Struct(_))); // Unnesting multiple fields at the same time let cols = vec!["strings", "structs", "struct_singular"] @@ -2223,11 +2220,11 @@ mod tests { #[test] fn test_change_redundant_column() -> Result<()> { - let t1_field_1 = LogicalField::new("a", LogicalType::Int32, false); - let t2_field_1 = LogicalField::new("a", LogicalType::Int32, false); - let t2_field_3 = LogicalField::new("a", LogicalType::Int32, false); - let t1_field_2 = LogicalField::new("b", LogicalType::Int32, false); - let t2_field_2 = LogicalField::new("b", LogicalType::Int32, false); + let t1_field_1 = LogicalField::new("a", DataType::Int32, false); + let t2_field_1 = LogicalField::new("a", DataType::Int32, false); + let t2_field_3 = LogicalField::new("a", DataType::Int32, false); + let t1_field_2 = LogicalField::new("b", DataType::Int32, false); + let t2_field_2 = LogicalField::new("b", DataType::Int32, false); let field_vec = vec![t1_field_1, t2_field_1, t1_field_2, t2_field_2, t2_field_3]; let remove_redundant = change_redundant_column(&LogicalFields::from(field_vec)); @@ -2235,11 +2232,11 @@ mod tests { assert_eq!( remove_redundant, vec![ - LogicalField::new("a", LogicalType::Int32, false), - LogicalField::new("a:1", LogicalType::Int32, false), - LogicalField::new("b", LogicalType::Int32, false), - LogicalField::new("b:1", LogicalType::Int32, false), - LogicalField::new("a:2", LogicalType::Int32, false), + LogicalField::new("a", DataType::Int32, false), + LogicalField::new("a:1", DataType::Int32, false), + LogicalField::new("b", DataType::Int32, false), + LogicalField::new("b:1", DataType::Int32, false), + LogicalField::new("a:2", DataType::Int32, false), ] ); Ok(()) diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 8b938a808b96..84706d6a9061 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, Volatility}; use datafusion_common::{Constraints, DFSchemaRef, SchemaReference, TableReference}; use sqlparser::ast::Ident; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; /// Various types of DDL (CREATE / DROP) catalog manipulation #[derive(Clone, PartialEq, Eq, Hash)] @@ -322,7 +322,7 @@ pub struct CreateFunction { pub temporary: bool, pub name: String, pub args: Option>, - pub return_type: Option, + pub return_type: Option, pub params: CreateFunctionBody, /// Dummy schema pub schema: DFSchemaRef, @@ -332,7 +332,7 @@ pub struct OperateFunctionArg { // TODO: figure out how to support mode // pub mode: Option, pub name: Option, - pub data_type: LogicalType, + pub data_type: TypeRelation, pub default_expr: Option, } #[derive(Clone, PartialEq, Eq, Hash, Debug)] diff --git a/datafusion/expr/src/logical_plan/dml.rs b/datafusion/expr/src/logical_plan/dml.rs index 04124984af3f..d6353a9ad096 100644 --- a/datafusion/expr/src/logical_plan/dml.rs +++ b/datafusion/expr/src/logical_plan/dml.rs @@ -20,10 +20,10 @@ use std::fmt::{self, Display}; use std::hash::{Hash, Hasher}; use std::sync::Arc; +use arrow::datatypes::DataType; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{DFSchemaRef, TableReference}; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; use datafusion_common::logical_type::schema::LogicalSchema; use crate::LogicalPlan; @@ -131,6 +131,6 @@ impl Display for WriteOp { fn make_count_schema() -> DFSchemaRef { Arc::new( - LogicalSchema::new(vec![LogicalField::new("count", LogicalType::UInt64, false)]).try_into().unwrap() + LogicalSchema::new(vec![LogicalField::new("count", DataType::UInt64, false)]).try_into().unwrap() ) } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 0da3b156967a..d19d0f4a6621 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -41,6 +41,7 @@ use crate::{ }; use arrow::datatypes::{DataType, Field, Schema}; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -56,7 +57,7 @@ use crate::logical_plan::tree_node::unwrap_arc; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; /// A `LogicalPlan` is a node in a tree of relational operators (such as @@ -1391,8 +1392,8 @@ impl LogicalPlan { /// Walk the logical plan, find any `Placeholder` tokens, and return a map of their IDs and DataTypes pub fn get_parameter_types( &self, - ) -> Result>, DataFusionError> { - let mut param_types: HashMap> = HashMap::new(); + ) -> Result>, DataFusionError> { + let mut param_types: HashMap> = HashMap::new(); self.apply_with_subqueries(|plan| { plan.apply_expressions(|expr| { @@ -1654,7 +1655,7 @@ impl LogicalPlan { let schema = source.schema(); let names: Vec<&str> = indices .iter() - .map(|i| schema.field(*i).name().as_str()) + .map(|i| schema.field(*i).name()) .collect(); format!(" projection=[{}]", names.join(", ")) } @@ -1910,7 +1911,7 @@ impl LogicalPlan { .map(|i| &input_columns[*i]) .collect::>(); // get items from input_columns indexed by list_col_indices - write!(f, "Unnest: lists[{}] structs[{}]", + write!(f, "Unnest: lists[{}] structs[{}]", expr_vec_fmt!(list_type_columns), expr_vec_fmt!(struct_type_columns)) } @@ -2127,7 +2128,7 @@ impl Filter { // construction (such as with correlated subqueries) so we make a best effort here and // ignore errors resolving the expression against the schema. if let Ok(predicate_type) = predicate.get_type(input.schema()) { - if predicate_type != LogicalType::Boolean { + if *predicate_type.logical() != LogicalType::Boolean { return plan_err!( "Cannot create filter with non-boolean predicate '{predicate}' returning {predicate_type}" ); @@ -2401,7 +2402,7 @@ impl TableScan { if table_name.table().is_empty() { return plan_err!("table_name cannot be empty"); } - let schema: LogicalSchema = table_source.schema().as_ref().clone().into(); + let schema = table_source.schema(); let func_dependencies = FunctionalDependencies::new_from_constraints( table_source.constraints(), schema.fields.len(), @@ -2476,7 +2477,7 @@ pub struct Prepare { /// The name of the statement pub name: String, /// Data types of the parameters ([`Expr::Placeholder`]) - pub data_types: Vec, + pub data_types: Vec, /// The logical plan of the statements pub input: Arc, } diff --git a/datafusion/expr/src/simplify.rs b/datafusion/expr/src/simplify.rs index 8c647f16bd75..25b3384e2baf 100644 --- a/datafusion/expr/src/simplify.rs +++ b/datafusion/expr/src/simplify.rs @@ -17,8 +17,9 @@ //! Structs and traits to provide the information needed for expression simplification. +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::{DFSchemaRef, DataFusionError, Result}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use crate::{execution_props::ExecutionProps, Expr, ExprSchemable}; /// Provides the information necessary to apply algebraic simplification to an @@ -38,7 +39,7 @@ pub trait SimplifyInfo { fn execution_props(&self) -> &ExecutionProps; /// Returns data type of this expr needed for determining optimized int type of a value - fn get_data_type(&self, expr: &Expr) -> Result; + fn get_data_type(&self, expr: &Expr) -> Result; } /// Provides simplification information based on DFSchema and @@ -74,7 +75,7 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { /// returns true if this Expr has boolean type fn is_boolean_type(&self, expr: &Expr) -> Result { for schema in &self.schema { - if let Ok(LogicalType::Boolean) = expr.get_type(schema) { + if let Ok(LogicalType::Boolean) = expr.get_type(schema).map(|t| t.logical().clone()) { return Ok(true); } } @@ -93,7 +94,7 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { } /// Returns data type of this expr needed for determining optimized int type of a value - fn get_data_type(&self, expr: &Expr) -> Result { + fn get_data_type(&self, expr: &Expr) -> Result { let schema = self.schema.as_ref().ok_or_else(|| { DataFusionError::Internal( "attempt to get data type without schema".to_string(), diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 72ed51f44415..729304097477 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -19,8 +19,8 @@ use crate::{Expr, LogicalPlan}; -use arrow::datatypes::SchemaRef; use datafusion_common::{Constraints, Result}; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use std::any::Any; @@ -83,7 +83,7 @@ pub trait TableSource: Sync + Send { fn as_any(&self) -> &dyn Any; /// Get a reference to the schema for this table - fn schema(&self) -> SchemaRef; + fn schema(&self) -> LogicalSchemaRef; /// Get primary key indices, if one exists. fn constraints(&self) -> Option<&Constraints> { diff --git a/datafusion/expr/src/type_coercion/mod.rs b/datafusion/expr/src/type_coercion/mod.rs index 275700adc9a1..14bd82e18d8f 100644 --- a/datafusion/expr/src/type_coercion/mod.rs +++ b/datafusion/expr/src/type_coercion/mod.rs @@ -36,53 +36,54 @@ pub mod binary; pub mod functions; pub mod other; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{signature::LogicalType, TypeRelation, ExtensionType}; /// Determine whether the given data type `dt` represents signed numeric values. -pub fn is_signed_numeric(dt: &LogicalType) -> bool { +pub fn is_signed_numeric(dt: &TypeRelation) -> bool { + use LogicalType::*; matches!( - dt, - LogicalType::Int8 - | LogicalType::Int16 - | LogicalType::Int32 - | LogicalType::Int64 - | LogicalType::Float16 - | LogicalType::Float32 - | LogicalType::Float64 - | LogicalType::Decimal128(_, _) - | LogicalType::Decimal256(_, _), + dt.logical(), + Int8 + | Int16 + | Int32 + | Int64 + | Float16 + | Float32 + | Float64 + | Decimal128(_, _) + | Decimal256(_, _), ) } /// Determine whether the given data type `dt` is `Null`. -pub fn is_null(dt: &LogicalType) -> bool { - *dt == LogicalType::Null +pub fn is_null(dt: &TypeRelation) -> bool { + *dt.logical() == LogicalType::Null } /// Determine whether the given data type `dt` is a `Timestamp`. -pub fn is_timestamp(dt: &LogicalType) -> bool { - matches!(dt, LogicalType::Timestamp(_, _)) +pub fn is_timestamp(dt: &TypeRelation) -> bool { + matches!(dt.logical(), LogicalType::Timestamp(_, _)) } /// Determine whether the given data type 'dt' is a `Interval`. -pub fn is_interval(dt: &LogicalType) -> bool { - matches!(dt, LogicalType::Interval(_)) +pub fn is_interval(dt: &TypeRelation) -> bool { + matches!(dt.logical(), LogicalType::Interval(_)) } /// Determine whether the given data type `dt` is a `Date` or `Timestamp`. -pub fn is_datetime(dt: &LogicalType) -> bool { +pub fn is_datetime(dt: &TypeRelation) -> bool { matches!( - dt, - LogicalType::Date32 | LogicalType::Date64 | LogicalType::Timestamp(_, _) + dt.logical(), + LogicalType::Date | LogicalType::Timestamp(_, _) ) } /// Determine whether the given data type `dt` is a `Utf8` or `LargeUtf8`. -pub fn is_utf8_or_large_utf8(dt: &LogicalType) -> bool { - matches!(dt, LogicalType::Utf8 | LogicalType::LargeUtf8) +pub fn is_utf8_or_large_utf8(dt: &TypeRelation) -> bool { + matches!(dt.logical(), LogicalType::Utf8) } /// Determine whether the given data type `dt` is a `Decimal`. -pub fn is_decimal(dt: &LogicalType) -> bool { - matches!(dt, LogicalType::Decimal128(_, _) | LogicalType::Decimal256(_, _)) +pub fn is_decimal(dt: &TypeRelation) -> bool { + matches!(dt.logical(), LogicalType::Decimal128(_, _) | LogicalType::Decimal256(_, _)) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 3a55297d39ba..df4da26cd818 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -28,7 +28,8 @@ use crate::{ and, BinaryExpr, Expr, ExprSchemable, Filter, GroupingSet, LogicalPlan, Operator, }; -use arrow::datatypes::{DataType, TimeUnit}; +use arrow::datatypes::DataType; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -38,10 +39,10 @@ use datafusion_common::{ ScalarValue, TableReference, }; -use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; +use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; /// The value to which `COUNT(*)` is expanded to in /// `COUNT()` expressions @@ -833,37 +834,29 @@ pub(crate) fn find_column_indexes_referenced_by_expr( /// can this data type be used in hash join equal conditions?? /// data types here come from function 'equal_rows', if more data types are supported /// in equal_rows(hash join), add those data types here to generate join logical plan. -pub fn can_hash(data_type: &LogicalType) -> bool { - match data_type { - LogicalType::Null => true, - LogicalType::Boolean => true, - LogicalType::Int8 => true, - LogicalType::Int16 => true, - LogicalType::Int32 => true, - LogicalType::Int64 => true, - LogicalType::UInt8 => true, - LogicalType::UInt16 => true, - LogicalType::UInt32 => true, - LogicalType::UInt64 => true, - LogicalType::Float32 => true, - LogicalType::Float64 => true, - LogicalType::Timestamp(time_unit, _) => match time_unit { - TimeUnit::Second => true, - TimeUnit::Millisecond => true, - TimeUnit::Microsecond => true, - TimeUnit::Nanosecond => true, - }, - LogicalType::Utf8 => true, - LogicalType::LargeUtf8 => true, - LogicalType::Decimal128(_, _) => true, - LogicalType::Date32 => true, - LogicalType::Date64 => true, - LogicalType::FixedSizeBinary(_) => true, - LogicalType::List(_) => true, - LogicalType::LargeList(_) => true, - LogicalType::FixedSizeList(_, _) => true, - _ => false, - } +pub fn can_hash(data_type: &TypeRelation) -> bool { + use LogicalType::*; + matches!( + data_type.logical(), + Null + | Boolean + | Int8 + | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float16 + | Float32 + | Float64 + | Timestamp(_, _) + | Utf8 + | Decimal128(_, _) + | Date + | List(_) + ) } /// Check whether all columns are from the schema. @@ -1247,7 +1240,6 @@ impl AggregateOrderSensitivity { #[cfg(test)] mod tests { - use datafusion_common::logical_type::LogicalType; use super::*; use crate::{ col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, @@ -1702,11 +1694,11 @@ mod tests { fn test_collect_expr() -> Result<()> { let mut accum: HashSet = HashSet::new(); expr_to_columns( - &Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Float64)), + &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), &mut accum, )?; expr_to_columns( - &Expr::Cast(Cast::new(Box::new(col("a")), LogicalType::Float64)), + &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), &mut accum, )?; assert_eq!(1, accum.len()); diff --git a/datafusion/expr/src/var_provider.rs b/datafusion/expr/src/var_provider.rs index b746955630b3..a1f4c40843ee 100644 --- a/datafusion/expr/src/var_provider.rs +++ b/datafusion/expr/src/var_provider.rs @@ -18,7 +18,7 @@ //! Variable provider use datafusion_common::{Result, ScalarValue}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; /// Variable type, system/user defined #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -35,7 +35,7 @@ pub trait VarProvider: std::fmt::Debug { fn get_value(&self, var_names: Vec) -> Result; /// Return the type of the given variable - fn get_type(&self, var_names: &[String]) -> Option; + fn get_type(&self, var_names: &[String]) -> Option; } pub fn is_system_variables(variable_names: &[String]) -> bool { diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index 8aaae7b740d2..4f4ca28a206c 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -20,14 +20,14 @@ use arrow::array::{ }; use arrow::datatypes::DataType; use datafusion_common::cast::{as_map_array, as_struct_array}; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::{ exec_err, plan_datafusion_err, plan_err, ExprSchema, Result, ScalarValue, }; use datafusion_expr::{ColumnarValue, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::ExtensionType; #[derive(Debug)] pub struct GetFieldFunc { @@ -109,16 +109,16 @@ impl ScalarUDFImpl for GetFieldFunc { }; // TODO(@notfilippo): avoid converting to physical type let data_type = args[0].get_type(schema)?; - match (data_type, name) { + match (data_type.logical(), name) { (LogicalType::Map(fields, _), _) => { - match fields.data_type() { + match fields.data_type().logical() { LogicalType::Struct(fields) if fields.len() == 2 => { // Arrow's MapArray is essentially a ListArray of structs with two columns. They are // often named "key", and "value", but we don't require any specific naming here; // instead, we assume that the second columnis the "value" column both here and in // execution. let value_field = fields.get(1).expect("fields should have exactly two members"); - Ok(value_field.data_type().physical_type()) + Ok(value_field.data_type().physical().clone()) }, _ => plan_err!("Map fields must contain a Struct with exactly 2 fields"), } @@ -130,7 +130,7 @@ impl ScalarUDFImpl for GetFieldFunc { ) } else { let field = fields.iter().find(|f| f.name() == s); - field.ok_or(plan_datafusion_err!("Field {s} not found in struct")).map(|f| f.data_type().clone().physical_type()) + field.ok_or(plan_datafusion_err!("Field {s} not found in struct")).map(|f| f.data_type().clone().physical().clone()) } } (LogicalType::Struct(_), _) => plan_err!( diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index eeadd18c6215..9e98023eeafe 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -17,12 +17,12 @@ use arrow::array::StructArray; use arrow::datatypes::{DataType, Field, Fields}; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; use datafusion_expr::{ColumnarValue, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; use std::sync::Arc; -use datafusion_common::logical_type::extension::ExtensionType; /// put values in a struct array. fn named_struct_expr(args: &[ColumnarValue]) -> Result { @@ -140,7 +140,7 @@ impl ScalarUDFImpl for NamedStructFunc { let value = &chunk[1]; if let Expr::Literal(ScalarValue::Utf8(Some(name))) = name { - Ok(Field::new(name, value.get_type(schema)?.physical_type(), true)) + Ok(Field::new(name, value.get_type(schema)?.physical().clone(), true)) } else { exec_err!("named_struct even arguments must be string literals, got {name} instead at position {}", i * 2) } diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index 0d4d66eb5e42..630bab95cf46 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -24,11 +24,11 @@ use super::power::PowerFunc; use arrow::array::{ArrayRef, Float32Array, Float64Array}; use arrow::datatypes::DataType; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{ exec_err, internal_err, plan_datafusion_err, plan_err, DataFusionError, Result, ScalarValue, }; -use datafusion_common::logical_type::extension::ExtensionType; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; @@ -184,13 +184,13 @@ impl ScalarUDFImpl for LogFunc { let base = if let Some(base) = args.pop() { base } else { - lit(ScalarValue::new_ten(&number_datatype.physical_type())?) + lit(ScalarValue::new_ten(&number_datatype.physical())?) }; match number { - Expr::Literal(value) if value == ScalarValue::new_one(&number_datatype.physical_type())? => { + Expr::Literal(value) if value == ScalarValue::new_one(&number_datatype.physical())? => { Ok(ExprSimplifyResult::Simplified(lit(ScalarValue::new_zero( - &info.get_data_type(&base)?.physical_type(), + &info.get_data_type(&base)?.physical(), )?))) } Expr::ScalarFunction(ScalarFunction { func, mut args }) @@ -202,7 +202,7 @@ impl ScalarUDFImpl for LogFunc { number => { if number == base { Ok(ExprSimplifyResult::Simplified(lit(ScalarValue::new_one( - &number_datatype.physical_type(), + &number_datatype.physical(), )?))) } else { let args = match num_args { diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index 77a131997c41..9d47e27d8c65 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -19,6 +19,7 @@ use arrow::datatypes::{ArrowNativeTypeOp, DataType}; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{ arrow_datafusion_err, exec_datafusion_err, exec_err, plan_datafusion_err, DataFusionError, Result, ScalarValue, @@ -32,7 +33,6 @@ use datafusion_expr::TypeSignature::*; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; use std::sync::Arc; -use datafusion_common::logical_type::extension::ExtensionType; use super::log::LogFunc; #[derive(Debug)] @@ -144,11 +144,11 @@ impl ScalarUDFImpl for PowerFunc { plan_datafusion_err!("Expected power to have 2 arguments, got 1") })?; - let exponent_type = info.get_data_type(&exponent)?.physical_type(); + let exponent_type = info.get_data_type(&exponent)?.physical().clone(); match exponent { Expr::Literal(value) if value == ScalarValue::new_zero(&exponent_type)? => { Ok(ExprSimplifyResult::Simplified(Expr::Literal( - ScalarValue::new_one(&info.get_data_type(&base)?.physical_type())?, + ScalarValue::new_one(&info.get_data_type(&base)?.physical().clone())?, ))) } Expr::Literal(value) if value == ScalarValue::new_one(&exponent_type)? => { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 1074c04aa395..4b15d279898b 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -98,6 +98,7 @@ fn analyze_internal(plan: LogicalPlan) -> Result> { mod tests { use super::*; use crate::test::*; + use arrow::datatypes::DataType; use datafusion_common::ScalarValue; use datafusion_expr::expr::Sort; use datafusion_expr::{ @@ -107,7 +108,6 @@ mod tests { }; use datafusion_functions_aggregate::count::count_udaf; use std::sync::Arc; - use datafusion_common::logical_type::LogicalType; use datafusion_functions_aggregate::expr_fn::{count, sum}; fn assert_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { @@ -191,7 +191,7 @@ mod tests { .filter( scalar_subquery(Arc::new( LogicalPlanBuilder::from(table_scan_t2) - .filter(out_ref_col(LogicalType::UInt32, "t1.a").eq(col("t2.a")))? + .filter(out_ref_col(DataType::UInt32, "t1.a").eq(col("t2.a")))? .aggregate( Vec::::new(), vec![count(lit(COUNT_STAR_EXPANSION))], diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index 73ab37cb11d8..9e814de73101 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -106,6 +106,8 @@ mod tests { use crate::test::assert_analyzed_plan_eq; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, TableSource}; pub struct RawTableSource {} @@ -115,10 +117,10 @@ mod tests { self } - fn schema(&self) -> arrow::datatypes::SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, false), - Field::new("b", DataType::Int64, false), + fn schema(&self) -> LogicalSchemaRef { + Arc::new(LogicalSchema::new(vec![ + LogicalField::new("a", DataType::Int64, false), + LogicalField::new("b", DataType::Int64, false), ])) } @@ -159,8 +161,8 @@ mod tests { Ok(datafusion_expr::TableProviderFilterPushDown::Exact) } - fn schema(&self) -> arrow::datatypes::SchemaRef { - Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])) + fn schema(&self) -> LogicalSchemaRef { + Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]).into()) } fn get_logical_plan(&self) -> Option<&LogicalPlan> { diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index fb479257ee3c..f5c8e98f0fc3 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -22,13 +22,13 @@ use std::sync::Arc; use arrow::datatypes::{DataType, IntervalUnit}; use datafusion_common::config::ConfigOptions; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::expr::{ self, AggregateFunctionDefinition, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, ScalarFunction, WindowFunction, @@ -163,9 +163,9 @@ impl<'a> TypeCoercionRewriter<'a> { right: Expr, ) -> Result<(Expr, Expr)> { let (left_type, right_type) = get_input_types( - &left.get_type(self.schema)?.physical_type(), + &left.get_type(self.schema)?.physical(), op, - &right.get_type(self.schema)?.physical_type(), + &right.get_type(self.schema)?.physical(), )?; Ok(( left.cast_to(&left_type.into(), self.schema)?, @@ -212,7 +212,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { analyze_internal(self.schema, unwrap_arc(subquery.subquery))?.data; let expr_type = expr.get_type(self.schema)?; let subquery_type = new_plan.schema().field(0).data_type(); - let common_type = comparison_coercion(&expr_type.physical_type(), &subquery_type.physical_type()).ok_or(plan_datafusion_err!( + let common_type = comparison_coercion(&expr_type.physical(), &subquery_type.physical()).ok_or(plan_datafusion_err!( "expr type {expr_type:?} can't cast to {subquery_type:?} in InSubquery" ), )?; @@ -257,7 +257,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { }) => { let left_type = expr.get_type(self.schema)?; let right_type = pattern.get_type(self.schema)?; - let coerced_type = like_coercion(&left_type.physical_type(), &right_type.physical_type()).ok_or_else(|| { + let coerced_type = like_coercion(&left_type.physical(), &right_type.physical()).ok_or_else(|| { let op_name = if case_insensitive { "ILIKE" } else { @@ -295,14 +295,14 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { }) => { let expr_type = expr.get_type(self.schema)?; let low_type = low.get_type(self.schema)?; - let low_coerced_type = comparison_coercion(&expr_type.physical_type(), &low_type.physical_type()) + let low_coerced_type = comparison_coercion(&expr_type.physical(), &low_type.physical()) .ok_or_else(|| { DataFusionError::Internal(format!( "Failed to coerce types {expr_type} and {low_type} in BETWEEN expression" )) })?; let high_type = high.get_type(self.schema)?; - let high_coerced_type = comparison_coercion(&expr_type.physical_type(), &low_type.physical_type()) + let high_coerced_type = comparison_coercion(&expr_type.physical(), &low_type.physical()) .ok_or_else(|| { DataFusionError::Internal(format!( "Failed to coerce types {expr_type} and {high_type} in BETWEEN expression" @@ -331,10 +331,10 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { let expr_data_type = expr.get_type(self.schema)?; let list_data_types = list .iter() - .map(|list_expr| list_expr.get_type(self.schema).map(|t| t.physical_type())) + .map(|list_expr| list_expr.get_type(self.schema).map(|t| t.physical().clone())) .collect::>>()?; let result_type = - get_coerce_type_for_list(&expr_data_type.physical_type(), &list_data_types); + get_coerce_type_for_list(&expr_data_type.physical(), &list_data_types); match result_type { None => plan_err!( "Can not find compatible types to compare {expr_data_type:?} with {list_data_types:?}" @@ -367,7 +367,6 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { self.schema, &func, )?; - let new_expr = coerce_arguments_for_fun(new_expr, self.schema, &func)?; Ok(Transformed::yes(Expr::ScalarFunction( ScalarFunction::new_udf(func, new_expr), ))) @@ -476,16 +475,16 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { /// Casts the given `value` to `target_type`. Note that this function /// only considers `Null` or `Utf8` values. -fn coerce_scalar(target_type: &LogicalType, value: &ScalarValue) -> Result { +fn coerce_scalar(target_type: &TypeRelation, value: &ScalarValue) -> Result { match value { // Coerce Utf8 values: ScalarValue::Utf8(Some(val)) => { - ScalarValue::try_from_string(val.clone(), &target_type.physical_type()) + ScalarValue::try_from_string(val.clone(), &target_type.physical()) } s => { if s.is_null() { // Coerce `Null` values: - ScalarValue::try_from(target_type) + ScalarValue::try_from(target_type.physical().clone()) } else { // Values except `Utf8`/`Null` variants already have the right type // (casted before) since we convert `sqlparser` outputs to `Utf8` @@ -503,15 +502,15 @@ fn coerce_scalar(target_type: &LogicalType, value: &ScalarValue) -> Result Result { coerce_scalar(target_type, &value).or_else(|err| { // If type coercion fails, check if the largest type in family works: if let Some(largest_type) = get_widest_type_in_family(target_type) { - coerce_scalar(largest_type, &value).map_or_else( + coerce_scalar(&largest_type, &value).map_or_else( |_| exec_err!("Cannot cast {value:?} to {target_type:?}"), - |_| ScalarValue::try_from(target_type), + |_| ScalarValue::try_from(target_type.physical().clone()), ) } else { Err(err) @@ -522,18 +521,19 @@ fn coerce_scalar_range_aware( /// This function returns the widest type in the family of `given_type`. /// If the given type is already the widest type, it returns `None`. /// For example, if `given_type` is `Int8`, it returns `Int64`. -fn get_widest_type_in_family(given_type: &LogicalType) -> Option<&LogicalType> { - match given_type { - LogicalType::UInt8 | LogicalType::UInt16 | LogicalType::UInt32 => Some(&LogicalType::UInt64), - LogicalType::Int8 | LogicalType::Int16 | LogicalType::Int32 => Some(&LogicalType::Int64), - LogicalType::Float16 | LogicalType::Float32 => Some(&LogicalType::Float64), +fn get_widest_type_in_family(given_type: &TypeRelation) -> Option { + use LogicalType::*; + match given_type.logical() { + UInt8 | UInt16 | UInt32 => Some(DataType::UInt64.into()), + Int8 | Int16 | Int32 => Some(DataType::Int64.into()), + Float16 | Float32 => Some(DataType::Float64.into()), _ => None, } } /// Coerces the given (window frame) `bound` to `target_type`. fn coerce_frame_bound( - target_type: &LogicalType, + target_type: &TypeRelation, bound: WindowFrameBound, ) -> Result { match bound { @@ -562,13 +562,13 @@ fn coerce_window_frame( let target_type = match window_frame.units { WindowFrameUnits::Range => { if let Some(col_type) = current_types.first() { - if col_type.is_numeric() + if col_type.logical().is_numeric() || is_utf8_or_large_utf8(col_type) - || matches!(col_type, LogicalType::Null) + || matches!(col_type.logical(), LogicalType::Null) { - col_type + col_type.clone() } else if is_datetime(col_type) { - &LogicalType::Interval(IntervalUnit::MonthDayNano) + DataType::Interval(IntervalUnit::MonthDayNano).into() } else { return internal_err!( "Cannot run range queries on datatype: {col_type:?}" @@ -578,10 +578,10 @@ fn coerce_window_frame( return internal_err!("ORDER BY column cannot be empty"); } } - WindowFrameUnits::Rows | WindowFrameUnits::Groups => &LogicalType::UInt64, + WindowFrameUnits::Rows | WindowFrameUnits::Groups => DataType::UInt64.into(), }; - window_frame.start_bound = coerce_frame_bound(target_type, window_frame.start_bound)?; - window_frame.end_bound = coerce_frame_bound(target_type, window_frame.end_bound)?; + window_frame.start_bound = coerce_frame_bound(&target_type, window_frame.start_bound)?; + window_frame.end_bound = coerce_frame_bound(&target_type, window_frame.end_bound)?; Ok(window_frame) } @@ -589,8 +589,8 @@ fn coerce_window_frame( // The above op will be rewrite to the binary op when creating the physical op. fn get_casted_expr_for_bool_op(expr: Expr, schema: &DFSchema) -> Result { let left_type = expr.get_type(schema)?; - get_input_types(&left_type.physical_type(), &Operator::IsDistinctFrom, &DataType::Boolean)?; - expr.cast_to(&LogicalType::Boolean, schema) + get_input_types(&left_type.physical(), &Operator::IsDistinctFrom, &DataType::Boolean)?; + expr.cast_to(&DataType::Boolean.into(), schema) } /// Returns `expressions` coerced to types compatible with @@ -608,7 +608,7 @@ fn coerce_arguments_for_signature_with_scalar_udf( let current_types = expressions .iter() - .map(|e| e.get_type(schema).map(|t| t.physical_type())) + .map(|e| e.get_type(schema).map(|t| t.physical().clone())) .collect::>>()?; let new_types = data_types_with_scalar_udf(¤t_types, func)?; @@ -635,7 +635,7 @@ fn coerce_arguments_for_signature_with_aggregate_udf( let current_types = expressions .iter() - .map(|e| e.get_type(schema).map(|t| t.physical_type())) + .map(|e| e.get_type(schema).map(|t| t.physical().clone())) .collect::>>()?; let new_types = data_types_with_aggregate_udf(¤t_types, func)?; @@ -647,30 +647,6 @@ fn coerce_arguments_for_signature_with_aggregate_udf( .collect() } -fn coerce_arguments_for_fun( - expressions: Vec, - schema: &DFSchema, - fun: &Arc, -) -> Result> { - // Cast Fixedsizelist to List for array functions - if fun.name() == "make_array" { - expressions - .into_iter() - .map(|expr| { - let data_type = expr.get_type(schema).unwrap(); - if let LogicalType::FixedSizeList(field, _) = data_type { - let to_type = LogicalType::List(field.clone()); - expr.cast_to(&to_type, schema) - } else { - Ok(expr) - } - }) - .collect() - } else { - Ok(expressions) - } -} - /// Returns the coerced exprs for each `input_exprs`. /// Get the coerced data type from `aggregate_rule::coerce_types` and add `try_cast` if the /// data type of `input_exprs` need to be coerced. @@ -685,7 +661,7 @@ fn coerce_agg_exprs_for_signature( } let current_types = input_exprs .iter() - .map(|e| e.get_type(schema).map(|t| t.physical_type())) + .map(|e| e.get_type(schema).map(|t| t.physical().clone())) .collect::>>()?; let coerced_types = @@ -738,12 +714,12 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { let then_types = case .when_then_expr .iter() - .map(|(_when, then)| then.get_type(schema).map(|t| t.physical_type())) + .map(|(_when, then)| then.get_type(schema).map(|t| t.physical().clone())) .collect::>>()?; let else_type = case .else_expr .as_ref() - .map(|expr| expr.get_type(schema).map(|t| t.physical_type())) + .map(|expr| expr.get_type(schema).map(|t| t.physical().clone())) .transpose()?; // find common coercible types @@ -753,10 +729,10 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { let when_types = case .when_then_expr .iter() - .map(|(when, _then)| when.get_type(schema).map(|t| t.physical_type())) + .map(|(when, _then)| when.get_type(schema).map(|t| t.physical().clone())) .collect::>>()?; let coerced_type = - get_coerce_type_for_case_expression(&when_types, Some(&case_type.physical_type())); + get_coerce_type_for_case_expression(&when_types, Some(&case_type.physical())); coerced_type.ok_or_else(|| { plan_datafusion_err!( "Failed to coerce case ({case_type:?}) and when ({when_types:?}) \ @@ -820,7 +796,7 @@ mod test { use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{DFSchema, DFSchemaRef, Result, ScalarValue}; use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::LogicalType; + use datafusion_common::logical_type::TypeRelation; use datafusion_expr::expr::{self, InSubquery, Like, ScalarFunction}; use datafusion_expr::logical_plan::{EmptyRelation, Projection}; use datafusion_expr::test::function_stub::avg_udaf; @@ -844,7 +820,7 @@ mod test { })) } - fn empty_with_type(data_type: LogicalType) -> Arc { + fn empty_with_type(data_type: impl Into) -> Arc { Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::new( @@ -860,7 +836,7 @@ mod test { #[test] fn simple_case() -> Result<()> { let expr = col("a").lt(lit(2_u32)); - let empty = empty_with_type(LogicalType::Float64); + let empty = empty_with_type(DataType::Float64); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a < CAST(UInt32(2) AS Float64)\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected) @@ -869,7 +845,7 @@ mod test { #[test] fn nested_case() -> Result<()> { let expr = col("a").lt(lit(2_u32)); - let empty = empty_with_type(LogicalType::Float64); + let empty = empty_with_type(DataType::Float64); let plan = LogicalPlan::Projection(Projection::try_new( vec![expr.clone().or(expr)], @@ -1024,10 +1000,10 @@ mod test { let expected = "Projection: avg(Float64(12))\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(LogicalType::Int32); + let empty = empty_with_type(DataType::Int32); let agg_expr = Expr::AggregateFunction(expr::AggregateFunction::new_udf( avg_udaf(), - vec![cast(col("a"), LogicalType::Float64)], + vec![cast(col("a"), DataType::Float64)], false, None, None, @@ -1061,7 +1037,7 @@ mod test { #[test] fn binary_op_date32_op_interval() -> Result<()> { // CAST(Utf8("1998-03-18") AS Date32) + IntervalDayTime("...") - let expr = cast(lit("1998-03-18"), LogicalType::Date32) + let expr = cast(lit("1998-03-18"), DataType::Date32) + lit(ScalarValue::new_interval_dt(123, 456)); let empty = empty(); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); @@ -1075,7 +1051,7 @@ mod test { fn inlist_case() -> Result<()> { // a in (1,4,8), a is int64 let expr = col("a").in_list(vec![lit(1_i32), lit(4_i8), lit(8_i64)], false); - let empty = empty_with_type(LogicalType::Int64); + let empty = empty_with_type(DataType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\ @@ -1087,7 +1063,7 @@ mod test { let empty = Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", LogicalType::Decimal128(12, 4), true)].into(), + vec![LogicalField::new("a", DataType::Decimal128(12, 4), true)].into(), std::collections::HashMap::new(), )?), })); @@ -1103,10 +1079,10 @@ mod test { let expr = col("a").between( lit("2002-05-08"), // (cast('2002-05-08' as date) + interval '1 months') - cast(lit("2002-05-08"), LogicalType::Date32) + cast(lit("2002-05-08"), DataType::Date32) + lit(ScalarValue::new_interval_ym(0, 1)), ); - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Filter(Filter::try_new(expr, empty)?); let expected = "Filter: a BETWEEN Utf8(\"2002-05-08\") AND CAST(CAST(Utf8(\"2002-05-08\") AS Date32) + IntervalYearMonth(\"1\") AS Utf8)\ @@ -1118,11 +1094,11 @@ mod test { fn between_infer_cheap_type() -> Result<()> { let expr = col("a").between( // (cast('2002-05-08' as date) + interval '1 months') - cast(lit("2002-05-08"), LogicalType::Date32) + cast(lit("2002-05-08"), DataType::Date32) + lit(ScalarValue::new_interval_ym(0, 1)), lit("2002-12-08"), ); - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Filter(Filter::try_new(expr, empty)?); // TODO: we should cast col(a). let expected = @@ -1135,13 +1111,13 @@ mod test { fn is_bool_for_type_coercion() -> Result<()> { // is true let expr = col("a").is_true(); - let empty = empty_with_type(LogicalType::Boolean); + let empty = empty_with_type(DataType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr.clone()], empty)?); let expected = "Projection: a IS TRUE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(LogicalType::Int64); + let empty = empty_with_type(DataType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let ret = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, ""); let err = ret.unwrap_err().to_string(); @@ -1149,21 +1125,21 @@ mod test { // is not true let expr = col("a").is_not_true(); - let empty = empty_with_type(LogicalType::Boolean); + let empty = empty_with_type(DataType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS NOT TRUE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; // is false let expr = col("a").is_false(); - let empty = empty_with_type(LogicalType::Boolean); + let empty = empty_with_type(DataType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS FALSE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; // is not false let expr = col("a").is_not_false(); - let empty = empty_with_type(LogicalType::Boolean); + let empty = empty_with_type(DataType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS NOT FALSE\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1177,7 +1153,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let expected = "Projection: a LIKE Utf8(\"abc\")\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1185,7 +1161,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::Null)); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let expected = "Projection: a LIKE CAST(NULL AS Utf8) AS a LIKE NULL\ \n EmptyRelation"; @@ -1194,7 +1170,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(LogicalType::Int64); + let empty = empty_with_type(DataType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); assert!(err.is_err()); @@ -1207,7 +1183,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let expected = "Projection: a ILIKE Utf8(\"abc\")\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1215,7 +1191,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::Null)); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let expected = "Projection: a ILIKE CAST(NULL AS Utf8) AS a ILIKE NULL\ \n EmptyRelation"; @@ -1224,7 +1200,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(LogicalType::Int64); + let empty = empty_with_type(DataType::Int64); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); assert!(err.is_err()); @@ -1238,13 +1214,13 @@ mod test { fn unknown_for_type_coercion() -> Result<()> { // unknown let expr = col("a").is_unknown(); - let empty = empty_with_type(LogicalType::Boolean); + let empty = empty_with_type(DataType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr.clone()], empty)?); let expected = "Projection: a IS UNKNOWN\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let ret = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); let err = ret.unwrap_err().to_string(); @@ -1252,7 +1228,7 @@ mod test { // is not unknown let expr = col("a").is_not_unknown(); - let empty = empty_with_type(LogicalType::Boolean); + let empty = empty_with_type(DataType::Boolean); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let expected = "Projection: a IS NOT UNKNOWN\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1262,7 +1238,7 @@ mod test { #[test] fn concat_for_type_coercion() -> Result<()> { - let empty = empty_with_type(LogicalType::Utf8); + let empty = empty_with_type(DataType::Utf8); let args = [col("a"), lit("b"), lit(true), lit(false), lit(13)]; // concat-type signature @@ -1285,34 +1261,34 @@ mod test { fn test_type_coercion_rewrite() -> Result<()> { // gt let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", LogicalType::Int64, true)].into(), + vec![LogicalField::new("a", DataType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; let expr = is_true(lit(12i32).gt(lit(13i64))); - let expected = is_true(cast(lit(12i32), LogicalType::Int64).gt(lit(13i64))); + let expected = is_true(cast(lit(12i32), DataType::Int64).gt(lit(13i64))); let result = expr.rewrite(&mut rewriter).data()?; assert_eq!(expected, result); // eq let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", LogicalType::Int64, true)].into(), + vec![LogicalField::new("a", DataType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; let expr = is_true(lit(12i32).eq(lit(13i64))); - let expected = is_true(cast(lit(12i32), LogicalType::Int64).eq(lit(13i64))); + let expected = is_true(cast(lit(12i32), DataType::Int64).eq(lit(13i64))); let result = expr.rewrite(&mut rewriter).data()?; assert_eq!(expected, result); // lt let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", LogicalType::Int64, true)].into(), + vec![LogicalField::new("a", DataType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; let expr = is_true(lit(12i32).lt(lit(13i64))); - let expected = is_true(cast(lit(12i32), LogicalType::Int64).lt(lit(13i64))); + let expected = is_true(cast(lit(12i32), DataType::Int64).lt(lit(13i64))); let result = expr.rewrite(&mut rewriter).data()?; assert_eq!(expected, result); @@ -1323,9 +1299,9 @@ mod test { fn binary_op_date32_eq_ts() -> Result<()> { let expr = cast( lit("1998-03-18"), - LogicalType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, None), ) - .eq(cast(lit("1998-03-18"), LogicalType::Date32)); + .eq(cast(lit("1998-03-18"), DataType::Date32)); let empty = empty(); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); dbg!(&plan); @@ -1337,7 +1313,7 @@ mod test { fn cast_if_not_same_type( expr: Box, - data_type: &LogicalType, + data_type: &TypeRelation, schema: &DFSchemaRef, ) -> Box { if &expr.get_type(schema).unwrap() != data_type { @@ -1349,8 +1325,8 @@ mod test { fn cast_helper( case: Case, - case_when_type: LogicalType, - then_else_type: LogicalType, + case_when_type: TypeRelation, + then_else_type: TypeRelation, schema: &DFSchemaRef, ) -> Case { let expr = case @@ -1381,23 +1357,23 @@ mod test { fn test_case_expression_coercion() -> Result<()> { let schema = Arc::new(DFSchema::from_unqualified_fields( vec![ - LogicalField::new("boolean", LogicalType::Boolean, true), - LogicalField::new("integer", LogicalType::Int32, true), - LogicalField::new("float", LogicalType::Float32, true), + LogicalField::new("boolean", DataType::Boolean, true), + LogicalField::new("integer", DataType::Int32, true), + LogicalField::new("float", DataType::Float32, true), LogicalField::new( "timestamp", - LogicalType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, None), true, ), - LogicalField::new("date", LogicalType::Date32, true), + LogicalField::new("date", DataType::Date32, true), LogicalField::new( "interval", - LogicalType::Interval(arrow::datatypes::IntervalUnit::MonthDayNano), + DataType::Interval(arrow::datatypes::IntervalUnit::MonthDayNano), true, ), - LogicalField::new("binary", LogicalType::Binary, true), - LogicalField::new("string", LogicalType::Utf8, true), - LogicalField::new("decimal", LogicalType::Decimal128(10, 10), true), + LogicalField::new("binary", DataType::Binary, true), + LogicalField::new("string", DataType::Utf8, true), + LogicalField::new("decimal", DataType::Decimal128(10, 10), true), ] .into(), std::collections::HashMap::new(), @@ -1412,12 +1388,12 @@ mod test { ], else_expr: None, }; - let case_when_common_type = LogicalType::Boolean; - let then_else_common_type = LogicalType::Utf8; + let case_when_common_type = DataType::Boolean; + let then_else_common_type = DataType::Utf8; let expected = cast_helper( case.clone(), - case_when_common_type, - then_else_common_type, + case_when_common_type.into(), + then_else_common_type.into(), &schema, ); let actual = coerce_case_expression(case, &schema)?; @@ -1432,12 +1408,12 @@ mod test { ], else_expr: Some(Box::new(col("string"))), }; - let case_when_common_type = LogicalType::Utf8; - let then_else_common_type = LogicalType::Utf8; + let case_when_common_type = DataType::Utf8; + let then_else_common_type = DataType::Utf8; let expected = cast_helper( case.clone(), - case_when_common_type, - then_else_common_type, + case_when_common_type.into(), + then_else_common_type.into(), &schema, ); let actual = coerce_case_expression(case, &schema)?; @@ -1490,7 +1466,7 @@ mod test { Operator::Plus, Box::new(cast( lit("2000-01-01T00:00:00"), - LogicalType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, None), )), )); let empty = empty(); @@ -1505,12 +1481,12 @@ mod test { let expr = Expr::BinaryExpr(BinaryExpr::new( Box::new(cast( lit("1998-03-18"), - LogicalType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, None), )), Operator::Minus, Box::new(cast( lit("1998-03-18"), - LogicalType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, None), )), )); let empty = empty(); @@ -1524,8 +1500,8 @@ mod test { #[test] fn in_subquery_cast_subquery() -> Result<()> { - let empty_int32 = empty_with_type(LogicalType::Int32); - let empty_int64 = empty_with_type(LogicalType::Int64); + let empty_int32 = empty_with_type(DataType::Int32); + let empty_int64 = empty_with_type(DataType::Int64); let in_subquery_expr = Expr::InSubquery(InSubquery::new( Box::new(col("a")), @@ -1549,8 +1525,8 @@ mod test { #[test] fn in_subquery_cast_expr() -> Result<()> { - let empty_int32 = empty_with_type(LogicalType::Int32); - let empty_int64 = empty_with_type(LogicalType::Int64); + let empty_int32 = empty_with_type(DataType::Int32); + let empty_int64 = empty_with_type(DataType::Int64); let in_subquery_expr = Expr::InSubquery(InSubquery::new( Box::new(col("a")), @@ -1573,8 +1549,8 @@ mod test { #[test] fn in_subquery_cast_all() -> Result<()> { - let empty_inside = empty_with_type(LogicalType::Decimal128(10, 5)); - let empty_outside = empty_with_type(LogicalType::Decimal128(8, 8)); + let empty_inside = empty_with_type(DataType::Decimal128(10, 5)); + let empty_outside = empty_with_type(DataType::Decimal128(8, 8)); let in_subquery_expr = Expr::InSubquery(InSubquery::new( Box::new(col("a")), diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index d3032966b5ca..7e9dbebf1ebc 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1095,7 +1095,6 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::LogicalType; use datafusion_expr::expr::AggregateFunction; use datafusion_expr::logical_plan::{table_scan, JoinType}; use datafusion_expr::{ @@ -1646,9 +1645,9 @@ mod test { let plan = table_scan(Some("table"), &schema, None) .unwrap() .filter( - cast(col("a"), LogicalType::Int64) + cast(col("a"), DataType::Int64) .lt(lit(1_i64)) - .and(cast(col("a"), LogicalType::Int64).not_eq(lit(1_i64))), + .and(cast(col("a"), DataType::Int64).not_eq(lit(1_i64))), ) .unwrap() .build() @@ -1706,9 +1705,9 @@ mod test { let grouping = grouping_set(vec![vec![col("a"), col("b")], vec![col("c")]]); let schema = DFSchema::from_unqualified_fields( vec![ - LogicalField::new("a", LogicalType::Int32, false), - LogicalField::new("b", LogicalType::Int32, false), - LogicalField::new("c", LogicalType::Int32, false), + LogicalField::new("a", DataType::Int32, false), + LogicalField::new("b", DataType::Int32, false), + LogicalField::new("c", DataType::Int32, false), ] .into(), HashMap::default(), @@ -1725,8 +1724,8 @@ mod test { let grouping = grouping_set(vec![vec![col("a"), col("b")], vec![col("a")]]); let schema = DFSchema::from_unqualified_fields( vec![ - LogicalField::new("a", LogicalType::Int32, false), - LogicalField::new("b", LogicalType::Int32, false), + LogicalField::new("a", DataType::Int32, false), + LogicalField::new("b", DataType::Int32, false), ] .into(), HashMap::default(), @@ -1793,7 +1792,7 @@ mod test { fn test_extract_expressions_from_col() -> Result<()> { let mut result = Vec::with_capacity(1); let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", LogicalType::Int32, false)].into(), + vec![LogicalField::new("a", DataType::Int32, false)].into(), HashMap::default(), )?; extract_expressions(&col("a"), &schema, &mut result)?; diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 7f009cbd1164..81d6dc863af6 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -369,7 +369,7 @@ mod tests { use super::*; use crate::test::*; - use datafusion_common::logical_type::LogicalType; + use arrow::datatypes::DataType; use datafusion_expr::{and, binary_expr, col, lit, not, or, out_ref_col}; fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { @@ -564,7 +564,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), )? .project(vec![col("orders.o_custkey")])? .build()?, @@ -605,7 +605,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("lineitem")) .filter( col("lineitem.l_orderkey") - .eq(out_ref_col(LogicalType::Int64, "orders.o_orderkey")), + .eq(out_ref_col(DataType::Int64, "orders.o_orderkey")), )? .project(vec![col("lineitem.l_orderkey")])? .build()?, @@ -616,7 +616,7 @@ mod tests { .filter( in_subquery(col("orders.o_orderkey"), lineitem).and( col("orders.o_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), ), )? .project(vec![col("orders.o_custkey")])? @@ -653,7 +653,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .and(col("o_orderkey").eq(lit(1))), )? @@ -688,8 +688,8 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + out_ref_col(DataType::Int64, "customer.c_custkey") + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), )? .project(vec![col("orders.o_custkey")])? .build()?, @@ -752,7 +752,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .not_eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -785,7 +785,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .lt(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -818,7 +818,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .or(col("o_orderkey").eq(lit(1))), )? @@ -876,7 +876,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -909,7 +909,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey").add(lit(1))])? @@ -942,7 +942,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey"), col("orders.o_orderkey")])? @@ -971,7 +971,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1008,7 +1008,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1045,7 +1045,7 @@ mod tests { fn in_subquery_correlated() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(test_table_scan_with_name("sq")?) - .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq.a")))? .project(vec![col("c")])? .build()?, ); @@ -1203,7 +1203,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( - out_ref_col(LogicalType::UInt32, "test.a") + out_ref_col(DataType::UInt32, "test.a") .eq(col("sq.a")) .and(col("sq.a").add(lit(1u32)).eq(col("sq.b"))), )? @@ -1238,8 +1238,8 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( - out_ref_col(LogicalType::UInt32, "test.a") - .add(out_ref_col(LogicalType::UInt32, "test.b")) + out_ref_col(DataType::UInt32, "test.a") + .add(out_ref_col(DataType::UInt32, "test.b")) .eq(col("sq.a").add(col("sq.b"))) .and(col("sq.a").add(lit(1u32)).eq(col("sq.b"))), )? @@ -1274,12 +1274,12 @@ mod tests { let subquery_scan2 = test_table_scan_with_name("sq2")?; let subquery1 = LogicalPlanBuilder::from(subquery_scan1) - .filter(out_ref_col(LogicalType::UInt32, "test.a").gt(col("sq1.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").gt(col("sq1.a")))? .project(vec![col("c") * lit(2u32)])? .build()?; let subquery2 = LogicalPlanBuilder::from(subquery_scan2) - .filter(out_ref_col(LogicalType::UInt32, "test.a").gt(col("sq2.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").gt(col("sq2.a")))? .project(vec![col("c") * lit(2u32)])? .build()?; @@ -1351,7 +1351,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), )? .project(vec![col("orders.o_custkey")])? .build()?, @@ -1382,7 +1382,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("lineitem")) .filter( col("lineitem.l_orderkey") - .eq(out_ref_col(LogicalType::Int64, "orders.o_orderkey")), + .eq(out_ref_col(DataType::Int64, "orders.o_orderkey")), )? .project(vec![col("lineitem.l_orderkey")])? .build()?, @@ -1393,7 +1393,7 @@ mod tests { .filter( exists(lineitem).and( col("orders.o_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), ), )? .project(vec![col("orders.o_custkey")])? @@ -1424,7 +1424,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .and(col("o_orderkey").eq(lit(1))), )? @@ -1452,7 +1452,7 @@ mod tests { fn exists_subquery_no_cols() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) - .filter(out_ref_col(LogicalType::Int64, "customer.c_custkey").eq(lit(1u32)))? + .filter(out_ref_col(DataType::Int64, "customer.c_custkey").eq(lit(1u32)))? .project(vec![col("orders.o_custkey")])? .build()?, ); @@ -1497,7 +1497,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .not_eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1525,7 +1525,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .lt(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1553,7 +1553,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .or(col("o_orderkey").eq(lit(1))), )? @@ -1582,7 +1582,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .build()?, @@ -1608,7 +1608,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey").add(lit(1))])? @@ -1636,7 +1636,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .project(vec![col("orders.o_custkey")])? @@ -1690,7 +1690,7 @@ mod tests { fn exists_subquery_correlated() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(test_table_scan_with_name("sq")?) - .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq.a")))? .project(vec![col("c")])? .build()?, ); @@ -1741,12 +1741,12 @@ mod tests { let subquery_scan2 = test_table_scan_with_name("sq2")?; let subquery1 = LogicalPlanBuilder::from(subquery_scan1) - .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq1.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq1.a")))? .project(vec![col("c")])? .build()?; let subquery2 = LogicalPlanBuilder::from(subquery_scan2) - .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq2.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq2.a")))? .project(vec![col("c")])? .build()?; @@ -1780,7 +1780,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), )? .project(vec![lit(1u32)])? .build()?; @@ -1832,7 +1832,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), )? .project(vec![col("sq.c")])? .distinct()? @@ -1860,7 +1860,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), )? .project(vec![col("sq.b") + col("sq.c")])? .distinct()? @@ -1888,7 +1888,7 @@ mod tests { let subquery = LogicalPlanBuilder::from(subquery_scan) .filter( (lit(1u32) + col("sq.a")) - .gt(out_ref_col(LogicalType::UInt32, "test.a") * lit(2u32)), + .gt(out_ref_col(DataType::UInt32, "test.a") * lit(2u32)), )? .project(vec![lit(1u32), col("sq.c")])? .distinct()? diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index 882499cfa68b..ccc637a0eb01 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -301,7 +301,7 @@ fn extract_non_nullable_columns( mod tests { use super::*; use crate::test::*; - use datafusion_common::logical_type::LogicalType; + use arrow::datatypes::DataType; use datafusion_expr::{ binary_expr, cast, col, lit, logical_plan::builder::LogicalPlanBuilder, @@ -427,9 +427,9 @@ mod tests { None, )? .filter(binary_expr( - cast(col("t1.b"), LogicalType::Int64).gt(lit(10u32)), + cast(col("t1.b"), DataType::Int64).gt(lit(10u32)), And, - try_cast(col("t2.c"), LogicalType::Int64).lt(lit(20u32)), + try_cast(col("t2.c"), DataType::Int64).lt(lit(20u32)), ))? .build()?; let expected = "\ diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 820ef149c347..2c4b47b4d5a0 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -156,11 +156,11 @@ fn split_eq_and_noneq_join_predicate( mod tests { use super::*; use crate::test::*; + use arrow::datatypes::DataType; use datafusion_expr::{ col, lit, logical_plan::builder::LogicalPlanBuilder, JoinType, }; use std::sync::Arc; - use datafusion_common::logical_type::LogicalType; fn assert_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_display_indent( @@ -362,8 +362,8 @@ mod tests { // filter: t1.a + CAST(Int64(1), UInt32) = t2.a + CAST(Int64(2), UInt32) as t1.a + 1 = t2.a + 2 let filter = Expr::eq( - col("t1.a") + lit(1i64).cast_to(&LogicalType::UInt32, &t1_schema)?, - col("t2.a") + lit(2i32).cast_to(&LogicalType::UInt32, &t2_schema)?, + col("t1.a") + lit(1i64).cast_to(&DataType::UInt32.into(), &t1_schema)?, + col("t2.a") + lit(2i32).cast_to(&DataType::UInt32.into(), &t2_schema)?, ) .alias("t1.a + 1 = t2.a + 2"); let plan = LogicalPlanBuilder::from(t1) diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index e3d0bbf243c2..503a2fc54acf 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -811,7 +811,6 @@ mod tests { Column, DFSchema, DFSchemaRef, JoinType, Result, TableReference, }; use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::LogicalType; use datafusion_expr::AggregateExt; use datafusion_expr::{ binary_expr, build_join_schema, @@ -1174,7 +1173,7 @@ mod tests { fn test_try_cast() -> Result<()> { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![try_cast(col("a"), LogicalType::Float64)])? + .project(vec![try_cast(col("a"), DataType::Float64)])? .build()?; let expected = "Projection: TRY_CAST(test.a AS Float64)\ @@ -1546,15 +1545,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(LogicalField::new("a", LogicalType::UInt32, false)) + Arc::new(LogicalField::new("a", DataType::UInt32, false)) ), ( Some("test".into()), - Arc::new(LogicalField::new("b", LogicalType::UInt32, false)) + Arc::new(LogicalField::new("b", DataType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(LogicalField::new("c1", LogicalType::UInt32, true)) + Arc::new(LogicalField::new("c1", DataType::UInt32, true)) ), ], HashMap::new() @@ -1598,15 +1597,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(LogicalField::new("a", LogicalType::UInt32, false)) + Arc::new(LogicalField::new("a", DataType::UInt32, false)) ), ( Some("test".into()), - Arc::new(LogicalField::new("b", LogicalType::UInt32, false)) + Arc::new(LogicalField::new("b", DataType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(LogicalField::new("c1", LogicalType::UInt32, true)) + Arc::new(LogicalField::new("c1", DataType::UInt32, true)) ), ], HashMap::new() @@ -1648,15 +1647,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(LogicalField::new("a", LogicalType::UInt32, false)) + Arc::new(LogicalField::new("a", DataType::UInt32, false)) ), ( Some("test".into()), - Arc::new(LogicalField::new("b", LogicalType::UInt32, false)) + Arc::new(LogicalField::new("b", DataType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(LogicalField::new("a", LogicalType::UInt32, true)) + Arc::new(LogicalField::new("a", DataType::UInt32, true)) ), ], HashMap::new() @@ -1673,7 +1672,7 @@ mod tests { let projection = LogicalPlanBuilder::from(table_scan) .project(vec![Expr::Cast(Cast::new( Box::new(col("c")), - LogicalType::Float64, + DataType::Float64, ))])? .build()?; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 91edc6844313..94a500778da1 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -1147,9 +1147,10 @@ mod tests { use std::any::Any; use std::fmt::{Debug, Formatter}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use arrow::datatypes::{DataType, Field, Schema}; use async_trait::async_trait; - use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; use datafusion_common::ScalarValue; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::table_scan; @@ -2355,10 +2356,10 @@ mod tests { #[async_trait] impl TableSource for PushDownProvider { - fn schema(&self) -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), + fn schema(&self) -> LogicalSchemaRef { + Arc::new(LogicalSchema::new(vec![ + LogicalField::new("a", DataType::Int32, true), + LogicalField::new("b", DataType::Int32, true), ])) } diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index b79343f9b041..0333cc8dde36 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -392,7 +392,7 @@ mod tests { use super::*; use crate::test::*; - use datafusion_common::logical_type::LogicalType; + use arrow::datatypes::DataType; use datafusion_expr::test::function_stub::sum; use datafusion_expr::{col, lit, max, min, out_ref_col, scalar_subquery, Between}; @@ -403,7 +403,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? .project(vec![max(col("orders.o_custkey"))])? @@ -447,7 +447,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("lineitem")) .filter( col("lineitem.l_orderkey") - .eq(out_ref_col(LogicalType::Int64, "orders.o_orderkey")), + .eq(out_ref_col(DataType::Int64, "orders.o_orderkey")), )? .aggregate( Vec::::new(), @@ -461,7 +461,7 @@ mod tests { LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( col("orders.o_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")) + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")) .and(col("orders.o_totalprice").lt(scalar_subquery(lineitem))), )? .aggregate(Vec::::new(), vec![sum(col("orders.o_totalprice"))])? @@ -502,7 +502,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .and(col("o_orderkey").eq(lit(1))), )? @@ -540,8 +540,8 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") - .eq(out_ref_col(LogicalType::Int64, "customer.c_custkey")), + out_ref_col(DataType::Int64, "customer.c_custkey") + .eq(out_ref_col(DataType::Int64, "customer.c_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? .project(vec![max(col("orders.o_custkey"))])? @@ -610,7 +610,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .not_eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -637,7 +637,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .lt(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -664,7 +664,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")) .or(col("o_orderkey").eq(lit(1))), )? @@ -713,7 +713,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -775,7 +775,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -814,7 +814,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -854,7 +854,7 @@ mod tests { let sq = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? @@ -893,7 +893,7 @@ mod tests { fn exists_subquery_correlated() -> Result<()> { let sq = Arc::new( LogicalPlanBuilder::from(test_table_scan_with_name("sq")?) - .filter(out_ref_col(LogicalType::UInt32, "test.a").eq(col("sq.a")))? + .filter(out_ref_col(DataType::UInt32, "test.a").eq(col("sq.a")))? .aggregate(Vec::::new(), vec![min(col("c"))])? .project(vec![min(col("c"))])? .build()?, @@ -989,7 +989,7 @@ mod tests { let sq1 = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![min(col("orders.o_custkey"))])? @@ -999,7 +999,7 @@ mod tests { let sq2 = Arc::new( LogicalPlanBuilder::from(scan_tpch_table("orders")) .filter( - out_ref_col(LogicalType::Int64, "customer.c_custkey") + out_ref_col(DataType::Int64, "customer.c_custkey") .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index c2acdd17c70e..bf7cc3bd9001 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -28,11 +28,9 @@ use arrow::{ }; use datafusion_common::{ - cast::{as_large_list_array, as_list_array}, - tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, + cast::{as_large_list_array, as_list_array}, logical_type::ExtensionType, tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter} }; use datafusion_common::{internal_err, DFSchema, DataFusionError, Result, ScalarValue}; -use datafusion_common::logical_type::extension::ExtensionType; use datafusion_expr::expr::{ AggregateFunctionDefinition, InList, InSubquery, WindowFunction, }; @@ -944,7 +942,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Multiply, right, }) if !info.nullable(&left)? - && !info.get_data_type(&left)?.is_floating() + && !info.get_data_type(&left)?.logical().is_floating() && is_zero(&right) => { Transformed::yes(*right) @@ -955,7 +953,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Multiply, right, }) if !info.nullable(&right)? - && !info.get_data_type(&right)?.is_floating() + && !info.get_data_type(&right)?.logical().is_floating() && is_zero(&left) => { Transformed::yes(*left) @@ -1006,7 +1004,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Modulo, right, }) if !info.nullable(&left)? - && !info.get_data_type(&left)?.is_floating() + && !info.get_data_type(&left)?.logical().is_floating() && is_one(&right) => { Transformed::yes(lit(0)) @@ -1051,7 +1049,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&left, &right) && !info.nullable(&right)? => { Transformed::yes(Expr::Literal(ScalarValue::new_zero( - &info.get_data_type(&left)?.physical_type(), + &info.get_data_type(&left)?.physical(), )?)) } @@ -1062,7 +1060,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&right, &left) && !info.nullable(&left)? => { Transformed::yes(Expr::Literal(ScalarValue::new_zero( - &info.get_data_type(&left)?.physical_type(), + &info.get_data_type(&left)?.physical(), )?)) } @@ -1137,7 +1135,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&left, &right) && !info.nullable(&right)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?.physical_type(), + &info.get_data_type(&left)?.physical(), )?)) } @@ -1148,7 +1146,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&right, &left) && !info.nullable(&left)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?.physical_type(), + &info.get_data_type(&left)?.physical(), )?)) } @@ -1223,7 +1221,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&left, &right) && !info.nullable(&right)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?.physical_type(), + &info.get_data_type(&left)?.physical(), )?)) } @@ -1234,7 +1232,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if is_negative_of(&right, &left) && !info.nullable(&left)? => { Transformed::yes(Expr::Literal(ScalarValue::new_negative_one( - &info.get_data_type(&left)?.physical_type(), + &info.get_data_type(&left)?.physical(), )?)) } @@ -1246,7 +1244,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if expr_contains(&left, &right, BitwiseXor) => { let expr = delete_xor_in_complex_expr(&left, &right, false); Transformed::yes(if expr == *right { - Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&right)?.physical_type())?) + Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&right)?.physical())?) } else { expr }) @@ -1260,7 +1258,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if expr_contains(&right, &left, BitwiseXor) => { let expr = delete_xor_in_complex_expr(&right, &left, true); Transformed::yes(if expr == *left { - Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&left)?.physical_type())?) + Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&left)?.physical())?) } else { expr }) @@ -1770,7 +1768,7 @@ fn inlist_except(mut l1: InList, l2: InList) -> Result { #[cfg(test)] mod tests { - use datafusion_common::{assert_contains, DFSchemaRef, ToDFSchema}; + use datafusion_common::{assert_contains, logical_type::signature::LogicalType, DFSchemaRef, ToDFSchema}; use datafusion_expr::{ function::{ AccumulatorArgs, AggregateFunctionSimplification, @@ -1785,7 +1783,6 @@ mod tests { sync::Arc, }; use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::LogicalType; use datafusion_common::logical_type::schema::LogicalSchema; use crate::simplify_expressions::SimplifyContext; use crate::test::test_table_scan_with_name; @@ -1826,8 +1823,8 @@ mod tests { fn test_schema() -> DFSchemaRef { LogicalSchema::new(vec![ - LogicalField::new("i", LogicalType::Int64, false), - LogicalField::new("b", LogicalType::Boolean, true), + LogicalField::new("i", DataType::Int64, false), + LogicalField::new("b", DataType::Boolean, true), ]) .to_dfschema_ref() .unwrap() @@ -3010,14 +3007,14 @@ mod tests { Arc::new( DFSchema::from_unqualified_fields( vec![ - LogicalField::new("c1", LogicalType::Utf8, true), - LogicalField::new("c2", LogicalType::Boolean, true), - LogicalField::new("c3", LogicalType::Int64, true), - LogicalField::new("c4", LogicalType::UInt32, true), - LogicalField::new("c1_non_null", LogicalType::Utf8, false), - LogicalField::new("c2_non_null", LogicalType::Boolean, false), - LogicalField::new("c3_non_null", LogicalType::Int64, false), - LogicalField::new("c4_non_null", LogicalType::UInt32, false), + LogicalField::new("c1", DataType::Utf8, true), + LogicalField::new("c2", DataType::Boolean, true), + LogicalField::new("c3", DataType::Int64, true), + LogicalField::new("c4", DataType::UInt32, true), + LogicalField::new("c1_non_null", DataType::Utf8, false), + LogicalField::new("c2_non_null", DataType::Boolean, false), + LogicalField::new("c3_non_null", DataType::Int64, false), + LogicalField::new("c4_non_null", DataType::UInt32, false), ] .into(), HashMap::new(), @@ -3105,7 +3102,7 @@ mod tests { #[test] fn simplify_expr_eq() { let schema = expr_test_schema(); - assert_eq!(col("c2").get_type(&schema).unwrap(), LogicalType::Boolean); + assert_eq!(col("c2").get_type(&schema).unwrap().logical(), &LogicalType::Boolean); // true = true -> true assert_eq!(simplify(lit(true).eq(lit(true))), lit(true)); @@ -3129,7 +3126,7 @@ mod tests { // expression to non-boolean. // // Make sure c1 column to be used in tests is not boolean type - assert_eq!(col("c1").get_type(&schema).unwrap(), LogicalType::Utf8); + assert_eq!(col("c1").get_type(&schema).unwrap().logical(), &LogicalType::Utf8); // don't fold c1 = foo assert_eq!(simplify(col("c1").eq(lit("foo"))), col("c1").eq(lit("foo")),); @@ -3139,7 +3136,7 @@ mod tests { fn simplify_expr_not_eq() { let schema = expr_test_schema(); - assert_eq!(col("c2").get_type(&schema).unwrap(), LogicalType::Boolean); + assert_eq!(col("c2").get_type(&schema).unwrap().logical(), &LogicalType::Boolean); // c2 != true -> !c2 assert_eq!(simplify(col("c2").not_eq(lit(true))), col("c2").not(),); @@ -3160,7 +3157,7 @@ mod tests { // when one of the operand is not of boolean type, folding the // other boolean constant will change return type of // expression to non-boolean. - assert_eq!(col("c1").get_type(&schema).unwrap(), LogicalType::Utf8); + assert_eq!(col("c1").get_type(&schema).unwrap().logical(), &LogicalType::Utf8); assert_eq!( simplify(col("c1").not_eq(lit("foo"))), diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 628da80bb3dd..35e4e72c53f8 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -151,7 +151,6 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use chrono::{DateTime, Utc}; - use datafusion_common::logical_type::LogicalType; use crate::optimizer::Optimizer; use datafusion_expr::logical_plan::builder::table_scan_with_filters; use datafusion_expr::logical_plan::table_scan; @@ -445,7 +444,7 @@ mod tests { #[test] fn cast_expr() -> Result<()> { let table_scan = test_table_scan(); - let proj = vec![Expr::Cast(Cast::new(Box::new(lit("0")), LogicalType::Int32))]; + let proj = vec![Expr::Cast(Cast::new(Box::new(lit("0")), DataType::Int32))]; let plan = LogicalPlanBuilder::from(table_scan) .project(proj)? .build()?; @@ -703,9 +702,9 @@ mod tests { let t1 = test_table_scan_with_name("t1")?; let t2 = test_table_scan_with_name("t2")?; - let left_key = col("t1.a") + lit(1i64).cast_to(&LogicalType::UInt32, t1.schema())?; + let left_key = col("t1.a") + lit(1i64).cast_to(&DataType::UInt32.into(), t1.schema())?; let right_key = - col("t2.a") + lit(2i64).cast_to(&LogicalType::UInt32, t2.schema())?; + col("t2.a") + lit(2i64).cast_to(&DataType::UInt32.into(), t2.schema())?; let plan = LogicalPlanBuilder::from(t1) .join_with_expr_keys( t2, diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 5135272119af..b0880c566c75 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -26,12 +26,13 @@ use crate::{OptimizerConfig, OptimizerRule}; use crate::utils::NamePreserver; use arrow::datatypes::{ - TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION, + DataType, TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION }; use arrow::temporal_conversions::{MICROSECONDS, MILLISECONDS, NANOSECONDS}; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{internal_err, DFSchema, DFSchemaRef, Result, ScalarValue}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::expr::{BinaryExpr, Cast, InList, TryCast}; use datafusion_expr::utils::merge_schema; use datafusion_expr::{lit, Expr, ExprSchemable, LogicalPlan, Operator}; @@ -276,37 +277,38 @@ fn is_comparison_op(op: &Operator) -> bool { } /// Returns true if [UnwrapCastExprRewriter] supports this data type -fn is_supported_type(data_type: &LogicalType) -> bool { +fn is_supported_type(data_type: &TypeRelation) -> bool { is_supported_numeric_type(data_type) || is_supported_string_type(data_type) } /// Returns true if [[UnwrapCastExprRewriter]] suppors this numeric type -fn is_supported_numeric_type(data_type: &LogicalType) -> bool { +fn is_supported_numeric_type(data_type: &TypeRelation) -> bool { + use LogicalType::*; matches!( - data_type, - LogicalType::UInt8 - | LogicalType::UInt16 - | LogicalType::UInt32 - | LogicalType::UInt64 - | LogicalType::Int8 - | LogicalType::Int16 - | LogicalType::Int32 - | LogicalType::Int64 - | LogicalType::Decimal128(_, _) - | LogicalType::Timestamp(_, _) + data_type.logical(), + UInt8 + | UInt16 + | UInt32 + | UInt64 + | Int8 + | Int16 + | Int32 + | Int64 + | Decimal128(_, _) + | Timestamp(_, _) ) } /// Returns true if [UnwrapCastExprRewriter] supports casting this value as a string -fn is_supported_string_type(data_type: &LogicalType) -> bool { - matches!(data_type, LogicalType::Utf8 | LogicalType::LargeUtf8) +fn is_supported_string_type(data_type: &TypeRelation) -> bool { + matches!(data_type.logical(), LogicalType::Utf8) } /// Convert a literal value from one data type to another fn try_cast_literal_to_type( lit_value: &ScalarValue, - target_type: &LogicalType, + target_type: &TypeRelation, ) -> Option { let lit_data_type = lit_value.data_type().into(); if !is_supported_type(&lit_data_type) || !is_supported_type(target_type) { @@ -314,7 +316,7 @@ fn try_cast_literal_to_type( } if lit_value.is_null() { // null value can be cast to any type of null value - return ScalarValue::try_from(target_type).ok(); + return ScalarValue::try_from(target_type.physical().clone()).ok(); } try_cast_numeric_literal(lit_value, target_type) .or_else(|| try_cast_string_literal(lit_value, target_type)) @@ -323,7 +325,7 @@ fn try_cast_literal_to_type( /// Convert a numeric value from one numeric data type to another fn try_cast_numeric_literal( lit_value: &ScalarValue, - target_type: &LogicalType, + target_type: &TypeRelation, ) -> Option { let lit_data_type = lit_value.data_type().into(); if !is_supported_numeric_type(&lit_data_type) @@ -332,7 +334,7 @@ fn try_cast_numeric_literal( return None; } - let mul = match target_type { + let mul = match target_type.logical() { LogicalType::UInt8 | LogicalType::UInt16 | LogicalType::UInt32 @@ -345,7 +347,7 @@ fn try_cast_numeric_literal( LogicalType::Decimal128(_, scale) => 10_i128.pow(*scale as u32), _ => return None, }; - let (target_min, target_max) = match target_type { + let (target_min, target_max) = match target_type.logical() { LogicalType::UInt8 => (u8::MIN as i128, u8::MAX as i128), LogicalType::UInt16 => (u16::MIN as i128, u16::MAX as i128), LogicalType::UInt32 => (u32::MIN as i128, u32::MAX as i128), @@ -405,7 +407,7 @@ fn try_cast_numeric_literal( if value >= target_min && value <= target_max { // the value casted from lit to the target type is in the range of target type. // return the target type of scalar value - let result_scalar = match target_type { + let result_scalar = match target_type.logical() { LogicalType::Int8 => ScalarValue::Int8(Some(value as i8)), LogicalType::Int16 => ScalarValue::Int16(Some(value as i16)), LogicalType::Int32 => ScalarValue::Int32(Some(value as i32)), @@ -417,7 +419,7 @@ fn try_cast_numeric_literal( LogicalType::Timestamp(TimeUnit::Second, tz) => { let value = cast_between_timestamp( lit_data_type, - LogicalType::Timestamp(TimeUnit::Second, tz.clone()), + DataType::Timestamp(TimeUnit::Second, tz.clone()), value, ); ScalarValue::TimestampSecond(value, tz.clone()) @@ -425,7 +427,7 @@ fn try_cast_numeric_literal( LogicalType::Timestamp(TimeUnit::Millisecond, tz) => { let value = cast_between_timestamp( lit_data_type, - LogicalType::Timestamp(TimeUnit::Millisecond, tz.clone()), + DataType::Timestamp(TimeUnit::Millisecond, tz.clone()), value, ); ScalarValue::TimestampMillisecond(value, tz.clone()) @@ -433,7 +435,7 @@ fn try_cast_numeric_literal( LogicalType::Timestamp(TimeUnit::Microsecond, tz) => { let value = cast_between_timestamp( lit_data_type, - LogicalType::Timestamp(TimeUnit::Microsecond, tz.clone()), + DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), value, ); ScalarValue::TimestampMicrosecond(value, tz.clone()) @@ -441,7 +443,7 @@ fn try_cast_numeric_literal( LogicalType::Timestamp(TimeUnit::Nanosecond, tz) => { let value = cast_between_timestamp( lit_data_type, - LogicalType::Timestamp(TimeUnit::Nanosecond, tz.clone()), + DataType::Timestamp(TimeUnit::Nanosecond, tz.clone()), value, ); ScalarValue::TimestampNanosecond(value, tz.clone()) @@ -463,24 +465,23 @@ fn try_cast_numeric_literal( fn try_cast_string_literal( lit_value: &ScalarValue, - target_type: &LogicalType, + target_type: &TypeRelation, ) -> Option { let string_value = match lit_value { ScalarValue::Utf8(s) | ScalarValue::LargeUtf8(s) => s.clone(), _ => return None, }; - let scalar_value = match target_type { + let scalar_value = match target_type.logical() { LogicalType::Utf8 => ScalarValue::Utf8(string_value), - LogicalType::LargeUtf8 => ScalarValue::LargeUtf8(string_value), _ => return None, }; Some(scalar_value) } /// Cast a timestamp value from one unit to another -fn cast_between_timestamp(from: LogicalType, to: LogicalType, value: i128) -> Option { +fn cast_between_timestamp(from: impl Into, to: impl Into, value: i128) -> Option { let value = value as i64; - let from_scale = match from { + let from_scale = match from.into().logical() { LogicalType::Timestamp(TimeUnit::Second, _) => 1, LogicalType::Timestamp(TimeUnit::Millisecond, _) => MILLISECONDS, LogicalType::Timestamp(TimeUnit::Microsecond, _) => MICROSECONDS, @@ -488,7 +489,7 @@ fn cast_between_timestamp(from: LogicalType, to: LogicalType, value: i128) -> Op _ => return Some(value), }; - let to_scale = match to { + let to_scale = match to.into().logical() { LogicalType::Timestamp(TimeUnit::Second, _) => 1, LogicalType::Timestamp(TimeUnit::Millisecond, _) => MILLISECONDS, LogicalType::Timestamp(TimeUnit::Microsecond, _) => MICROSECONDS, @@ -509,8 +510,7 @@ mod tests { use super::*; - use arrow::compute::{cast_with_options, CastOptions}; - use datafusion_common::logical_type::extension::ExtensionType; + use arrow::{compute::{cast_with_options, CastOptions}, datatypes::DataType}; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::tree_node::TransformedResult; use datafusion_expr::{cast, col, in_list, try_cast}; @@ -519,7 +519,7 @@ mod tests { fn test_not_unwrap_cast_comparison() { let schema = expr_test_schema(); // cast(INT32(c1), INT64) > INT64(c2) - let c1_gt_c2 = cast(col("c1"), LogicalType::Int64).gt(col("c2")); + let c1_gt_c2 = cast(col("c1"), DataType::Int64).gt(col("c2")); assert_eq!(optimize_test(c1_gt_c2.clone(), &schema), c1_gt_c2); // INT32(c1) < INT32(16), the type is same @@ -527,7 +527,7 @@ mod tests { assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // the 99999999999 is not within the range of MAX(int32) and MIN(int32), we don't cast the lit(99999999999) to int32 type - let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(99999999999i64)); + let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(99999999999i64)); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); } @@ -536,25 +536,25 @@ mod tests { let schema = expr_test_schema(); // cast(c1, INT64) < INT64(16) -> INT32(c1) < cast(INT32(16)) // the 16 is within the range of MAX(int32) and MIN(int32), we can cast the 16 to int32(16) - let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(16i64)); + let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(16i64)); let expected = col("c1").lt(lit(16i32)); assert_eq!(optimize_test(expr_lt, &schema), expected); - let expr_lt = try_cast(col("c1"), LogicalType::Int64).lt(lit(16i64)); + let expr_lt = try_cast(col("c1"), DataType::Int64).lt(lit(16i64)); let expected = col("c1").lt(lit(16i32)); assert_eq!(optimize_test(expr_lt, &schema), expected); // cast(c2, INT32) = INT32(16) => INT64(c2) = INT64(16) - let c2_eq_lit = cast(col("c2"), LogicalType::Int32).eq(lit(16i32)); + let c2_eq_lit = cast(col("c2"), DataType::Int32).eq(lit(16i32)); let expected = col("c2").eq(lit(16i64)); assert_eq!(optimize_test(c2_eq_lit, &schema), expected); // cast(c1, INT64) < INT64(NULL) => INT32(c1) < INT32(NULL) - let c1_lt_lit_null = cast(col("c1"), LogicalType::Int64).lt(null_i64()); + let c1_lt_lit_null = cast(col("c1"), DataType::Int64).lt(null_i64()); let expected = col("c1").lt(null_i32()); assert_eq!(optimize_test(c1_lt_lit_null, &schema), expected); // cast(INT8(NULL), INT32) < INT32(12) => INT8(NULL) < INT8(12) - let lit_lt_lit = cast(null_i8(), LogicalType::Int32).lt(lit(12i32)); + let lit_lt_lit = cast(null_i8(), DataType::Int32).lt(lit(12i32)); let expected = null_i8().lt(lit(12i8)); assert_eq!(optimize_test(lit_lt_lit, &schema), expected); } @@ -563,7 +563,7 @@ mod tests { fn test_unwrap_cast_comparison_unsigned() { // "cast(c6, UINT64) = 0u64 => c6 = 0u32 let schema = expr_test_schema(); - let expr_input = cast(col("c6"), LogicalType::UInt64).eq(lit(0u64)); + let expr_input = cast(col("c6"), DataType::UInt64).eq(lit(0u64)); let expected = col("c6").eq(lit(0u32)); assert_eq!(optimize_test(expr_input, &schema), expected); } @@ -573,28 +573,28 @@ mod tests { let schema = expr_test_schema(); // integer to decimal: value is out of the bounds of the decimal // cast(c3, INT64) = INT64(100000000000000000) - let expr_eq = cast(col("c3"), LogicalType::Int64).eq(lit(100000000000000000i64)); + let expr_eq = cast(col("c3"), DataType::Int64).eq(lit(100000000000000000i64)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // cast(c4, INT64) = INT64(1000) will overflow the i128 - let expr_eq = cast(col("c4"), LogicalType::Int64).eq(lit(1000i64)); + let expr_eq = cast(col("c4"), DataType::Int64).eq(lit(1000i64)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // decimal to decimal: value will lose the scale when convert to the target data type // c3 = DECIMAL(12340,20,4) let expr_eq = - cast(col("c3"), LogicalType::Decimal128(20, 4)).eq(lit_decimal(12340, 20, 4)); + cast(col("c3"), DataType::Decimal128(20, 4)).eq(lit_decimal(12340, 20, 4)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // decimal to integer // c1 = DECIMAL(123, 10, 1): value will lose the scale when convert to the target data type let expr_eq = - cast(col("c1"), LogicalType::Decimal128(10, 1)).eq(lit_decimal(123, 10, 1)); + cast(col("c1"), DataType::Decimal128(10, 1)).eq(lit_decimal(123, 10, 1)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); // c1 = DECIMAL(1230, 10, 2): value will lose the scale when convert to the target data type let expr_eq = - cast(col("c1"), LogicalType::Decimal128(10, 2)).eq(lit_decimal(1230, 10, 2)); + cast(col("c1"), DataType::Decimal128(10, 2)).eq(lit_decimal(1230, 10, 2)); assert_eq!(optimize_test(expr_eq.clone(), &schema), expr_eq); } @@ -603,32 +603,32 @@ mod tests { let schema = expr_test_schema(); // integer to decimal // c3 < INT64(16) -> c3 < (CAST(INT64(16) AS DECIMAL(18,2)); - let expr_lt = try_cast(col("c3"), LogicalType::Int64).lt(lit(16i64)); + let expr_lt = try_cast(col("c3"), DataType::Int64).lt(lit(16i64)); let expected = col("c3").lt(lit_decimal(1600, 18, 2)); assert_eq!(optimize_test(expr_lt, &schema), expected); // c3 < INT64(NULL) - let c1_lt_lit_null = cast(col("c3"), LogicalType::Int64).lt(null_i64()); + let c1_lt_lit_null = cast(col("c3"), DataType::Int64).lt(null_i64()); let expected = col("c3").lt(null_decimal(18, 2)); assert_eq!(optimize_test(c1_lt_lit_null, &schema), expected); // decimal to decimal // c3 < Decimal(123,10,0) -> c3 < CAST(DECIMAL(123,10,0) AS DECIMAL(18,2)) -> c3 < DECIMAL(12300,18,2) let expr_lt = - cast(col("c3"), LogicalType::Decimal128(10, 0)).lt(lit_decimal(123, 10, 0)); + cast(col("c3"), DataType::Decimal128(10, 0)).lt(lit_decimal(123, 10, 0)); let expected = col("c3").lt(lit_decimal(12300, 18, 2)); assert_eq!(optimize_test(expr_lt, &schema), expected); // c3 < Decimal(1230,10,3) -> c3 < CAST(DECIMAL(1230,10,3) AS DECIMAL(18,2)) -> c3 < DECIMAL(123,18,2) let expr_lt = - cast(col("c3"), LogicalType::Decimal128(10, 3)).lt(lit_decimal(1230, 10, 3)); + cast(col("c3"), DataType::Decimal128(10, 3)).lt(lit_decimal(1230, 10, 3)); let expected = col("c3").lt(lit_decimal(123, 18, 2)); assert_eq!(optimize_test(expr_lt, &schema), expected); // decimal to integer // c1 < Decimal(12300, 10, 2) -> c1 < CAST(DECIMAL(12300,10,2) AS INT32) -> c1 < INT32(123) let expr_lt = - cast(col("c1"), LogicalType::Decimal128(10, 2)).lt(lit_decimal(12300, 10, 2)); + cast(col("c1"), DataType::Decimal128(10, 2)).lt(lit_decimal(12300, 10, 2)); let expected = col("c1").lt(lit(123i32)); assert_eq!(optimize_test(expr_lt, &schema), expected); } @@ -639,21 +639,21 @@ mod tests { // internal left type is not supported // FLOAT32(C5) in ... let expr_lt = - cast(col("c5"), LogicalType::Int64).in_list(vec![lit(12i64), lit(12i64)], false); + cast(col("c5"), DataType::Int64).in_list(vec![lit(12i64), lit(12i64)], false); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // cast(INT32(C1), Float32) in (FLOAT32(1.23), Float32(12), Float32(12)) - let expr_lt = cast(col("c1"), LogicalType::Float32) + let expr_lt = cast(col("c1"), DataType::Float32) .in_list(vec![lit(12.0f32), lit(12.0f32), lit(1.23f32)], false); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // INT32(C1) in (INT64(99999999999), INT64(12)) - let expr_lt = cast(col("c1"), LogicalType::Int64) + let expr_lt = cast(col("c1"), DataType::Int64) .in_list(vec![lit(12i32), lit(99999999999i64)], false); assert_eq!(optimize_test(expr_lt.clone(), &schema), expr_lt); // DECIMAL(C3) in (INT64(12), INT32(12), DECIMAL(128,12,3)) - let expr_lt = cast(col("c3"), LogicalType::Decimal128(12, 3)).in_list( + let expr_lt = cast(col("c3"), DataType::Decimal128(12, 3)).in_list( vec![ lit_decimal(12, 12, 3), lit_decimal(12, 12, 3), @@ -669,19 +669,19 @@ mod tests { let schema = expr_test_schema(); // INT32(C1) IN (INT32(12),INT64(24)) -> INT32(C1) IN (INT32(12),INT32(24)) let expr_lt = - cast(col("c1"), LogicalType::Int64).in_list(vec![lit(12i64), lit(24i64)], false); + cast(col("c1"), DataType::Int64).in_list(vec![lit(12i64), lit(24i64)], false); let expected = col("c1").in_list(vec![lit(12i32), lit(24i32)], false); assert_eq!(optimize_test(expr_lt, &schema), expected); // INT32(C2) IN (INT64(NULL),INT64(24)) -> INT32(C1) IN (INT32(12),INT32(24)) let expr_lt = - cast(col("c2"), LogicalType::Int32).in_list(vec![null_i32(), lit(14i32)], false); + cast(col("c2"), DataType::Int32).in_list(vec![null_i32(), lit(14i32)], false); let expected = col("c2").in_list(vec![null_i64(), lit(14i64)], false); assert_eq!(optimize_test(expr_lt, &schema), expected); // decimal test case // c3 is decimal(18,2) - let expr_lt = cast(col("c3"), LogicalType::Decimal128(19, 3)).in_list( + let expr_lt = cast(col("c3"), DataType::Decimal128(19, 3)).in_list( vec![ lit_decimal(12000, 19, 3), lit_decimal(24000, 19, 3), @@ -702,7 +702,7 @@ mod tests { assert_eq!(optimize_test(expr_lt, &schema), expected); // cast(INT32(12), INT64) IN (.....) - let expr_lt = cast(lit(12i32), LogicalType::Int64) + let expr_lt = cast(lit(12i32), DataType::Int64) .in_list(vec![lit(13i64), lit(12i64)], false); let expected = lit(12i32).in_list(vec![lit(13i32), lit(12i32)], false); assert_eq!(optimize_test(expr_lt, &schema), expected); @@ -713,7 +713,7 @@ mod tests { let schema = expr_test_schema(); // c1 < INT64(16) -> c1 < cast(INT32(16)) // the 16 is within the range of MAX(int32) and MIN(int32), we can cast the 16 to int32(16) - let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(16i64)).alias("x"); + let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(16i64)).alias("x"); let expected = col("c1").lt(lit(16i32)).alias("x"); assert_eq!(optimize_test(expr_lt, &schema), expected); } @@ -723,9 +723,9 @@ mod tests { let schema = expr_test_schema(); // c1 < INT64(16) OR c1 > INT64(32) -> c1 < INT32(16) OR c1 > INT32(32) // the 16 and 32 are within the range of MAX(int32) and MIN(int32), we can cast them to int32 - let expr_lt = cast(col("c1"), LogicalType::Int64).lt(lit(16i64)).or(cast( + let expr_lt = cast(col("c1"), DataType::Int64).lt(lit(16i64)).or(cast( col("c1"), - LogicalType::Int64, + DataType::Int64, ) .gt(lit(32i64))); let expected = col("c1").lt(lit(16i32)).or(col("c1").gt(lit(32i32))); @@ -738,12 +738,12 @@ mod tests { // but the type of c6 is uint32 // the rewriter will not throw error and just return the original expr let schema = expr_test_schema(); - let expr_input = cast(col("c6"), LogicalType::Float64).eq(lit(0f64)); + let expr_input = cast(col("c6"), DataType::Float64).eq(lit(0f64)); assert_eq!(optimize_test(expr_input.clone(), &schema), expr_input); // inlist for unsupported data type let expr_input = - in_list(cast(col("c6"), LogicalType::Float64), vec![lit(0f64)], false); + in_list(cast(col("c6"), DataType::Float64), vec![lit(0f64)], false); assert_eq!(optimize_test(expr_input.clone(), &schema), expr_input); } @@ -770,16 +770,16 @@ mod tests { Arc::new( DFSchema::from_unqualified_fields( vec![ - LogicalField::new("c1", LogicalType::Int32, false), - LogicalField::new("c2", LogicalType::Int64, false), - LogicalField::new("c3", LogicalType::Decimal128(18, 2), false), - LogicalField::new("c4", LogicalType::Decimal128(38, 37), false), - LogicalField::new("c5", LogicalType::Float32, false), - LogicalField::new("c6", LogicalType::UInt32, false), + LogicalField::new("c1", DataType::Int32, false), + LogicalField::new("c2", DataType::Int64, false), + LogicalField::new("c3", DataType::Decimal128(18, 2), false), + LogicalField::new("c4", DataType::Decimal128(38, 37), false), + LogicalField::new("c5", DataType::Float32, false), + LogicalField::new("c6", DataType::UInt32, false), LogicalField::new("ts_nano_none", timestamp_nano_none_type(), false), LogicalField::new("ts_nano_utf", timestamp_nano_utc_type(), false), - LogicalField::new("str1", LogicalType::Utf8, false), - LogicalField::new("largestr", LogicalType::LargeUtf8, false), + LogicalField::new("str1", DataType::Utf8, false), + LogicalField::new("largestr", DataType::LargeUtf8, false), ] .into(), HashMap::new(), @@ -817,14 +817,14 @@ mod tests { lit(ScalarValue::Decimal128(None, precision, scale)) } - fn timestamp_nano_none_type() -> LogicalType { - LogicalType::Timestamp(TimeUnit::Nanosecond, None) + fn timestamp_nano_none_type() -> TypeRelation { + DataType::Timestamp(TimeUnit::Nanosecond, None).into() } // this is the type that now() returns - fn timestamp_nano_utc_type() -> LogicalType { + fn timestamp_nano_utc_type() -> TypeRelation { let utc = Some("+0:00".into()); - LogicalType::Timestamp(TimeUnit::Nanosecond, utc) + DataType::Timestamp(TimeUnit::Nanosecond, utc).into() } #[test] @@ -849,7 +849,7 @@ mod tests { for s2 in &scalars { let expected_value = ExpectedCast::Value(s2.clone()); - expect_cast(s1.clone(), s2.data_type().into(), expected_value); + expect_cast(s1.clone(), s2.data_type(), expected_value); } } } @@ -874,28 +874,28 @@ mod tests { for s2 in &scalars { let expected_value = ExpectedCast::Value(s2.clone()); - expect_cast(s1.clone(), s2.data_type().into(), expected_value); + expect_cast(s1.clone(), s2.data_type(), expected_value); } } let max_i32 = ScalarValue::Int32(Some(i32::MAX)); expect_cast( max_i32, - LogicalType::UInt64, + DataType::UInt64, ExpectedCast::Value(ScalarValue::UInt64(Some(i32::MAX as u64))), ); let min_i32 = ScalarValue::Int32(Some(i32::MIN)); expect_cast( min_i32, - LogicalType::Int64, + DataType::Int64, ExpectedCast::Value(ScalarValue::Int64(Some(i32::MIN as i64))), ); let max_i64 = ScalarValue::Int64(Some(i64::MAX)); expect_cast( max_i64, - LogicalType::UInt64, + DataType::UInt64, ExpectedCast::Value(ScalarValue::UInt64(Some(i64::MAX as u64))), ); } @@ -907,28 +907,28 @@ mod tests { let max_i64 = ScalarValue::Int64(Some(i64::MAX)); let max_u64 = ScalarValue::UInt64(Some(u64::MAX)); - expect_cast(max_i64.clone(), LogicalType::Int8, ExpectedCast::NoValue); + expect_cast(max_i64.clone(), DataType::Int8, ExpectedCast::NoValue); - expect_cast(max_i64.clone(), LogicalType::Int16, ExpectedCast::NoValue); + expect_cast(max_i64.clone(), DataType::Int16, ExpectedCast::NoValue); - expect_cast(max_i64, LogicalType::Int32, ExpectedCast::NoValue); + expect_cast(max_i64, DataType::Int32, ExpectedCast::NoValue); - expect_cast(max_u64, LogicalType::Int64, ExpectedCast::NoValue); + expect_cast(max_u64, DataType::Int64, ExpectedCast::NoValue); - expect_cast(min_i64, LogicalType::UInt64, ExpectedCast::NoValue); + expect_cast(min_i64, DataType::UInt64, ExpectedCast::NoValue); - expect_cast(min_i32, LogicalType::UInt64, ExpectedCast::NoValue); + expect_cast(min_i32, DataType::UInt64, ExpectedCast::NoValue); // decimal out of range expect_cast( ScalarValue::Decimal128(Some(99999999999999999999999999999999999900), 38, 0), - LogicalType::Int64, + DataType::Int64, ExpectedCast::NoValue, ); expect_cast( ScalarValue::Decimal128(Some(-9999999999999999999999999999999999), 37, 1), - LogicalType::Int64, + DataType::Int64, ExpectedCast::NoValue, ); } @@ -937,19 +937,19 @@ mod tests { fn test_try_decimal_cast_in_range() { expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - LogicalType::Decimal128(3, 0), + DataType::Decimal128(3, 0), ExpectedCast::Value(ScalarValue::Decimal128(Some(123), 3, 0)), ); expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - LogicalType::Decimal128(8, 0), + DataType::Decimal128(8, 0), ExpectedCast::Value(ScalarValue::Decimal128(Some(123), 8, 0)), ); expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - LogicalType::Decimal128(8, 5), + DataType::Decimal128(8, 5), ExpectedCast::Value(ScalarValue::Decimal128(Some(12300000), 8, 5)), ); } @@ -959,14 +959,14 @@ mod tests { // decimal would lose precision expect_cast( ScalarValue::Decimal128(Some(12345), 5, 2), - LogicalType::Decimal128(3, 0), + DataType::Decimal128(3, 0), ExpectedCast::NoValue, ); // decimal would lose precision expect_cast( ScalarValue::Decimal128(Some(12300), 5, 2), - LogicalType::Decimal128(2, 0), + DataType::Decimal128(2, 0), ExpectedCast::NoValue, ); } @@ -1008,10 +1008,10 @@ mod tests { assert_eq!(lit_tz_none, lit_tz_utc); // e.g. LogicalType::Timestamp(_, None) - let dt_tz_none: LogicalType = lit_tz_none.data_type().into(); + let dt_tz_none: TypeRelation = lit_tz_none.data_type().into(); // e.g. LogicalType::Timestamp(_, Some(utc)) - let dt_tz_utc: LogicalType = lit_tz_utc.data_type().into(); + let dt_tz_utc: TypeRelation = lit_tz_utc.data_type().into(); // None <--> None expect_cast( @@ -1044,7 +1044,7 @@ mod tests { // timestamp to int64 expect_cast( lit_tz_utc.clone(), - LogicalType::Int64, + DataType::Int64, ExpectedCast::Value(ScalarValue::Int64(Some(12345))), ); @@ -1065,7 +1065,7 @@ mod tests { // timestamp to string (not supported yet) expect_cast( lit_tz_utc.clone(), - LogicalType::LargeUtf8, + DataType::LargeUtf8, ExpectedCast::NoValue, ); } @@ -1076,7 +1076,7 @@ mod tests { // int64 to list expect_cast( ScalarValue::Int64(Some(12345)), - LogicalType::new_list(LogicalType::Int32, true), + DataType::new_list(DataType::Int32, true), ExpectedCast::NoValue, ); } @@ -1094,9 +1094,10 @@ mod tests { /// casting is consistent with the Arrow kernels fn expect_cast( literal: ScalarValue, - target_type: LogicalType, + target_type: impl Into, expected_result: ExpectedCast, ) { + let target_type = target_type.into(); let actual_value = try_cast_literal_to_type(&literal, &target_type); println!("expect_cast: "); @@ -1122,7 +1123,7 @@ mod tests { .expect("Failed to convert to array of size"); let cast_array = cast_with_options( &literal_array, - &target_type.physical_type(), + target_type.physical(), &CastOptions::default(), ) .expect("Expected to be cast array with arrow cast kernel"); @@ -1135,8 +1136,8 @@ mod tests { // Verify that for timestamp types the timezones are the same // (ScalarValue::cmp doesn't account for timezones); if let ( - LogicalType::Timestamp(left_unit, left_tz), - LogicalType::Timestamp(right_unit, right_tz), + DataType::Timestamp(left_unit, left_tz), + DataType::Timestamp(right_unit, right_tz), ) = (actual_value.data_type().into(), expected_value.data_type().into()) { assert_eq!(left_unit, right_unit); @@ -1157,7 +1158,7 @@ mod tests { // same timestamp let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &LogicalType::Timestamp(TimeUnit::Nanosecond, None), + &DataType::Timestamp(TimeUnit::Nanosecond, None).into(), ) .unwrap(); @@ -1169,7 +1170,7 @@ mod tests { // TimestampNanosecond to TimestampMicrosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &LogicalType::Timestamp(TimeUnit::Microsecond, None), + &DataType::Timestamp(TimeUnit::Microsecond, None).into(), ) .unwrap(); @@ -1181,7 +1182,7 @@ mod tests { // TimestampNanosecond to TimestampMillisecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &LogicalType::Timestamp(TimeUnit::Millisecond, None), + &DataType::Timestamp(TimeUnit::Millisecond, None).into(), ) .unwrap(); @@ -1190,7 +1191,7 @@ mod tests { // TimestampNanosecond to TimestampSecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampNanosecond(Some(123456), None), - &LogicalType::Timestamp(TimeUnit::Second, None), + &DataType::Timestamp(TimeUnit::Second, None).into(), ) .unwrap(); @@ -1199,7 +1200,7 @@ mod tests { // TimestampMicrosecond to TimestampNanosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMicrosecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Nanosecond, None), + &DataType::Timestamp(TimeUnit::Nanosecond, None).into(), ) .unwrap(); @@ -1211,7 +1212,7 @@ mod tests { // TimestampMicrosecond to TimestampMillisecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMicrosecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Millisecond, None), + &DataType::Timestamp(TimeUnit::Millisecond, None).into(), ) .unwrap(); @@ -1220,7 +1221,7 @@ mod tests { // TimestampMicrosecond to TimestampSecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMicrosecond(Some(123456789), None), - &LogicalType::Timestamp(TimeUnit::Second, None), + &DataType::Timestamp(TimeUnit::Second, None).into(), ) .unwrap(); assert_eq!(new_scalar, ScalarValue::TimestampSecond(Some(123), None)); @@ -1228,7 +1229,7 @@ mod tests { // TimestampMillisecond to TimestampNanosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMillisecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Nanosecond, None), + &DataType::Timestamp(TimeUnit::Nanosecond, None).into(), ) .unwrap(); assert_eq!( @@ -1239,7 +1240,7 @@ mod tests { // TimestampMillisecond to TimestampMicrosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMillisecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Microsecond, None), + &DataType::Timestamp(TimeUnit::Microsecond, None).into(), ) .unwrap(); assert_eq!( @@ -1249,7 +1250,7 @@ mod tests { // TimestampMillisecond to TimestampSecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampMillisecond(Some(123456789), None), - &LogicalType::Timestamp(TimeUnit::Second, None), + &DataType::Timestamp(TimeUnit::Second, None).into(), ) .unwrap(); assert_eq!(new_scalar, ScalarValue::TimestampSecond(Some(123456), None)); @@ -1257,7 +1258,7 @@ mod tests { // TimestampSecond to TimestampNanosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Nanosecond, None), + &DataType::Timestamp(TimeUnit::Nanosecond, None).into(), ) .unwrap(); assert_eq!( @@ -1268,7 +1269,7 @@ mod tests { // TimestampSecond to TimestampMicrosecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Microsecond, None), + &DataType::Timestamp(TimeUnit::Microsecond, None).into(), ) .unwrap(); assert_eq!( @@ -1279,7 +1280,7 @@ mod tests { // TimestampSecond to TimestampMillisecond let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(123), None), - &LogicalType::Timestamp(TimeUnit::Millisecond, None), + &DataType::Timestamp(TimeUnit::Millisecond, None).into(), ) .unwrap(); assert_eq!( @@ -1290,7 +1291,7 @@ mod tests { // overflow let new_scalar = try_cast_literal_to_type( &ScalarValue::TimestampSecond(Some(i64::MAX), None), - &LogicalType::Timestamp(TimeUnit::Millisecond, None), + &DataType::Timestamp(TimeUnit::Millisecond, None).into(), ) .unwrap(); assert_eq!(new_scalar, ScalarValue::TimestampMillisecond(None, None)); @@ -1307,7 +1308,7 @@ mod tests { for s2 in &scalars { let expected_value = ExpectedCast::Value(s2.clone()); - expect_cast(s1.clone(), s2.data_type().into(), expected_value); + expect_cast(s1.clone(), s2.data_type(), expected_value); } } } diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 33678a98da08..4843cca9c633 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -23,7 +23,8 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use datafusion_common::config::ConfigOptions; use datafusion_common::{plan_err, Result}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::TypeRelation; use datafusion_expr::test::function_stub::sum_udaf; use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF}; use datafusion_functions_aggregate::average::avg_udaf; @@ -400,7 +401,7 @@ impl ContextProvider for MyContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -434,7 +435,7 @@ impl TableSource for MyTableSource { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.schema.clone().into()) } } diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 39305401288d..1212d6113cc0 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -21,8 +21,8 @@ use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::Schema; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{exec_err, Result}; -use datafusion_common::logical_type::extension::ExtensionType; use datafusion_expr::expr::Alias; use datafusion_expr::sort_properties::ExprProperties; use datafusion_expr::Expr; @@ -128,7 +128,8 @@ pub fn limited_convert_logical_expr_to_physical_expr( cast_expr.expr.as_ref(), schema, )?, - cast_expr.data_type.physical_type(), + // TODO(@notfilippo): do not convert to physical type + cast_expr.data_type.physical().clone(), None, ))), Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index ae0f6e455532..8bac68e108d4 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -24,8 +24,8 @@ use crate::{ }; use arrow::datatypes::Schema; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{exec_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, ToDFSchema}; -use datafusion_common::logical_type::extension::ExtensionType; use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction}; @@ -259,12 +259,12 @@ pub fn create_physical_expr( Expr::Cast(Cast { expr, data_type }) => expressions::cast( create_physical_expr(expr, input_dfschema, execution_props)?, input_schema, - data_type.clone().physical_type(), + data_type.physical().clone(), ), Expr::TryCast(TryCast { expr, data_type }) => expressions::try_cast( create_physical_expr(expr, input_dfschema, execution_props)?, input_schema, - data_type.clone().physical_type(), + data_type.physical().clone(), ), Expr::Not(expr) => { expressions::not(create_physical_expr(expr, input_dfschema, execution_props)?) diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 6f54ac19626f..abb4f3d814f9 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -43,7 +43,7 @@ use datafusion_common::{ DataFusionError, JoinSide, ScalarValue, Statistics, TableReference, }; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; #[derive(Debug)] pub enum Error { @@ -192,7 +192,7 @@ impl TryFrom for DFSchemaRef { } } -impl TryFrom<&protobuf::ArrowType> for LogicalType { +impl TryFrom<&protobuf::ArrowType> for TypeRelation { type Error = Error; fn try_from( diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index d343554e6fe6..2b611a8ffce6 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -36,8 +36,7 @@ use datafusion_common::{ Column, ColumnStatistics, Constraint, Constraints, DFSchema, DFSchemaRef, DataFusionError, JoinSide, ScalarValue, Statistics, }; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; #[derive(Debug)] pub enum Error { @@ -111,11 +110,11 @@ impl TryFrom<&DataType> for protobuf::ArrowType { } } -impl TryFrom<&LogicalType> for protobuf::ArrowType { +impl TryFrom<&TypeRelation> for protobuf::ArrowType { type Error = Error; - fn try_from(val: &LogicalType) -> Result { - let arrow_type_enum: ArrowTypeEnum = (&val.physical_type()).try_into()?; + fn try_from(val: &TypeRelation) -> Result { + let arrow_type_enum: ArrowTypeEnum = val.physical().try_into()?; Ok(Self { arrow_type_enum: Some(arrow_type_enum), }) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 21331a94c18c..2a06c0e0c877 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use datafusion::execution::registry::FunctionRegistry; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ internal_err, plan_datafusion_err, DataFusionError, Result, ScalarValue, TableReference, UnnestOptions, @@ -552,7 +553,7 @@ pub fn parse_expr( "expr", codec, )?); - let data_type = cast.arrow_type.as_ref().required("arrow_type")?; + let data_type: TypeRelation = cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::Cast(Cast::new(expr, data_type))) } ExprType::TryCast(cast) => { @@ -562,7 +563,7 @@ pub fn parse_expr( "expr", codec, )?); - let data_type = cast.arrow_type.as_ref().required("arrow_type")?; + let data_type: TypeRelation = cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::TryCast(TryCast::new(expr, data_type))) } ExprType::Sort(sort) => Ok(Expr::Sort(Sort::new( diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index be21a5aa2178..05412f02de33 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -65,8 +65,7 @@ use datafusion_expr::{ use prost::bytes::BufMut; use prost::Message; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_proto_common::ArrowType; use self::to_proto::serialize_expr; @@ -838,7 +837,7 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Prepare(prepare) => { let input: LogicalPlan = into_logical_plan!(prepare.input, ctx, extension_codec)?; - let data_types: Vec = prepare + let data_types: Vec = prepare .data_types .iter() .map(|t| DataType::try_from(t).map(|t| t.into())) @@ -936,7 +935,7 @@ impl AsLogicalPlan for LogicalPlanNode { .. }) => { let provider = source_as_provider(source)?; - let schema = provider.schema(); + let schema= SchemaRef::new(provider.schema().as_ref().clone().into()); let source = provider.as_any(); let projection = match projection { @@ -1562,7 +1561,7 @@ impl AsLogicalPlan for LogicalPlanNode { name: name.clone(), data_types: data_types .iter() - .map(|t| (&t.physical_type()).try_into()) + .map(|t| t.physical().try_into()) .collect::, _>>()?, input: Some(Box::new(input)), }, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 906f1762d1e1..9a7010284be4 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -27,7 +27,6 @@ use arrow::datatypes::{ IntervalUnit, Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, }; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::LogicalType; use datafusion::datasource::file_format::arrow::ArrowFormatFactory; use datafusion::datasource::file_format::csv::CsvFormatFactory; use datafusion::datasource::file_format::format_as_file_type; @@ -592,7 +591,7 @@ async fn roundtrip_expr_api() -> Result<()> { // list of expressions to round trip let expr_list = vec![ - encode(col("a").cast_to(&LogicalType::Utf8, &schema)?, lit("hex")), + encode(col("a").cast_to(&DataType::Utf8.into(), &schema)?, lit("hex")), decode(lit("1234"), lit("hex")), array_to_string(make_array(vec![lit(1), lit(2), lit(3)]), lit(",")), array_dims(make_array(vec![lit(1), lit(2), lit(3)])), @@ -700,7 +699,7 @@ async fn roundtrip_expr_api() -> Result<()> { bit_and(lit(2)), bit_or(lit(2)), bit_xor(lit(2)), - string_agg(col("a").cast_to(&LogicalType::Utf8, &schema)?, lit("|")), + string_agg(col("a").cast_to(&DataType::Utf8.into(), &schema)?, lit("|")), bool_and(lit(true)), bool_or(lit(true)), ]; @@ -1561,11 +1560,11 @@ fn roundtrip_schema() { fn roundtrip_dfschema() { let dfschema = DFSchema::new_with_metadata( vec![ - (None, Arc::new(LogicalField::new("a", LogicalType::Int64, false))), + (None, Arc::new(LogicalField::new("a", DataType::Int64, false))), ( Some("t".into()), Arc::new( - LogicalField::new("b", LogicalType::Decimal128(15, 2), true).with_metadata( + LogicalField::new("b", DataType::Decimal128(15, 2), true).with_metadata( HashMap::from([(String::from("k1"), String::from("v1"))]), ), ), @@ -1694,7 +1693,7 @@ fn roundtrip_null_literal() { #[test] fn roundtrip_cast() { - let test_expr = Expr::Cast(Cast::new(Box::new(lit(1.0_f32)), LogicalType::Boolean)); + let test_expr = Expr::Cast(Cast::new(Box::new(lit(1.0_f32)), DataType::Boolean)); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); @@ -1703,13 +1702,13 @@ fn roundtrip_cast() { #[test] fn roundtrip_try_cast() { let test_expr = - Expr::TryCast(TryCast::new(Box::new(lit(1.0_f32)), LogicalType::Boolean)); + Expr::TryCast(TryCast::new(Box::new(lit(1.0_f32)), DataType::Boolean)); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); let test_expr = - Expr::TryCast(TryCast::new(Box::new(lit("not a bool")), LogicalType::Boolean)); + Expr::TryCast(TryCast::new(Box::new(lit("not a bool")), DataType::Boolean)); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); diff --git a/datafusion/sql/examples/sql.rs b/datafusion/sql/examples/sql.rs index 159bf46f97c6..9c27433cd129 100644 --- a/datafusion/sql/examples/sql.rs +++ b/datafusion/sql/examples/sql.rs @@ -30,7 +30,7 @@ use datafusion_sql::{ TableReference, }; use std::{collections::HashMap, sync::Arc}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; fn main() { let sql = "SELECT \ @@ -133,7 +133,7 @@ impl ContextProvider for MyContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index a1e4719d7c70..d9f2578725d6 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -16,6 +16,7 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::{ internal_datafusion_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, Dependency, Result, @@ -35,7 +36,7 @@ use sqlparser::ast::{ }; use std::str::FromStr; use strum::IntoEnumIterator; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::ExtensionType; /// Suggest a valid function based on an invalid input function name pub fn suggest_valid_function( @@ -473,11 +474,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn check_unnest_arg(arg: &Expr, schema: &DFSchema) -> Result<()> { // Check argument type, array types are supported - match arg.get_type(schema)? { + match arg.get_type(schema)?.logical() { LogicalType::List(_) - | LogicalType::LargeList(_) - | LogicalType::FixedSizeList(_, _) - | LogicalType::Struct(_) => Ok(()), + | LogicalType::Struct(_) => Ok(()), LogicalType::Null => { not_impl_err!("unnest() does not support null yet") } diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 04ecdce3f164..b9a8f7d35245 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::TimeUnit; +use arrow_schema::{DataType, TimeUnit}; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::utils::list_ndims; use sqlparser::ast::{CastKind, Expr as SQLExpr, Subscript, TrimWhereField, Value}; @@ -23,8 +24,7 @@ use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, }; -use datafusion_common::logical_type::extension::ExtensionType; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::expr::InList; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ @@ -114,8 +114,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if op == Operator::StringConcat { let left_type = left.get_type(schema)?; let right_type = right.get_type(schema)?; - let left_list_ndims = list_ndims(&left_type.physical_type()); - let right_list_ndims = list_ndims(&right_type.physical_type()); + let left_list_ndims = list_ndims(&left_type.physical()); + let right_list_ndims = list_ndims(&right_type.physical()); // We determine the target function to rewrite based on the list n-dimension, the check is not exact but sufficient. // The exact validity check is handled in the actual function, so even if there is 3d list appended with 1d list, it is also fine to rewrite. @@ -351,13 +351,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // numeric constants are treated as seconds (rather as nanoseconds) // to align with postgres / duckdb semantics - let expr = match &dt { + let expr = match dt.logical() { LogicalType::Timestamp(TimeUnit::Nanosecond, tz) - if expr.get_type(schema)? == LogicalType::Int64 => + if expr.get_type(schema)?.logical() == &LogicalType::Int64 => { Expr::Cast(Cast::new( Box::new(expr), - LogicalType::Timestamp(TimeUnit::Second, tz.clone()), + TypeRelation::from(DataType::Timestamp(TimeUnit::Second, tz.clone())), )) } _ => expr, @@ -636,7 +636,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { )?), match *time_zone { SQLExpr::Value(Value::SingleQuotedString(s)) => { - LogicalType::Timestamp(TimeUnit::Nanosecond, Some(s.into())) + TypeRelation::from(DataType::Timestamp(TimeUnit::Nanosecond, Some(s.into()))) } _ => { return not_impl_err!( @@ -814,7 +814,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let pattern = self.sql_expr_to_logical_expr(pattern, schema, planner_context)?; let pattern_type = pattern.get_type(schema)?; - if pattern_type != LogicalType::Utf8 && pattern_type != LogicalType::Null { + if !matches!(pattern_type.logical(), LogicalType::Utf8 | LogicalType::Null) { return plan_err!("Invalid pattern in LIKE expression"); } let escape_char = if let Some(char) = escape_char { @@ -845,7 +845,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let pattern = self.sql_expr_to_logical_expr(pattern, schema, planner_context)?; let pattern_type = pattern.get_type(schema)?; - if pattern_type != LogicalType::Utf8 && pattern_type != LogicalType::Null { + if !matches!(pattern_type.logical(), LogicalType::Utf8 | LogicalType::Null) { return plan_err!("Invalid pattern in SIMILAR TO expression"); } let escape_char = if let Some(char) = escape_char { @@ -1076,7 +1076,7 @@ mod tests { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index a9a2698397a9..1b5196bb4c95 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -27,13 +27,13 @@ use log::debug; use sqlparser::ast::{BinaryOperator, Expr as SQLExpr, Interval, Value}; use sqlparser::parser::ParserError::ParserError; use std::borrow::Cow; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn parse_value( &self, value: Value, - param_data_types: &[LogicalType], + param_data_types: &[TypeRelation], ) -> Result { match value { Value::Number(n, _) => self.parse_sql_number(&n, false), @@ -96,7 +96,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// number 1, 2, ... etc. For example, `$1` is the first placeholder; $2 is the second one and so on. fn create_placeholder_expr( param: String, - param_data_types: &[LogicalType], + param_data_types: &[TypeRelation], ) -> Result { // Parse the placeholder as a number because it is the only support from sqlparser and postgres let index = param[1..].parse::(); diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index c0dd69a2d828..46c8818edcb9 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -39,7 +39,7 @@ use datafusion_common::{ }; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::logical_plan::{LogicalPlan, LogicalPlanBuilder}; use datafusion_expr::utils::find_column_exprs; @@ -87,7 +87,7 @@ pub trait ContextProvider { /// Getter for a UDWF fn get_window_meta(&self, name: &str) -> Option>; /// Getter for system/user-defined variable type - fn get_variable_type(&self, variable_names: &[String]) -> Option; + fn get_variable_type(&self, variable_names: &[String]) -> Option; /// Get configuration options fn options(&self) -> &ConfigOptions; @@ -162,7 +162,7 @@ impl IdentNormalizer { pub struct PlannerContext { /// Data types for numbered parameters ($1, $2, etc), if supplied /// in `PREPARE` statement - prepare_param_data_types: Arc>, + prepare_param_data_types: Arc>, /// Map of CTE name to logical plan of the WITH clause. /// Use `Arc` to allow cheap cloning ctes: HashMap>, @@ -189,7 +189,7 @@ impl PlannerContext { /// Update the PlannerContext with provided prepare_param_data_types pub fn with_prepare_param_data_types( mut self, - prepare_param_data_types: Vec, + prepare_param_data_types: Vec, ) -> Self { self.prepare_param_data_types = prepare_param_data_types.into(); self @@ -211,7 +211,7 @@ impl PlannerContext { } /// Return the types of parameters (`$1`, `$2`, etc) if known - pub fn prepare_param_data_types(&self) -> &[LogicalType] { + pub fn prepare_param_data_types(&self) -> &[TypeRelation] { &self.prepare_param_data_types } @@ -383,13 +383,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) } - pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { + pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { SQLDataType::Array(ArrayElemTypeDef::AngleBracket(inner_sql_type)) | SQLDataType::Array(ArrayElemTypeDef::SquareBracket(inner_sql_type, _)) => { // Arrays may be multi-dimensional. let inner_data_type = self.convert_data_type(inner_sql_type)?; - Ok(LogicalType::new_list(inner_data_type, true).into()) + Ok(TypeRelation::new_list(inner_data_type, true).into()) } SQLDataType::Array(ArrayElemTypeDef::None) => { not_impl_err!("Arrays with unspecified type is not supported") @@ -398,31 +398,31 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - fn convert_simple_data_type(&self, sql_type: &SQLDataType) -> Result { + fn convert_simple_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { - SQLDataType::Boolean | SQLDataType::Bool => Ok(LogicalType::Boolean), - SQLDataType::TinyInt(_) => Ok(LogicalType::Int8), - SQLDataType::SmallInt(_) | SQLDataType::Int2(_) => Ok(LogicalType::Int16), - SQLDataType::Int(_) | SQLDataType::Integer(_) | SQLDataType::Int4(_) => Ok(LogicalType::Int32), - SQLDataType::BigInt(_) | SQLDataType::Int8(_) => Ok(LogicalType::Int64), - SQLDataType::UnsignedTinyInt(_) => Ok(LogicalType::UInt8), - SQLDataType::UnsignedSmallInt(_) | SQLDataType::UnsignedInt2(_) => Ok(LogicalType::UInt16), + SQLDataType::Boolean | SQLDataType::Bool => Ok(DataType::Boolean.into()), + SQLDataType::TinyInt(_) => Ok(DataType::Int8.into()), + SQLDataType::SmallInt(_) | SQLDataType::Int2(_) => Ok(DataType::Int16.into()), + SQLDataType::Int(_) | SQLDataType::Integer(_) | SQLDataType::Int4(_) => Ok(DataType::Int32.into()), + SQLDataType::BigInt(_) | SQLDataType::Int8(_) => Ok(DataType::Int64.into()), + SQLDataType::UnsignedTinyInt(_) => Ok(DataType::UInt8.into()), + SQLDataType::UnsignedSmallInt(_) | SQLDataType::UnsignedInt2(_) => Ok(DataType::UInt16.into()), SQLDataType::UnsignedInt(_) | SQLDataType::UnsignedInteger(_) | SQLDataType::UnsignedInt4(_) => { - Ok(LogicalType::UInt32) + Ok(DataType::UInt32.into()) } SQLDataType::Varchar(length) => { match (length, self.options.support_varchar_with_length) { (Some(_), false) => plan_err!("does not support Varchar with length, please set `support_varchar_with_length` to be true"), - _ => Ok(LogicalType::Utf8), + _ => Ok(DataType::Utf8.into()), } } - SQLDataType::UnsignedBigInt(_) | SQLDataType::UnsignedInt8(_) => Ok(LogicalType::UInt64), - SQLDataType::Float(_) => Ok(LogicalType::Float32), - SQLDataType::Real | SQLDataType::Float4 => Ok(LogicalType::Float32), - SQLDataType::Double | SQLDataType::DoublePrecision | SQLDataType::Float8 => Ok(LogicalType::Float64), + SQLDataType::UnsignedBigInt(_) | SQLDataType::UnsignedInt8(_) => Ok(DataType::UInt64.into()), + SQLDataType::Float(_) => Ok(DataType::Float32.into()), + SQLDataType::Real | SQLDataType::Float4 => Ok(DataType::Float32.into()), + SQLDataType::Double | SQLDataType::DoublePrecision | SQLDataType::Float8 => Ok(DataType::Float64.into()), SQLDataType::Char(_) | SQLDataType::Text - | SQLDataType::String(_) => Ok(LogicalType::Utf8), + | SQLDataType::String(_) => Ok(DataType::Utf8.into()), SQLDataType::Timestamp(None, tz_info) => { let tz = if matches!(tz_info, TimezoneInfo::Tz) || matches!(tz_info, TimezoneInfo::WithTimeZone) @@ -435,14 +435,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Timestamp Without Time zone None }; - Ok(LogicalType::Timestamp(TimeUnit::Nanosecond, tz.map(Into::into))) + Ok(DataType::Timestamp(TimeUnit::Nanosecond, tz.map(Into::into)).into()) } - SQLDataType::Date => Ok(LogicalType::Date32), + SQLDataType::Date => Ok(DataType::Date32.into()), SQLDataType::Time(None, tz_info) => { if matches!(tz_info, TimezoneInfo::None) || matches!(tz_info, TimezoneInfo::WithoutTimeZone) { - Ok(LogicalType::Time64(TimeUnit::Nanosecond)) + Ok(DataType::Time64(TimeUnit::Nanosecond).into()) } else { // We dont support TIMETZ and TIME WITH TIME ZONE for now not_impl_err!( @@ -461,8 +461,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; make_decimal_type(precision, scale) } - SQLDataType::Bytea => Ok(LogicalType::Binary), - SQLDataType::Interval => Ok(LogicalType::Interval(IntervalUnit::MonthDayNano)), + SQLDataType::Bytea => Ok(DataType::Binary.into()), + SQLDataType::Interval => Ok(DataType::Interval(IntervalUnit::MonthDayNano).into()), SQLDataType::Struct(fields) => { let fields = fields .iter() @@ -480,7 +480,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ))) }) .collect::>>()?; - Ok(LogicalType::Struct(LogicalFields::from(fields))) + Ok(TypeRelation::new_struct(LogicalFields::from(fields))) } // Explicitly list all other types so that if sqlparser // adds/changes the `SQLDataType` the compiler will tell us on upgrade diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index bb2f32853245..d501588c68af 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -19,7 +19,7 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::path::Path; use std::str::FromStr; use std::sync::Arc; - +use arrow_schema::SchemaRef; use crate::parser::{ CopyToSource, CopyToStatement, CreateExternalTable, DFParser, ExplainStatement, LexOrdering, Statement as DFStatement, @@ -60,7 +60,7 @@ use sqlparser::ast::{ }; use sqlparser::parser::ParserError::ParserError; use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::logical_type::schema::LogicalSchema; fn ident_to_string(ident: &Ident) -> String { @@ -455,7 +455,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { statement, } => { // Convert parser data types to DataFusion data types - let data_types: Vec = data_types + let data_types: Vec = data_types .into_iter() .map(|t| self.convert_data_type(&t)) .collect::>()?; @@ -856,7 +856,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let output_schema = DFSchema::try_from(LogicalPlan::describe_schema()).unwrap(); Ok(LogicalPlan::DescribeTable(DescribeTable { - schema, + schema: SchemaRef::new(schema.as_ref().clone().into()), output_schema: Arc::new(output_schema), })) } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 33186312ac0c..19b9f2a61895 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -27,6 +27,7 @@ use arrow_array::types::{ TimestampNanosecondType, TimestampSecondType, }; use arrow_array::{Date32Array, Date64Array, PrimitiveArray}; +use datafusion_common::logical_type::signature::LogicalType; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ self, Expr as AstExpr, Function, FunctionArg, Ident, Interval, TimezoneInfo, @@ -37,7 +38,7 @@ use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, Column, Result, ScalarValue, }; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::{ expr::{Alias, Exists, InList, ScalarFunction, Sort, WindowFunction}, Between, BinaryExpr, Case, Cast, Expr, GroupingSet, Like, Operator, TryCast, @@ -957,8 +958,8 @@ impl Unparser<'_> { } } - fn arrow_dtype_to_ast_dtype(&self, data_type: &LogicalType) -> Result { - match data_type { + fn arrow_dtype_to_ast_dtype(&self, data_type: &TypeRelation) -> Result { + match data_type.logical() { LogicalType::Null => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } @@ -979,8 +980,7 @@ impl Unparser<'_> { LogicalType::Timestamp(_, _) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - LogicalType::Date32 => Ok(ast::DataType::Date), - LogicalType::Date64 => Ok(ast::DataType::Datetime(None)), + LogicalType::Date => Ok(ast::DataType::Date), LogicalType::Time32(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } @@ -996,24 +996,14 @@ impl Unparser<'_> { LogicalType::Binary => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - LogicalType::FixedSizeBinary(_) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - LogicalType::LargeBinary => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } LogicalType::Utf8 => Ok(ast::DataType::Varchar(None)), - LogicalType::LargeUtf8 => Ok(ast::DataType::Text), LogicalType::List(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - LogicalType::FixedSizeList(_, _) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } - LogicalType::LargeList(_) => { + LogicalType::Struct(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - LogicalType::Struct(_) => { + LogicalType::Union(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } LogicalType::Decimal128(precision, scale) @@ -1032,9 +1022,6 @@ impl Unparser<'_> { LogicalType::Map(_, _) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - LogicalType::Extension(_) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") - } } } } @@ -1134,14 +1121,14 @@ mod tests { ( Expr::Cast(Cast { expr: Box::new(col("a")), - data_type: LogicalType::Date64, + data_type: DataType::Date64.into(), }), r#"CAST(a AS DATETIME)"#, ), ( Expr::Cast(Cast { expr: Box::new(col("a")), - data_type: LogicalType::UInt32, + data_type: DataType::UInt32.into(), }), r#"CAST(a AS INTEGER UNSIGNED)"#, ), @@ -1367,27 +1354,27 @@ mod tests { r#"NOT EXISTS (SELECT t.a FROM t WHERE (t.a = 1))"#, ), ( - try_cast(col("a"), LogicalType::Date64), + try_cast(col("a"), DataType::Date64), r#"TRY_CAST(a AS DATETIME)"#, ), ( - try_cast(col("a"), LogicalType::UInt32), + try_cast(col("a"), DataType::UInt32), r#"TRY_CAST(a AS INTEGER UNSIGNED)"#, ), ( - Expr::ScalarVariable(LogicalType::Int8, vec![String::from("@a")]), + Expr::ScalarVariable(DataType::Int8.into(), vec![String::from("@a")]), r#"@a"#, ), ( Expr::ScalarVariable( - LogicalType::Int8, + DataType::Int8.into(), vec![String::from("@root"), String::from("foo")], ), r#"@root.foo"#, ), (col("x").eq(placeholder("$1")), r#"(x = $1)"#), ( - out_ref_col(LogicalType::Int32, "t.a").gt(lit(1)), + out_ref_col(DataType::Int32, "t.a").gt(lit(1)), r#"(t.a > 1)"#, ), ( @@ -1462,7 +1449,7 @@ mod tests { ( Expr::Cast(Cast { expr: Box::new(col("a")), - data_type: LogicalType::Decimal128(10, -2), + data_type: DataType::Decimal128(10, -2).into(), }), r#"CAST(a AS DECIMAL(12,0))"#, ), diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index fbe8cd099f2a..3cd6ff0a7248 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -20,8 +20,9 @@ use std::collections::HashMap; use arrow_schema::{ - DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, + DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE }; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ exec_err, internal_err, plan_err, Column, DataFusionError, Result, ScalarValue, @@ -31,7 +32,7 @@ use datafusion_expr::expr::{Alias, GroupingSet, Unnest, WindowFunction}; use datafusion_expr::utils::{expr_as_column_expr, find_column_exprs}; use datafusion_expr::{expr_vec_fmt, Expr, ExprSchemable, LogicalPlan}; use sqlparser::ast::Ident; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::{TypeRelation, ExtensionType}; /// Make a best-effort attempt at resolving all columns in the expression tree pub(crate) fn resolve_columns(expr: &Expr, plan: &LogicalPlan) -> Result { @@ -227,7 +228,7 @@ pub fn window_expr_common_partition_keys(window_exprs: &[Expr]) -> Result<&[Expr pub(crate) fn make_decimal_type( precision: Option, scale: Option, -) -> Result { +) -> Result { // postgres like behavior let (precision, scale) = match (precision, scale) { (Some(p), Some(s)) => (p as u8, s as i8), @@ -248,9 +249,9 @@ pub(crate) fn make_decimal_type( } else if precision > DECIMAL128_MAX_PRECISION && precision <= DECIMAL256_MAX_PRECISION { - Ok(LogicalType::Decimal256(precision, scale)) + Ok(DataType::Decimal256(precision, scale).into()) } else { - Ok(LogicalType::Decimal128(precision, scale)) + Ok(DataType::Decimal128(precision, scale).into()) } } @@ -317,7 +318,7 @@ pub(crate) fn recursive_transform_unnest( } = original_expr.transform_up(|expr: Expr| { if let Expr::Unnest(Unnest { expr: ref arg }) = expr { let (data_type, _) = arg.data_type_and_nullable(input.schema())?; - if let LogicalType::Struct(_) = data_type { + if let LogicalType::Struct(_) = data_type.logical() { return internal_err!("unnest on struct can ony be applied at the root level of select expression"); } let transformed_exprs = transform(&expr, arg)?; diff --git a/datafusion/sql/tests/common/mod.rs b/datafusion/sql/tests/common/mod.rs index d7124d5e0180..9c002a339338 100644 --- a/datafusion/sql/tests/common/mod.rs +++ b/datafusion/sql/tests/common/mod.rs @@ -25,7 +25,8 @@ use arrow_schema::*; use datafusion_common::config::ConfigOptions; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{plan_err, GetExt, Result, TableReference}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::TypeRelation; use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource, WindowUDF}; use datafusion_sql::planner::ContextProvider; @@ -203,7 +204,7 @@ impl ContextProvider for MockContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _: &[String]) -> Option { + fn get_variable_type(&self, _: &[String]) -> Option { unimplemented!() } @@ -258,7 +259,7 @@ impl TableSource for EmptyTable { self } - fn schema(&self) -> SchemaRef { - self.table_schema.clone() + fn schema(&self) -> LogicalSchemaRef { + LogicalSchemaRef::new(self.table_schema.clone().into()) } } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index d89419c9edcb..5ac9d9241071 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -43,7 +43,7 @@ use datafusion_functions_aggregate::{ }; use rstest::rstest; use sqlparser::dialect::{Dialect, GenericDialect, HiveDialect, MySqlDialect}; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::TypeRelation; mod cases; mod common; @@ -3663,8 +3663,8 @@ fn test_prepare_statement_should_infer_types() { let plan = logical_plan(sql).unwrap(); let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(LogicalType::Int32)), - ("$2".to_string(), Some(LogicalType::Int64)), + ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), + ("$2".to_string(), Some(TypeRelation::from(DataType::Int64))), ]); assert_eq!(actual_types, expected_types); } @@ -3677,7 +3677,7 @@ fn test_non_prepare_statement_should_infer_types() { let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ // constant 1 is inferred to be int64 - ("$1".to_string(), Some(LogicalType::Int64)), + ("$1".to_string(), Some(TypeRelation::from(DataType::Int64))), ]); assert_eq!(actual_types, expected_types); } @@ -3852,7 +3852,7 @@ Projection: person.id, orders.order_id let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(LogicalType::Int32))]); + let expected_types = HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3884,7 +3884,7 @@ Projection: person.id, person.age let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(LogicalType::Int32))]); + let expected_types = HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3916,8 +3916,8 @@ Projection: person.id, person.age let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(LogicalType::Int32)), - ("$2".to_string(), Some(LogicalType::Int32)), + ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), + ("$2".to_string(), Some(TypeRelation::from(DataType::Int32))), ]); assert_eq!(actual_types, expected_types); @@ -3955,7 +3955,7 @@ Projection: person.id, person.age let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(LogicalType::UInt32))]); + let expected_types = HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::UInt32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3993,8 +3993,8 @@ Dml: op=[Update] table=[person] let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(LogicalType::Int32)), - ("$2".to_string(), Some(LogicalType::UInt32)), + ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), + ("$2".to_string(), Some(TypeRelation::from(DataType::UInt32))), ]); assert_eq!(actual_types, expected_types); @@ -4028,9 +4028,9 @@ fn test_prepare_statement_insert_infer() { let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(LogicalType::UInt32)), - ("$2".to_string(), Some(LogicalType::Utf8)), - ("$3".to_string(), Some(LogicalType::Utf8)), + ("$1".to_string(), Some(TypeRelation::from(DataType::UInt32))), + ("$2".to_string(), Some(TypeRelation::from(DataType::Utf8))), + ("$3".to_string(), Some(TypeRelation::from(DataType::Utf8))), ]); assert_eq!(actual_types, expected_types); diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 3dfcb34d1a3e..f9cca1ee06b2 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -19,7 +19,8 @@ use arrow::util::display::ArrayFormatter; use arrow::{array, array::ArrayRef, datatypes::DataType, record_batch::RecordBatch}; use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::LogicalType; +use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::DataFusionError; use std::path::PathBuf; use std::sync::OnceLock; @@ -248,7 +249,7 @@ pub(crate) fn convert_schema_to_types(columns: &LogicalFields) -> Vec DFColumnType::Boolean, LogicalType::Int8 | LogicalType::Int16 @@ -263,9 +264,8 @@ pub(crate) fn convert_schema_to_types(columns: &LogicalFields) -> Vec DFColumnType::Float, - LogicalType::Utf8 | LogicalType::LargeUtf8 => DFColumnType::Text, - LogicalType::Date32 - | LogicalType::Date64 + LogicalType::Utf8 => DFColumnType::Text, + LogicalType::Date | LogicalType::Time32(_) | LogicalType::Time64(_) => DFColumnType::DateTime, LogicalType::Timestamp(_, _) => DFColumnType::Timestamp, diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index dd27727e3ad5..ba834f225deb 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -25,7 +25,7 @@ use arrow::array::{ ArrayRef, BinaryArray, Float64Array, Int32Array, LargeBinaryArray, LargeStringArray, StringArray, TimestampNanosecondArray, }; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arrow::record_batch::RecordBatch; use datafusion::execution::context::SessionState; use datafusion::logical_expr::{create_udf, ColumnarValue, Expr, ScalarUDF, Volatility}; @@ -42,6 +42,7 @@ use datafusion_common::DataFusionError; use async_trait::async_trait; use log::info; use tempfile::TempDir; +use datafusion_common::logical_type::schema::LogicalSchemaRef; /// Context for running tests pub struct TestContext { @@ -215,7 +216,7 @@ pub async fn register_temp_table(ctx: &SessionContext) { self.0 } - fn schema(&self) -> SchemaRef { + fn schema(&self) -> LogicalSchemaRef { unimplemented!() } diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 56f659f1d870..079d9d950536 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -17,13 +17,11 @@ use async_recursion::async_recursion; use datafusion::arrow::datatypes::{ - Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, + DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit }; -use datafusion::common::logical_type::extension::ExtensionType; -use datafusion::common::logical_type::field::{LogicalField, LogicalFieldRef}; -use datafusion::common::logical_type::fields::LogicalFields; +use datafusion::common::logical_type::field::LogicalField; use datafusion::common::logical_type::schema::LogicalSchema; -use datafusion::common::logical_type::LogicalType; +use datafusion::common::logical_type::{TypeRelation, ExtensionType}; use datafusion::common::{ not_impl_err, substrait_datafusion_err, substrait_err, DFSchema, DFSchemaRef, }; @@ -354,12 +352,12 @@ fn make_renamed_schema( dfs_names: &Vec, ) -> Result { fn rename_inner_fields( - dtype: &LogicalType, + dtype: &DataType, dfs_names: &Vec, name_idx: &mut usize, - ) -> Result { + ) -> Result { match dtype { - LogicalType::Struct(fields) => { + DataType::Struct(fields) => { let fields = fields .iter() .map(|f| { @@ -369,16 +367,16 @@ fn make_renamed_schema( )) }) .collect::>()?; - Ok(LogicalType::Struct(fields)) + Ok(DataType::Struct(fields)) } - LogicalType::List(inner) => Ok(LogicalType::List(LogicalFieldRef::new( + DataType::List(inner) => Ok(DataType::List(FieldRef::new( (**inner).to_owned().with_data_type(rename_inner_fields( inner.data_type(), dfs_names, name_idx, )?), ))), - LogicalType::LargeList(inner) => Ok(LogicalType::LargeList(LogicalFieldRef::new( + DataType::LargeList(inner) => Ok(DataType::LargeList(FieldRef::new( (**inner).to_owned().with_data_type(rename_inner_fields( inner.data_type(), dfs_names, @@ -401,10 +399,10 @@ fn make_renamed_schema( .to_owned() .with_name(name) .with_data_type(rename_inner_fields( - f.data_type(), + f.data_type().physical(), dfs_names, &mut name_idx, - )?), + )?.into()), )) }) .collect::>>()? @@ -1353,7 +1351,7 @@ pub async fn from_substrait_rex( } } -pub(crate) fn from_substrait_type_without_names(dt: &Type) -> Result { +pub(crate) fn from_substrait_type_without_names(dt: &Type) -> Result { from_substrait_type(dt, &[], &mut 0) } @@ -1361,77 +1359,77 @@ fn from_substrait_type( dt: &Type, dfs_names: &[String], name_idx: &mut usize, -) -> Result { +) -> Result { match &dt.kind { Some(s_kind) => match s_kind { - r#type::Kind::Bool(_) => Ok(LogicalType::Boolean), + r#type::Kind::Bool(_) => Ok(DataType::Boolean.into()), r#type::Kind::I8(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int8), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt8), + DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int8.into()), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt8.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::I16(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int16), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt16), + DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int16.into()), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt16.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::I32(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int32), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt32), + DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int32.into()), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt32.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::I64(integer) => match integer.type_variation_reference { - DEFAULT_TYPE_VARIATION_REF => Ok(LogicalType::Int64), - UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(LogicalType::UInt64), + DEFAULT_TYPE_VARIATION_REF => Ok(DataType::Int64.into()), + UNSIGNED_INTEGER_TYPE_VARIATION_REF => Ok(DataType::UInt64.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, - r#type::Kind::Fp32(_) => Ok(LogicalType::Float32), - r#type::Kind::Fp64(_) => Ok(LogicalType::Float64), + r#type::Kind::Fp32(_) => Ok(DataType::Float32.into()), + r#type::Kind::Fp64(_) => Ok(DataType::Float64.into()), r#type::Kind::Timestamp(ts) => match ts.type_variation_reference { TIMESTAMP_SECOND_TYPE_VARIATION_REF => { - Ok(LogicalType::Timestamp(TimeUnit::Second, None)) + Ok(DataType::Timestamp(TimeUnit::Second, None).into()) } TIMESTAMP_MILLI_TYPE_VARIATION_REF => { - Ok(LogicalType::Timestamp(TimeUnit::Millisecond, None)) + Ok(DataType::Timestamp(TimeUnit::Millisecond, None).into()) } TIMESTAMP_MICRO_TYPE_VARIATION_REF => { - Ok(LogicalType::Timestamp(TimeUnit::Microsecond, None)) + Ok(DataType::Timestamp(TimeUnit::Microsecond, None).into()) } TIMESTAMP_NANO_TYPE_VARIATION_REF => { - Ok(LogicalType::Timestamp(TimeUnit::Nanosecond, None)) + Ok(DataType::Timestamp(TimeUnit::Nanosecond, None).into()) } v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::Date(date) => match date.type_variation_reference { - DATE_32_TYPE_VARIATION_REF => Ok(LogicalType::Date32), - DATE_64_TYPE_VARIATION_REF => Ok(LogicalType::Date64), + DATE_32_TYPE_VARIATION_REF => Ok(DataType::Date32.into()), + DATE_64_TYPE_VARIATION_REF => Ok(DataType::Date64.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::Binary(binary) => match binary.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::Binary), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::LargeBinary), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Binary.into()), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeBinary.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), }, r#type::Kind::FixedBinary(fixed) => { - Ok(LogicalType::FixedSizeBinary(fixed.length)) + Ok(DataType::FixedSizeBinary(fixed.length).into()) } r#type::Kind::String(string) => match string.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::Utf8), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::LargeUtf8), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Utf8.into()), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeUtf8.into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), @@ -1440,15 +1438,15 @@ fn from_substrait_type( let inner_type = list.r#type.as_ref().ok_or_else(|| { substrait_datafusion_err!("List type must have inner type") })?; - let field = Arc::new(LogicalField::new_list_field( - from_substrait_type(inner_type, dfs_names, name_idx)?, + let field = Arc::new(Field::new_list_field( + from_substrait_type(inner_type, dfs_names, name_idx)?.physical().clone(), // We ignore Substrait's nullability here to match to_substrait_literal // which always creates nullable lists true, )); match list.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::List(field)), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(LogicalType::LargeList(field)), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::List(field).into()), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeList(field).into()), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" )?, @@ -1461,23 +1459,23 @@ fn from_substrait_type( let value_type = map.value.as_ref().ok_or_else(|| { substrait_datafusion_err!("Map type must have value type") })?; - let key_field = Arc::new(LogicalField::new( + let key_field = Arc::new(Field::new( "key", - from_substrait_type(key_type, dfs_names, name_idx)?, + from_substrait_type(key_type, dfs_names, name_idx)?.physical().clone(), false, )); - let value_field = Arc::new(LogicalField::new( + let value_field = Arc::new(Field::new( "value", - from_substrait_type(value_type, dfs_names, name_idx)?, + from_substrait_type(value_type, dfs_names, name_idx)?.physical().clone(), true, )); match map.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => { - Ok(LogicalType::Map(Arc::new(LogicalField::new_struct( + Ok(DataType::Map(Arc::new(Field::new_struct( "entries", [key_field, value_field], false, // The inner map field is always non-nullable (Arrow #1697), - )), false)) + )), false).into()) }, v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" @@ -1486,10 +1484,10 @@ fn from_substrait_type( } r#type::Kind::Decimal(d) => match d.type_variation_reference { DECIMAL_128_TYPE_VARIATION_REF => { - Ok(LogicalType::Decimal128(d.precision as u8, d.scale as i8)) + Ok(DataType::Decimal128(d.precision as u8, d.scale as i8).into()) } DECIMAL_256_TYPE_VARIATION_REF => { - Ok(LogicalType::Decimal256(d.precision as u8, d.scale as i8)) + Ok(DataType::Decimal256(d.precision as u8, d.scale as i8).into()) } v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" @@ -1498,13 +1496,13 @@ fn from_substrait_type( r#type::Kind::UserDefined(u) => { match u.type_reference { INTERVAL_YEAR_MONTH_TYPE_REF => { - Ok(LogicalType::Interval(IntervalUnit::YearMonth)) + Ok(DataType::Interval(IntervalUnit::YearMonth).into()) } INTERVAL_DAY_TIME_TYPE_REF => { - Ok(LogicalType::Interval(IntervalUnit::DayTime)) + Ok(DataType::Interval(IntervalUnit::DayTime).into()) } INTERVAL_MONTH_DAY_NANO_TYPE_REF => { - Ok(LogicalType::Interval(IntervalUnit::MonthDayNano)) + Ok(DataType::Interval(IntervalUnit::MonthDayNano).into()) } _ => not_impl_err!( "Unsupported Substrait user defined type with ref {} and variation {}", @@ -1513,11 +1511,11 @@ fn from_substrait_type( ), } }, - r#type::Kind::Struct(s) => Ok(LogicalType::Struct(from_substrait_struct_type( + r#type::Kind::Struct(s) => Ok(DataType::Struct(from_substrait_struct_type( s, dfs_names, name_idx, - )?)), - r#type::Kind::Varchar(_) => Ok(LogicalType::Utf8), - r#type::Kind::FixedChar(_) => Ok(LogicalType::Utf8), + )?).into()), + r#type::Kind::Varchar(_) => Ok(DataType::Utf8.into()), + r#type::Kind::FixedChar(_) => Ok(DataType::Utf8.into()), _ => not_impl_err!("Unsupported Substrait type: {s_kind:?}"), }, _ => not_impl_err!("`None` Substrait kind is not supported"), @@ -1528,12 +1526,12 @@ fn from_substrait_struct_type( s: &r#type::Struct, dfs_names: &[String], name_idx: &mut usize, -) -> Result { +) -> Result { let mut fields = vec![]; for (i, f) in s.types.iter().enumerate() { - let field = LogicalField::new( + let field = Field::new( next_struct_field_name(i, dfs_names, name_idx)?, - from_substrait_type(f, dfs_names, name_idx)?, + from_substrait_type(f, dfs_names, name_idx)?.physical().clone(), is_substrait_type_nullable(f)?, ); fields.push(field); @@ -1783,10 +1781,10 @@ fn from_substrait_literal( )?; match lit.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => { - ScalarValue::List(ScalarValue::new_list_nullable(&[], &element_type.physical_type())) + ScalarValue::List(ScalarValue::new_list_nullable(&[], &element_type.physical())) } LARGE_CONTAINER_TYPE_VARIATION_REF => ScalarValue::LargeList( - ScalarValue::new_large_list(&[], &element_type.physical_type()), + ScalarValue::new_large_list(&[], &element_type.physical()), ), others => { return substrait_err!("Unknown type variation reference {others}"); diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 07c40f9a9fbd..1a728dc1efbc 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -21,7 +21,7 @@ use std::ops::Deref; use std::sync::Arc; use arrow_buffer::ToByteSlice; -use datafusion::arrow::datatypes::IntervalUnit; +use datafusion::arrow::datatypes::{DataType, IntervalUnit}; use datafusion::logical_expr::{ CrossJoin, Distinct, Like, Partitioning, WindowFrameUnits, }; @@ -45,7 +45,7 @@ use crate::variation_const::{ TIMESTAMP_SECOND_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; -use datafusion::common::logical_type::LogicalType; +use datafusion::common::logical_type::ExtensionType; use datafusion::common::{ exec_err, internal_err, not_impl_err, plan_err, substrait_datafusion_err, }; @@ -586,9 +586,9 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { // Substrait wants a list of all field names, including nested fields from structs, // also from within e.g. lists and maps. However, it does not want the list and map field names // themselves - only proper structs fields are considered to have useful names. - fn names_dfs(dtype: &LogicalType) -> Result> { + fn names_dfs(dtype: &DataType) -> Result> { match dtype { - LogicalType::Struct(fields) => { + DataType::Struct(fields) => { let mut names = Vec::new(); for field in fields { names.push(field.name().to_string()); @@ -596,10 +596,10 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { } Ok(names) } - LogicalType::List(l) => names_dfs(l.data_type()), - LogicalType::LargeList(l) => names_dfs(l.data_type()), - LogicalType::Map(m, _) => match m.data_type() { - LogicalType::Struct(key_and_value) if key_and_value.len() == 2 => { + DataType::List(l) => names_dfs(l.data_type()), + DataType::LargeList(l) => names_dfs(l.data_type()), + DataType::Map(m, _) => match m.data_type() { + DataType::Struct(key_and_value) if key_and_value.len() == 2 => { let key_names = names_dfs(key_and_value.first().unwrap().data_type())?; let value_names = @@ -617,7 +617,7 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { .iter() .map(|f| { let mut names = vec![f.name().to_string()]; - names.extend(names_dfs(f.data_type())?); + names.extend(names_dfs(f.data_type().physical())?); Ok(names) }) .flatten_ok() @@ -627,7 +627,7 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { types: schema .fields() .iter() - .map(|f| to_substrait_type(f.data_type(), f.is_nullable())) + .map(|f| to_substrait_type(f.data_type().physical(), f.is_nullable())) .collect::>()?, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability: r#type::Nullability::Unspecified as i32, @@ -1226,7 +1226,7 @@ pub fn to_substrait_rex( Ok(Expression { rex_type: Some(RexType::Cast(Box::new( substrait::proto::expression::Cast { - r#type: Some(to_substrait_type(data_type, true)?), + r#type: Some(to_substrait_type(data_type.physical(), true)?), input: Some(Box::new(to_substrait_rex( ctx, expr, @@ -1433,83 +1433,83 @@ pub fn to_substrait_rex( } } -fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result { +fn to_substrait_type(dt: &DataType, nullable: bool) -> Result { let nullability = if nullable { r#type::Nullability::Nullable as i32 } else { r#type::Nullability::Required as i32 }; match dt { - LogicalType::Null => internal_err!("Null cast is not valid"), - LogicalType::Boolean => Ok(substrait::proto::Type { + DataType::Null => internal_err!("Null cast is not valid"), + DataType::Boolean => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Bool(r#type::Boolean { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Int8 => Ok(substrait::proto::Type { + DataType::Int8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I8(r#type::I8 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::UInt8 => Ok(substrait::proto::Type { + DataType::UInt8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I8(r#type::I8 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Int16 => Ok(substrait::proto::Type { + DataType::Int16 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I16(r#type::I16 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::UInt16 => Ok(substrait::proto::Type { + DataType::UInt16 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I16(r#type::I16 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Int32 => Ok(substrait::proto::Type { + DataType::Int32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I32(r#type::I32 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::UInt32 => Ok(substrait::proto::Type { + DataType::UInt32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I32(r#type::I32 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Int64 => Ok(substrait::proto::Type { + DataType::Int64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I64(r#type::I64 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::UInt64 => Ok(substrait::proto::Type { + DataType::UInt64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I64(r#type::I64 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), // Float16 is not supported in Substrait - LogicalType::Float32 => Ok(substrait::proto::Type { + DataType::Float32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Fp32(r#type::Fp32 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Float64 => Ok(substrait::proto::Type { + DataType::Float64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Fp64(r#type::Fp64 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), // Timezone is ignored. - LogicalType::Timestamp(unit, _) => { + DataType::Timestamp(unit, _) => { let type_variation_reference = match unit { TimeUnit::Second => TIMESTAMP_SECOND_TYPE_VARIATION_REF, TimeUnit::Millisecond => TIMESTAMP_MILLI_TYPE_VARIATION_REF, @@ -1523,19 +1523,19 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result Ok(substrait::proto::Type { + DataType::Date32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Date(r#type::Date { type_variation_reference: DATE_32_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Date64 => Ok(substrait::proto::Type { + DataType::Date64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Date(r#type::Date { type_variation_reference: DATE_64_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Interval(interval_unit) => { + DataType::Interval(interval_unit) => { // define two type parameters for convenience let i32_param = Parameter { parameter: Some(parameter::Parameter::DataType(substrait::proto::Type { @@ -1578,38 +1578,38 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result Ok(substrait::proto::Type { + DataType::Binary => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Binary(r#type::Binary { type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::FixedSizeBinary(length) => Ok(substrait::proto::Type { + DataType::FixedSizeBinary(length) => Ok(substrait::proto::Type { kind: Some(r#type::Kind::FixedBinary(r#type::FixedBinary { length: *length, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::LargeBinary => Ok(substrait::proto::Type { + DataType::LargeBinary => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Binary(r#type::Binary { type_variation_reference: LARGE_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::Utf8 => Ok(substrait::proto::Type { + DataType::Utf8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::String(r#type::String { type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::LargeUtf8 => Ok(substrait::proto::Type { + DataType::LargeUtf8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::String(r#type::String { type_variation_reference: LARGE_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), - LogicalType::List(inner) => { + DataType::List(inner) => { let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { @@ -1619,7 +1619,7 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result { + DataType::LargeList(inner) => { let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { @@ -1629,8 +1629,8 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result match inner.data_type() { - LogicalType::Struct(key_and_value) if key_and_value.len() == 2 => { + DataType::Map(inner, _) => match inner.data_type() { + DataType::Struct(key_and_value) if key_and_value.len() == 2 => { let key_type = to_substrait_type( key_and_value[0].data_type(), key_and_value[0].is_nullable(), @@ -1650,7 +1650,7 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result plan_err!("Map fields must contain a Struct with exactly 2 fields"), }, - LogicalType::Struct(fields) => { + DataType::Struct(fields) => { let field_types = fields .iter() .map(|field| to_substrait_type(field.data_type(), field.is_nullable())) @@ -1663,7 +1663,7 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result Ok(substrait::proto::Type { + DataType::Decimal128(p, s) => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Decimal(r#type::Decimal { type_variation_reference: DECIMAL_128_TYPE_VARIATION_REF, nullability, @@ -1671,7 +1671,7 @@ fn to_substrait_type(dt: &LogicalType, nullable: bool) -> Result Ok(substrait::proto::Type { + DataType::Decimal256(p, s) => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Decimal(r#type::Decimal { type_variation_reference: DECIMAL_256_TYPE_VARIATION_REF, nullability, @@ -1897,7 +1897,7 @@ fn to_substrait_literal(value: &ScalarValue) -> Result { nullable: true, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, literal_type: Some(LiteralType::Null(to_substrait_type( - &value.data_type().into(), + &value.data_type(), true, )?)), }); @@ -2097,7 +2097,7 @@ fn convert_array_to_literal_list( .collect::>>()?; if values.is_empty() { - let et = match to_substrait_type(&array.data_type().to_owned().into(), array.is_nullable())? { + let et = match to_substrait_type(&array.data_type(), array.is_nullable())? { substrait::proto::Type { kind: Some(r#type::Kind::List(lt)), } => lt.as_ref().to_owned(), @@ -2379,7 +2379,7 @@ mod test { let lt = dt.into(); let substrait = to_substrait_type(<, true)?; let roundtrip_dt = from_substrait_type_without_names(&substrait)?; - assert_eq!(lt, roundtrip_dt); + assert_eq!(<, roundtrip_dt.physical()); Ok(()) } } From d65e857b4eb47ac5980ef6f3812d4ed5d2874c4e Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Tue, 2 Jul 2024 15:49:23 +0200 Subject: [PATCH 03/10] Add example with TODOs --- datafusion-examples/examples/logical_type.rs | 111 ++++++++++++++++++- 1 file changed, 108 insertions(+), 3 deletions(-) diff --git a/datafusion-examples/examples/logical_type.rs b/datafusion-examples/examples/logical_type.rs index 8a55e8035009..859f208f4621 100644 --- a/datafusion-examples/examples/logical_type.rs +++ b/datafusion-examples/examples/logical_type.rs @@ -1,3 +1,108 @@ -fn main() { - -} \ No newline at end of file +use datafusion::datasource::TableProvider; +use datafusion::execution::context::SessionState; +use datafusion::physical_plan::ExecutionPlan; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion::error::Result; +use datafusion_expr::{Expr, TableType}; +use std::any::Any; +use std::sync::Arc; +use arrow::util::pretty::pretty_format_batches; +use arrow_schema::{DataType, Field, TimeUnit}; +use datafusion::prelude::SessionContext; +use datafusion_common::logical_type::{ExtensionType}; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::signature::LogicalType; + +#[tokio::main] +async fn main() -> Result<()> { + let ctx = SessionContext::new(); + ctx.register_table("example", Arc::new(ExampleTableSource::default()))?; + + let df = ctx.sql("SELECT * FROM example").await?; + let records = df.collect().await?; + + println!("{}", pretty_format_batches(&records)?); + + Ok(()) +} + +#[derive(Debug)] +struct CustomMagicalType { + logical: LogicalType, + physical: DataType +} + +impl Default for CustomMagicalType { + fn default() -> Self { + Self { + logical: LogicalType::Utf8, + physical: DataType::new_list(DataType::UInt8, false), + } + } +} + +impl ExtensionType for CustomMagicalType { + fn logical(&self) -> &LogicalType { + &self.logical + } + + fn physical(&self) -> &DataType { + &self.physical + } + + // TODO: materialisation methods? +} + +#[derive(Default)] +struct ExampleTableSource {} + +#[async_trait::async_trait] +impl TableProvider for ExampleTableSource { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> LogicalSchemaRef { + // TODO: ugly? + let custom_magical_type: Arc = Arc::new(CustomMagicalType::default()); + + // This schema will be equivalent to: + // a -> Timestamp(Microsecond, None) + // b -> Utf8 + // c -> Int64 + Arc::new(LogicalSchema::new(vec![ + LogicalField::new( + "a", + DataType::RunEndEncoded( + Arc::new(Field::new("run_ends", DataType::Int64, false)), + Arc::new(Field::new("values", DataType::Timestamp(TimeUnit::Microsecond, None), false)) + ), + false + ), + LogicalField::new( + "b", + custom_magical_type, + false + ), + LogicalField::new( + "c", + DataType::Int64, + true, + ) + ])) + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + _state: &SessionState, + _projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + todo!() + } +} From ce68d1b24d338ee813074f0b8cec8d3215fbf3f5 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Mon, 8 Jul 2024 10:59:11 +0200 Subject: [PATCH 04/10] Format --- .../examples/advanced_parquet_index.rs | 5 +- .../examples/custom_datasource.rs | 2 +- datafusion-examples/examples/expr_api.rs | 6 +- datafusion-examples/examples/logical_type.rs | 39 +++--- datafusion-examples/examples/parquet_index.rs | 2 +- datafusion-examples/examples/rewrite_expr.rs | 2 +- datafusion-examples/examples/simple_udtf.rs | 2 +- datafusion/common/src/column.rs | 2 +- datafusion/common/src/dfschema.rs | 47 ++++--- datafusion/common/src/lib.rs | 2 +- datafusion/common/src/logical_type/field.rs | 15 +- datafusion/common/src/logical_type/fields.rs | 19 ++- datafusion/common/src/logical_type/mod.rs | 8 +- datafusion/common/src/logical_type/schema.rs | 10 +- .../common/src/logical_type/signature.rs | 8 +- datafusion/common/src/param_value.rs | 2 +- datafusion/core/src/dataframe/mod.rs | 12 +- .../src/datasource/default_table_source.rs | 2 +- .../core/src/datasource/listing/helpers.rs | 2 +- .../core/src/datasource/listing/table.rs | 33 +++-- datafusion/core/src/datasource/memory.rs | 30 ++-- .../physical_plan/file_scan_config.rs | 2 +- .../physical_plan/parquet/row_filter.rs | 2 - datafusion/core/src/datasource/provider.rs | 2 +- datafusion/core/src/datasource/stream.rs | 2 +- datafusion/core/src/datasource/streaming.rs | 8 +- datafusion/core/src/datasource/view.rs | 2 +- .../core/src/execution/session_state.rs | 2 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/core/src/test_util/mod.rs | 2 +- datafusion/core/src/test_util/parquet.rs | 5 +- .../core/tests/custom_sources_cases/mod.rs | 2 +- datafusion/core/tests/dataframe/mod.rs | 2 +- .../core/tests/expr_api/parse_sql_expr.rs | 2 +- .../core/tests/expr_api/simplification.rs | 6 +- datafusion/core/tests/parquet/page_pruning.rs | 6 +- datafusion/core/tests/sql/create_drop.rs | 17 ++- .../user_defined_scalar_functions.rs | 11 +- .../user_defined_table_functions.rs | 2 +- .../expr/src/conditional_expressions.rs | 2 +- datafusion/expr/src/expr.rs | 14 +- datafusion/expr/src/expr_fn.rs | 2 +- datafusion/expr/src/expr_rewriter/mod.rs | 3 +- datafusion/expr/src/expr_rewriter/order_by.rs | 2 +- datafusion/expr/src/expr_schema.rs | 40 ++++-- datafusion/expr/src/logical_plan/builder.rs | 130 ++++++++++-------- datafusion/expr/src/logical_plan/ddl.rs | 2 +- datafusion/expr/src/logical_plan/dml.rs | 8 +- datafusion/expr/src/logical_plan/extension.rs | 6 +- datafusion/expr/src/logical_plan/plan.rs | 22 +-- datafusion/expr/src/planner.rs | 4 +- datafusion/expr/src/simplify.rs | 8 +- datafusion/expr/src/table_source.rs | 2 +- datafusion/expr/src/type_coercion/mod.rs | 12 +- datafusion/expr/src/utils.rs | 5 +- datafusion/expr/src/var_provider.rs | 2 +- datafusion/functions-array/src/planner.rs | 2 +- datafusion/functions/src/core/getfield.rs | 2 +- datafusion/functions/src/math/log.rs | 4 +- datafusion/functions/src/math/power.rs | 2 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 44 ++++-- .../optimizer/src/eliminate_one_union.rs | 4 +- .../optimizer/src/optimize_projections/mod.rs | 2 +- .../optimizer/src/propagate_empty_relation.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 18 +-- .../simplify_expressions/expr_simplifier.rs | 44 ++++-- .../simplify_expressions/simplify_exprs.rs | 5 +- .../src/unwrap_cast_in_comparison.rs | 24 ++-- .../optimizer/tests/optimizer_integration.rs | 2 +- .../physical-expr/src/expressions/in_list.rs | 5 +- datafusion/physical-expr/src/planner.rs | 9 +- .../physical-expr/src/utils/guarantee.rs | 4 +- datafusion/proto-common/src/from_proto/mod.rs | 33 ++--- datafusion/proto-common/src/to_proto/mod.rs | 7 +- .../proto/src/logical_plan/from_proto.rs | 6 +- datafusion/proto/src/logical_plan/mod.rs | 8 +- .../tests/cases/roundtrip_logical_plan.rs | 20 ++- datafusion/sql/examples/sql.rs | 2 +- datafusion/sql/src/expr/function.rs | 5 +- datafusion/sql/src/expr/identifier.rs | 2 +- datafusion/sql/src/expr/mod.rs | 23 +++- datafusion/sql/src/expr/value.rs | 2 +- datafusion/sql/src/planner.rs | 13 +- datafusion/sql/src/statement.rs | 16 +-- datafusion/sql/src/unparser/expr.rs | 7 +- datafusion/sql/src/utils.rs | 4 +- datafusion/sql/tests/cases/plan_to_sql.rs | 2 +- datafusion/sql/tests/common/mod.rs | 2 +- datafusion/sql/tests/sql_integration.rs | 15 +- .../engines/datafusion_engine/normalize.rs | 6 +- datafusion/sqllogictest/src/test_context.rs | 2 +- .../substrait/src/logical_plan/consumer.rs | 44 +++--- .../substrait/src/logical_plan/producer.rs | 20 +-- 94 files changed, 603 insertions(+), 402 deletions(-) diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index a366ddc03c66..2e50850c3723 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -42,6 +42,7 @@ use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -60,7 +61,6 @@ use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use tempfile::TempDir; use url::Url; -use datafusion_common::logical_type::schema::LogicalSchemaRef; /// This example demonstrates using low level DataFusion APIs to read only /// certain row groups and ranges from parquet files, based on external @@ -301,8 +301,7 @@ impl IndexTableProvider { // analyze the predicate. In a real system, using // `PruningPredicate::prune` would likely be easier to do. let schema = SchemaRef::new(self.schema().as_ref().clone().into()); - let pruning_predicate = - PruningPredicate::try_new(Arc::clone(predicate), schema)?; + let pruning_predicate = PruningPredicate::try_new(Arc::clone(predicate), schema)?; // The PruningPredicate's guarantees must all be satisfied in order for // the predicate to possibly evaluate to true. diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 7b3d7ef23d8a..e5b44cb25f31 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -37,9 +37,9 @@ use datafusion_expr::LogicalPlanBuilder; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; -use tokio::time::timeout; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use tokio::time::timeout; /// This example demonstrates executing a simple query against a custom datasource #[tokio::main] diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 40533961e7ee..526cca87db93 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -158,7 +158,8 @@ fn simplify_demo() -> Result<()> { // However, DataFusion's simplification logic can do this for you // you need to tell DataFusion the type of column "ts": - let schema = LogicalSchema::from(Schema::new(vec![make_ts_field("ts")])).to_dfschema_ref()?; + let schema = + LogicalSchema::from(Schema::new(vec![make_ts_field("ts")])).to_dfschema_ref()?; // And then build a simplifier // the ExecutionProps carries information needed to simplify @@ -213,7 +214,8 @@ fn simplify_demo() -> Result<()> { // String --> Date simplification // `cast('2020-09-01' as date)` --> 18500 assert_eq!( - simplifier.simplify(lit("2020-09-01").cast_to(&DataType::Date32.into(), &schema)?)?, + simplifier + .simplify(lit("2020-09-01").cast_to(&DataType::Date32.into(), &schema)?)?, lit(ScalarValue::Date32(Some(18506))) ); diff --git a/datafusion-examples/examples/logical_type.rs b/datafusion-examples/examples/logical_type.rs index 859f208f4621..809b45d3b30e 100644 --- a/datafusion-examples/examples/logical_type.rs +++ b/datafusion-examples/examples/logical_type.rs @@ -1,17 +1,17 @@ +use arrow::util::pretty::pretty_format_batches; +use arrow_schema::{DataType, Field, TimeUnit}; use datafusion::datasource::TableProvider; +use datafusion::error::Result; use datafusion::execution::context::SessionState; use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; +use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; -use datafusion::error::Result; +use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::ExtensionType; use datafusion_expr::{Expr, TableType}; use std::any::Any; use std::sync::Arc; -use arrow::util::pretty::pretty_format_batches; -use arrow_schema::{DataType, Field, TimeUnit}; -use datafusion::prelude::SessionContext; -use datafusion_common::logical_type::{ExtensionType}; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::signature::LogicalType; #[tokio::main] async fn main() -> Result<()> { @@ -29,7 +29,7 @@ async fn main() -> Result<()> { #[derive(Debug)] struct CustomMagicalType { logical: LogicalType, - physical: DataType + physical: DataType, } impl Default for CustomMagicalType { @@ -64,7 +64,8 @@ impl TableProvider for ExampleTableSource { fn schema(&self) -> LogicalSchemaRef { // TODO: ugly? - let custom_magical_type: Arc = Arc::new(CustomMagicalType::default()); + let custom_magical_type: Arc = + Arc::new(CustomMagicalType::default()); // This schema will be equivalent to: // a -> Timestamp(Microsecond, None) @@ -75,20 +76,16 @@ impl TableProvider for ExampleTableSource { "a", DataType::RunEndEncoded( Arc::new(Field::new("run_ends", DataType::Int64, false)), - Arc::new(Field::new("values", DataType::Timestamp(TimeUnit::Microsecond, None), false)) + Arc::new(Field::new( + "values", + DataType::Timestamp(TimeUnit::Microsecond, None), + false, + )), ), - false + false, ), - LogicalField::new( - "b", - custom_magical_type, - false - ), - LogicalField::new( - "c", - DataType::Int64, - true, - ) + LogicalField::new("b", custom_magical_type, false), + LogicalField::new("c", DataType::Int64, true), ])) } diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 49870095caf4..3e8d2cfc9a71 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -37,6 +37,7 @@ use datafusion::parquet::arrow::{ use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -54,7 +55,6 @@ use std::sync::{ }; use tempfile::TempDir; use url::Url; -use datafusion_common::logical_type::schema::LogicalSchemaRef; /// This example demonstrates building a secondary index over multiple Parquet /// files and using that index during query to skip ("prune") files that do not diff --git a/datafusion-examples/examples/rewrite_expr.rs b/datafusion-examples/examples/rewrite_expr.rs index e9e9014e78f0..59234793d407 100644 --- a/datafusion-examples/examples/rewrite_expr.rs +++ b/datafusion-examples/examples/rewrite_expr.rs @@ -17,6 +17,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::logical_type::TypeRelation; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; @@ -32,7 +33,6 @@ use datafusion_sql::sqlparser::parser::Parser; use datafusion_sql::TableReference; use std::any::Any; use std::sync::Arc; -use datafusion_common::logical_type::schema::LogicalSchemaRef; pub fn main() -> Result<()> { // produce a logical plan using the datafusion-sql crate diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index 8776cc7599d1..df7f708c5650 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -27,6 +27,7 @@ use datafusion::execution::context::{ExecutionProps, SessionState}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::{plan_err, ScalarValue}; use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::{Expr, TableType}; @@ -35,7 +36,6 @@ use std::fs::File; use std::io::Seek; use std::path::Path; use std::sync::Arc; -use datafusion_common::logical_type::schema::LogicalSchemaRef; // To define your own table function, you only need to do the following 3 things: // 1. Implement your own [`TableProvider`] // 2. Implement your own [`TableFunctionImpl`] and return your [`TableProvider`] diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index ac54e87fd617..ca77df4ec95e 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -18,6 +18,7 @@ //! Column use crate::error::_schema_err; +use crate::logical_type::field::{LogicalField, LogicalFieldRef}; use crate::utils::{parse_identifiers_normalized, quote_identifier}; use crate::{DFSchema, DataFusionError, Result, SchemaError, TableReference}; use std::collections::HashSet; @@ -25,7 +26,6 @@ use std::convert::Infallible; use std::fmt; use std::str::FromStr; use std::sync::Arc; -use crate::logical_type::field::{LogicalField, LogicalFieldRef}; /// A named reference to a qualified field in a schema. #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index aa3bee9a70f1..c2c90c32735d 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -29,12 +29,14 @@ use crate::{ SchemaError, TableReference, }; -use arrow::compute::can_cast_types; -use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use crate::logical_type::field::{LogicalField, LogicalFieldRef}; use crate::logical_type::fields::LogicalFields; -use crate::logical_type::{TypeRelation, ExtensionType}; -use crate::logical_type::schema::{LogicalSchema, LogicalSchemaBuilder, LogicalSchemaRef}; +use crate::logical_type::schema::{ + LogicalSchema, LogicalSchemaBuilder, LogicalSchemaRef, +}; +use crate::logical_type::{ExtensionType, TypeRelation}; +use arrow::compute::can_cast_types; +use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; /// A reference-counted reference to a [DFSchema]. pub type DFSchemaRef = Arc; @@ -296,12 +298,8 @@ impl DFSchema { let self_fields: HashSet<(Option<&TableReference>, &LogicalFieldRef)> = self.iter().collect(); - let self_unqualified_names: HashSet<&str> = self - .inner - .fields - .iter() - .map(|field| field.name()) - .collect(); + let self_unqualified_names: HashSet<&str> = + self.inner.fields.iter().map(|field| field.name()).collect(); let mut schema_builder = LogicalSchemaBuilder::from(self.inner.fields.clone()); let mut qualifiers = Vec::new(); @@ -432,7 +430,10 @@ impl DFSchema { } /// Find all fields having the given qualifier - pub fn fields_with_qualified(&self, qualifier: &TableReference) -> Vec<&LogicalField> { + pub fn fields_with_qualified( + &self, + qualifier: &TableReference, + ) -> Vec<&LogicalField> { self.iter() .filter(|(q, _)| q.map(|q| q.eq(qualifier)).unwrap_or(false)) .map(|(_, f)| f.as_ref()) @@ -481,9 +482,7 @@ impl DFSchema { /// Return all `Column`s for the schema pub fn columns(&self) -> Vec { self.iter() - .map(|(qualifier, field)| { - Column::new(qualifier.cloned(), field.name()) - }) + .map(|(qualifier, field)| Column::new(qualifier.cloned(), field.name())) .collect() } @@ -674,7 +673,10 @@ impl DFSchema { self_fields.zip(other_fields).all(|((q1, f1), (q2, f2))| { q1 == q2 && f1.name() == f2.name() - && Self::datatype_is_semantically_equal(&f1.data_type().physical(), &f2.data_type().physical()) + && Self::datatype_is_semantically_equal( + &f1.data_type().physical(), + &f2.data_type().physical(), + ) }) } @@ -775,7 +777,9 @@ impl DFSchema { } /// Iterate over the qualifiers and fields in the DFSchema - pub fn iter(&self) -> impl Iterator, &LogicalFieldRef)> { + pub fn iter( + &self, + ) -> impl Iterator, &LogicalFieldRef)> { self.field_qualifiers .iter() .zip(self.inner.fields.iter()) @@ -1048,7 +1052,8 @@ mod tests { &Schema::new(vec![ Field::new("CapitalColumn", DataType::Boolean, true), Field::new("field.with.period", DataType::Boolean, true), - ]).into(), + ]) + .into(), )?; // lookup with unqualified name "t1.c0" @@ -1095,7 +1100,10 @@ mod tests { Some("t0".into()), Arc::new(Field::new("c0", DataType::Boolean, true).into()), ), - (None, Arc::new(Field::new("c1", DataType::Boolean, true).into())), + ( + None, + Arc::new(Field::new("c1", DataType::Boolean, true).into()), + ), ], HashMap::new(), )?; @@ -1274,7 +1282,8 @@ mod tests { Schema::new(vec![ Field::new("c0", DataType::Boolean, true), Field::new("c1", DataType::Boolean, true), - ]).into() + ]) + .into() } #[test] fn test_dfschema_to_schema_convertion() { diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 7585f1a2c12a..9b32f325b698 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -35,6 +35,7 @@ pub mod file_options; pub mod format; pub mod hash_utils; pub mod instant; +pub mod logical_type; pub mod parsers; pub mod rounding; pub mod scalar; @@ -42,7 +43,6 @@ pub mod stats; pub mod test_util; pub mod tree_node; pub mod utils; -pub mod logical_type; /// Reexport arrow crate pub use arrow; diff --git a/datafusion/common/src/logical_type/field.rs b/datafusion/common/src/logical_type/field.rs index 720370a9dbac..96b66201c2d4 100644 --- a/datafusion/common/src/logical_type/field.rs +++ b/datafusion/common/src/logical_type/field.rs @@ -39,7 +39,7 @@ impl From<&Field> for LogicalField { name: value.name().clone(), data_type: value.data_type().clone().into(), nullable: value.is_nullable(), - metadata: value.metadata().clone() + metadata: value.metadata().clone(), } } } @@ -64,11 +64,8 @@ impl From for LogicalField { impl Into for LogicalField { fn into(self) -> Field { - Field::new( - self.name, - self.data_type.physical().clone(), - self.nullable - ).with_metadata(self.metadata) + Field::new(self.name, self.data_type.physical().clone(), self.nullable) + .with_metadata(self.metadata) } } @@ -100,7 +97,11 @@ impl Hash for LogicalField { } impl LogicalField { - pub fn new(name: impl Into, data_type: impl Into, nullable: bool) -> Self { + pub fn new( + name: impl Into, + data_type: impl Into, + nullable: bool, + ) -> Self { LogicalField { name: name.into(), data_type: data_type.into(), diff --git a/datafusion/common/src/logical_type/fields.rs b/datafusion/common/src/logical_type/fields.rs index 99b9f7904a97..3aa2445aa153 100644 --- a/datafusion/common/src/logical_type/fields.rs +++ b/datafusion/common/src/logical_type/fields.rs @@ -33,7 +33,12 @@ impl std::fmt::Debug for LogicalFields { impl From<&Fields> for LogicalFields { fn from(value: &Fields) -> Self { - Self(value.iter().map(|v| LogicalFieldRef::new(v.into())).collect()) + Self( + value + .iter() + .map(|v| LogicalFieldRef::new(v.into())) + .collect(), + ) } } @@ -48,7 +53,7 @@ impl Into for LogicalFields { Fields::from( self.iter() .map(|f| f.as_ref().clone().into()) - .collect::>() + .collect::>(), ) } } @@ -60,13 +65,13 @@ impl Default for LogicalFields { } impl FromIterator for LogicalFields { - fn from_iter>(iter: T) -> Self { + fn from_iter>(iter: T) -> Self { iter.into_iter().map(Arc::new).collect() } } impl FromIterator for LogicalFields { - fn from_iter>(iter: T) -> Self { + fn from_iter>(iter: T) -> Self { Self(iter.into_iter().collect()) } } @@ -135,7 +140,11 @@ impl FromIterator<(i8, LogicalFieldRef)> for LogicalUnionFields { impl From<&UnionFields> for LogicalUnionFields { fn from(value: &UnionFields) -> Self { - Self::from_iter(value.iter().map(|(i, f)| (i, LogicalFieldRef::new(f.into())))) + Self::from_iter( + value + .iter() + .map(|(i, f)| (i, LogicalFieldRef::new(f.into()))), + ) } } diff --git a/datafusion/common/src/logical_type/mod.rs b/datafusion/common/src/logical_type/mod.rs index 1629aa705c6b..bb93727e838e 100644 --- a/datafusion/common/src/logical_type/mod.rs +++ b/datafusion/common/src/logical_type/mod.rs @@ -70,7 +70,9 @@ impl NativeType { pub fn new_list(inner: TypeRelation, nullable: bool) -> Self { Self { physical: DataType::new_list(inner.physical().clone(), nullable), - logical: LogicalType::List(LogicalFieldRef::new(LogicalField::new_list_field(inner, nullable))), + logical: LogicalType::List(LogicalFieldRef::new( + LogicalField::new_list_field(inner, nullable), + )), } } @@ -161,9 +163,7 @@ impl From<&DataType> for LogicalType { DataType::Float16 => LogicalType::Float16, DataType::Float32 => LogicalType::Float32, DataType::Float64 => LogicalType::Float64, - DataType::Timestamp(tu, tz) => { - LogicalType::Timestamp(tu.clone(), tz.clone()) - } + DataType::Timestamp(tu, tz) => LogicalType::Timestamp(tu.clone(), tz.clone()), DataType::Date32 | DataType::Date64 => LogicalType::Date, DataType::Time32(tu) => LogicalType::Time32(tu.clone()), DataType::Time64(tu) => LogicalType::Time64(tu.clone()), diff --git a/datafusion/common/src/logical_type/schema.rs b/datafusion/common/src/logical_type/schema.rs index 7e64d1d8df77..99cf5754c559 100644 --- a/datafusion/common/src/logical_type/schema.rs +++ b/datafusion/common/src/logical_type/schema.rs @@ -111,7 +111,7 @@ impl From for LogicalSchemaBuilder { } impl Extend for LogicalSchemaBuilder { - fn extend>(&mut self, iter: T) { + fn extend>(&mut self, iter: T) { let iter = iter.into_iter(); self.fields.reserve(iter.size_hint().0); for f in iter { @@ -121,7 +121,7 @@ impl Extend for LogicalSchemaBuilder { } impl Extend for LogicalSchemaBuilder { - fn extend>(&mut self, iter: T) { + fn extend>(&mut self, iter: T) { let iter = iter.into_iter(); self.fields.reserve(iter.size_hint().0); for f in iter { @@ -174,14 +174,16 @@ impl Into for LogicalSchema { } } - impl LogicalSchema { pub fn new(fields: impl Into) -> Self { Self::new_with_metadata(fields, HashMap::new()) } #[inline] - pub fn new_with_metadata(fields: impl Into, metadata: HashMap) -> Self { + pub fn new_with_metadata( + fields: impl Into, + metadata: HashMap, + ) -> Self { Self { fields: fields.into(), metadata, diff --git a/datafusion/common/src/logical_type/signature.rs b/datafusion/common/src/logical_type/signature.rs index 2d75d4ea0a24..d01063a67305 100644 --- a/datafusion/common/src/logical_type/signature.rs +++ b/datafusion/common/src/logical_type/signature.rs @@ -20,7 +20,10 @@ use std::sync::Arc; use arrow_schema::{IntervalUnit, TimeUnit}; -use super::{field::LogicalFieldRef, fields::{LogicalFields, LogicalUnionFields}}; +use super::{ + field::LogicalFieldRef, + fields::{LogicalFields, LogicalUnionFields}, +}; #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub enum LogicalType { @@ -50,8 +53,7 @@ pub enum LogicalType { Map(LogicalFieldRef, bool), Decimal128(u8, i8), Decimal256(u8, i8), - Union(LogicalUnionFields) - // TODO: extension signatures? + Union(LogicalUnionFields), // TODO: extension signatures? } impl fmt::Display for LogicalType { diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index 84684509abaa..66908d04626d 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -16,9 +16,9 @@ // under the License. use crate::error::{_plan_datafusion_err, _plan_err}; +use crate::logical_type::TypeRelation; use crate::{Result, ScalarValue}; use std::collections::HashMap; -use crate::logical_type::TypeRelation; /// The parameter value corresponding to the placeholder #[derive(Debug, Clone)] diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 875c8cc907da..f921e9e493d2 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -59,8 +59,8 @@ use datafusion_expr::{ use datafusion_functions_aggregate::expr_fn::{avg, count, median, stddev, sum}; use async_trait::async_trait; -use datafusion_common::logical_type::ExtensionType; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::ExtensionType; /// Contains options that control how data is /// written out from a DataFrame @@ -669,7 +669,10 @@ impl DataFrame { original_schema_fields .clone() .filter(|f| { - !matches!(f.data_type().logical(), LogicalType::Binary | LogicalType::Boolean) + !matches!( + f.data_type().logical(), + LogicalType::Binary | LogicalType::Boolean + ) }) .map(|f| min(col(f.name())).alias(f.name())) .collect::>(), @@ -680,7 +683,10 @@ impl DataFrame { original_schema_fields .clone() .filter(|f| { - !matches!(f.data_type().logical(), LogicalType::Binary | LogicalType::Boolean) + !matches!( + f.data_type().logical(), + LogicalType::Binary | LogicalType::Boolean + ) }) .map(|f| max(col(f.name())).alias(f.name())) .collect::>(), diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 434c36250ee8..bd85edb31fa9 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -22,8 +22,8 @@ use std::sync::Arc; use crate::datasource::TableProvider; -use datafusion_common::{internal_err, Constraints}; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::{internal_err, Constraints}; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; /// DataFusion default table source, wrapping TableProvider. diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 6ea768963798..5f8e0ae461d4 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -38,13 +38,13 @@ use futures::stream::FuturesUnordered; use futures::{stream::BoxStream, StreamExt, TryStreamExt}; use log::{debug, trace}; +use datafusion_common::logical_type::field::LogicalField; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion_common::{Column, DFSchema, DataFusionError}; use datafusion_expr::{Expr, Volatility}; use datafusion_physical_expr::create_physical_expr; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; -use datafusion_common::logical_type::field::LogicalField; /// Check whether the given expression can be resolved using only the columns `col_names`. /// This means that if this function returns true: diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e6b3ef5a7f1c..15786224c742 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -41,7 +41,10 @@ use crate::{ use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; -use datafusion_common::{config_datafusion_err, internal_err, plan_err, project_schema, Constraints, ToDFSchema, SchemaExt}; +use datafusion_common::{ + config_datafusion_err, internal_err, plan_err, project_schema, Constraints, + SchemaExt, ToDFSchema, +}; use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; use datafusion_physical_expr::{ @@ -49,10 +52,10 @@ use datafusion_physical_expr::{ }; use async_trait::async_trait; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; /// Configuration for creating a [`ListingTable`] #[derive(Debug, Clone)] @@ -786,7 +789,8 @@ impl TableProvider for ListingTable { let filters = if let Some(expr) = conjunction(filters.to_vec()) { // NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns. - let table_df_schema = LogicalSchema::from(self.table_schema.as_ref().clone()).to_dfschema()?; + let table_df_schema = + LogicalSchema::from(self.table_schema.as_ref().clone()).to_dfschema()?; let filters = create_physical_expr(&expr, &table_df_schema, state.execution_props())?; Some(filters) @@ -857,9 +861,7 @@ impl TableProvider for ListingTable { ) -> Result> { let schema = SchemaRef::new(self.schema().as_ref().clone().into()); // Check that the schema of the plan matches the schema of this table. - if !schema - .logically_equivalent_names_and_types(&input.schema()) - { + if !schema.logically_equivalent_names_and_types(&input.schema()) { return plan_err!( // Return an error if schema of the input query does not match with the table schema. "Inserting query must have the same schema with the table." @@ -980,13 +982,9 @@ impl ListingTable { .buffered(ctx.config_options().execution.meta_fetch_concurrency); let schema = SchemaRef::new(self.schema().as_ref().clone().into()); - let (files, statistics) = get_statistics_with_limit( - files, - schema, - limit, - self.options.collect_stat, - ) - .await?; + let (files, statistics) = + get_statistics_with_limit(files, schema, limit, self.options.collect_stat) + .await?; Ok(( split_files(files, self.options.target_partitions), @@ -1879,8 +1877,13 @@ mod tests { // Since logical plan contains a filter, increasing parallelism is helpful. // Therefore, we will have 8 partitions in the final plan. // Create an insert plan to insert the source data into the initial table - let insert_into_table = - LogicalPlanBuilder::insert_into(scan_plan, "t", &schema.as_ref().clone().into(), false)?.build()?; + let insert_into_table = LogicalPlanBuilder::insert_into( + scan_plan, + "t", + &schema.as_ref().clone().into(), + false, + )? + .build()?; // Create a physical plan from the insert plan let plan = session_ctx .state() diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index cc5023f1f6b8..d54b21ebef0e 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -17,10 +17,6 @@ //! [`MemTable`] for querying `Vec` by DataFusion. -use std::any::Any; -use std::collections::HashMap; -use std::fmt::{self, Debug}; -use std::sync::Arc; use crate::datasource::{TableProvider, TableType}; use crate::error::Result; use crate::execution::context::SessionState; @@ -33,6 +29,10 @@ use crate::physical_plan::{ Partitioning, SendableRecordBatchStream, }; use crate::physical_planner::create_physical_sort_exprs; +use std::any::Any; +use std::collections::HashMap; +use std::fmt::{self, Debug}; +use std::sync::Arc; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -41,12 +41,12 @@ use datafusion_execution::TaskContext; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use futures::StreamExt; use log::debug; use parking_lot::Mutex; use tokio::sync::RwLock; use tokio::task::JoinSet; -use datafusion_common::logical_type::schema::LogicalSchemaRef; /// Type alias for partition data pub type PartitionData = Arc>>; @@ -218,8 +218,11 @@ impl TableProvider for MemTable { partitions.push(inner_vec.clone()) } - let mut exec = - MemoryExec::try_new(&partitions, SchemaRef::new(self.schema().as_ref().clone().into()), projection.cloned())?; + let mut exec = MemoryExec::try_new( + &partitions, + SchemaRef::new(self.schema().as_ref().clone().into()), + projection.cloned(), + )?; let show_sizes = state.config_options().explain.show_sizes; exec = exec.with_show_sizes(show_sizes); @@ -269,9 +272,7 @@ impl TableProvider for MemTable { // Create a physical plan from the logical plan. // Check that the schema of the plan matches the schema of this table. let schema = SchemaRef::new(self.schema.as_ref().clone()); - if !schema - .logically_equivalent_names_and_types(&input.schema()) - { + if !schema.logically_equivalent_names_and_types(&input.schema()) { return plan_err!( "Inserting query must have the same schema with the table." ); @@ -624,8 +625,13 @@ mod tests { // Create a table scan logical plan to read from the source table let scan_plan = LogicalPlanBuilder::scan("source", source, None)?.build()?; // Create an insert plan to insert the source data into the initial table - let insert_into_table = - LogicalPlanBuilder::insert_into(scan_plan, "t", &schema.as_ref().clone().into(), false)?.build()?; + let insert_into_table = LogicalPlanBuilder::insert_into( + scan_plan, + "t", + &schema.as_ref().clone().into(), + false, + )? + .build()?; // Create a physical plan from the insert plan let plan = session_ctx .state() diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index f09e77a9a8be..684a30c50cfa 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -625,9 +625,9 @@ fn create_output_array( #[cfg(test)] mod tests { - use arrow_array::Int32Array; use super::*; use crate::{test::columns, test_util::aggr_test_schema}; + use arrow_array::Int32Array; #[test] fn physical_plan_config_no_projection() { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index b7d9336c6c05..f9cce5f783ff 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -425,8 +425,6 @@ mod test { use parquet::file::reader::{FileReader, SerializedFileReader}; use rand::prelude::*; - - // We should ignore predicate that read non-primitive columns #[test] fn test_filter_candidate_builder_ignore_complex_types() { diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 0a8605e55cfa..46765269ad4b 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -21,8 +21,8 @@ use std::any::Any; use std::sync::Arc; use async_trait::async_trait; -use datafusion_common::{not_impl_err, Constraints, Statistics}; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::{not_impl_err, Constraints, Statistics}; use datafusion_expr::{CreateExternalTable, LogicalPlan}; pub use datafusion_expr::{TableProviderFilterPushDown, TableType}; diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index bf078229130e..24b49fe7b179 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -42,8 +42,8 @@ use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use async_trait::async_trait; -use futures::StreamExt; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use futures::StreamExt; /// A [`TableProviderFactory`] for [`StreamTable`] #[derive(Debug, Default)] diff --git a/datafusion/core/src/datasource/streaming.rs b/datafusion/core/src/datasource/streaming.rs index f8aefed7bec8..5810cbf226bb 100644 --- a/datafusion/core/src/datasource/streaming.rs +++ b/datafusion/core/src/datasource/streaming.rs @@ -23,14 +23,14 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use async_trait::async_trait; -use datafusion_common::{plan_err, Result}; -use datafusion_expr::{Expr, TableType}; -use log::debug; -use datafusion_common::logical_type::schema::LogicalSchemaRef; use crate::datasource::TableProvider; use crate::execution::context::SessionState; use crate::physical_plan::streaming::{PartitionStream, StreamingTableExec}; use crate::physical_plan::ExecutionPlan; +use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::{plan_err, Result}; +use datafusion_expr::{Expr, TableType}; +use log::debug; /// A [`TableProvider`] that streams a set of [`PartitionStream`] pub struct StreamingTable { diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 648b8411eeed..6dd1c4398262 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -21,8 +21,8 @@ use std::{any::Any, sync::Arc}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; -use datafusion_common::Column; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::Column; use datafusion_expr::{LogicalPlanBuilder, TableProviderFilterPushDown}; use crate::{ diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index cf45fef5169b..4ad15bbbec75 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -50,6 +50,7 @@ use datafusion_common::config::{ConfigExtension, ConfigOptions, TableOptions}; use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; use datafusion_common::file_options::file_type::FileType; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::tree_node::TreeNode; use datafusion_common::{ config_err, not_impl_err, plan_datafusion_err, DFSchema, DataFusionError, @@ -85,7 +86,6 @@ use std::fmt::Debug; use std::sync::Arc; use url::Url; use uuid::Uuid; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; /// Execution context for registering data sources and executing queries. /// See [`SessionContext`] for a higher level API. diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 423eb8023f31..d3993cdb5ddd 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2271,8 +2271,8 @@ mod tests { use arrow::array::{ArrayRef, DictionaryArray, Int32Array}; use arrow::datatypes::{DataType, Field, Int32Type}; - use datafusion_common::{assert_contains, DFSchemaRef, TableReference}; use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::{assert_contains, DFSchemaRef, TableReference}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{col, lit, LogicalPlanBuilder, UserDefinedLogicalNodeCore}; diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 7bef5aad85a1..87af4c42093f 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -49,9 +49,9 @@ use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use futures::Stream; use tempfile::TempDir; -use datafusion_common::logical_type::schema::LogicalSchemaRef; // backwards compatibility #[cfg(feature = "parquet")] pub use datafusion_common::test_util::parquet_test_data; diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index e8e33badc0af..03bf63f0320d 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -38,11 +38,11 @@ use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig, SessionContext}; use crate::datasource::physical_plan::parquet::ParquetExecBuilder; +use datafusion_common::logical_type::schema::LogicalSchema; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; -use datafusion_common::logical_type::schema::LogicalSchema; /// a ParquetFile that has been created for testing. pub struct TestParquetFile { @@ -154,7 +154,8 @@ impl TestParquetFile { extensions: None, }); - let df_schema = LogicalSchema::from(self.schema.as_ref().clone()).to_dfschema_ref()?; + let df_schema = + LogicalSchema::from(self.schema.as_ref().clone()).to_dfschema_ref()?; // run coercion on the filters to coerce types etc. let props = ExecutionProps::new(); diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index b79b655bd2fd..75cc827b879e 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -43,8 +43,8 @@ use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::{ExecutionMode, PlanProperties}; use async_trait::async_trait; -use futures::stream::Stream; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use futures::stream::Stream; mod provider_filter_pushdown; mod statistics; diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 225d3023f520..93cdb0aa4bf7 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -46,8 +46,8 @@ use datafusion::prelude::JoinType; use datafusion::prelude::{CsvReadOptions, ParquetReadOptions}; use datafusion::test_util::{parquet_test_data, populate_csv_partitions}; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; -use datafusion_common::{assert_contains, DataFusionError, ScalarValue, UnnestOptions}; use datafusion_common::logical_type::TypeRelation; +use datafusion_common::{assert_contains, DataFusionError, ScalarValue, UnnestOptions}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_expr::expr::{GroupingSet, Sort}; diff --git a/datafusion/core/tests/expr_api/parse_sql_expr.rs b/datafusion/core/tests/expr_api/parse_sql_expr.rs index 4cd617dd6989..7938ae9a8185 100644 --- a/datafusion/core/tests/expr_api/parse_sql_expr.rs +++ b/datafusion/core/tests/expr_api/parse_sql_expr.rs @@ -17,9 +17,9 @@ use arrow_schema::DataType; use datafusion::prelude::{CsvReadOptions, SessionContext}; -use datafusion_common::{DFSchemaRef, Result, ToDFSchema}; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::{DFSchemaRef, Result, ToDFSchema}; use datafusion_expr::Expr; use datafusion_sql::unparser::Unparser; diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 73de1cc2cfdb..60f342c25cb8 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -23,7 +23,10 @@ use arrow_buffer::IntervalDayTime; use chrono::{DateTime, TimeZone, Utc}; use datafusion::{error::Result, execution::context::ExecutionProps, prelude::*}; use datafusion_common::cast::as_int32_array; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::ScalarValue; use datafusion_common::{DFSchemaRef, ToDFSchema}; use datafusion_expr::expr::ScalarFunction; @@ -38,9 +41,6 @@ use datafusion_optimizer::optimizer::Optimizer; use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyExpressions}; use datafusion_optimizer::{OptimizerContext, OptimizerRule}; use std::sync::Arc; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; -use datafusion_common::logical_type::schema::LogicalSchema; /// In order to simplify expressions, DataFusion must have information /// about the expressions. diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index cfe74b12e25a..31fb84d26624 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -32,10 +32,10 @@ use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; +use datafusion_common::logical_type::schema::LogicalSchema; use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; -use datafusion_common::logical_type::schema::LogicalSchema; async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { let object_store_url = ObjectStoreUrl::local_filesystem(); @@ -67,7 +67,9 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { extensions: None, }; - let df_schema = LogicalSchema::from(schema.as_ref().clone()).to_dfschema().unwrap(); + let df_schema = LogicalSchema::from(schema.as_ref().clone()) + .to_dfschema() + .unwrap(); let execution_props = ExecutionProps::new(); let predicate = create_physical_expr(&filter, &df_schema, &execution_props).unwrap(); diff --git a/datafusion/core/tests/sql/create_drop.rs b/datafusion/core/tests/sql/create_drop.rs index 9ab3098efb49..6934fba3d634 100644 --- a/datafusion/core/tests/sql/create_drop.rs +++ b/datafusion/core/tests/sql/create_drop.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +use super::*; use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::test_util::TestTableFactory; use datafusion_common::logical_type::ExtensionType; -use super::*; #[tokio::test] async fn create_custom_table() -> Result<()> { @@ -67,9 +67,18 @@ async fn create_external_table_with_ddl() -> Result<()> { assert_eq!(3, table_schema.fields().len()); - assert_eq!(&DataType::Int32, table_schema.field(0).data_type().physical()); - assert_eq!(&DataType::Utf8, table_schema.field(1).data_type().physical()); - assert_eq!(&DataType::Boolean, table_schema.field(2).data_type().physical()); + assert_eq!( + &DataType::Int32, + table_schema.field(0).data_type().physical() + ); + assert_eq!( + &DataType::Utf8, + table_schema.field(1).data_type().physical() + ); + assert_eq!( + &DataType::Boolean, + table_schema.field(2).data_type().physical() + ); Ok(()) } diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 886124312d20..07d6209676b0 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -25,6 +25,7 @@ use datafusion::execution::context::{FunctionFactory, RegisterFunction, SessionS use datafusion::prelude::*; use datafusion::{execution::registry::FunctionRegistry, test_util}; use datafusion_common::cast::{as_float64_array, as_int32_array}; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, assert_contains, exec_err, internal_err, @@ -39,7 +40,6 @@ use datafusion_expr::{ use datafusion_functions_array::range::range_udf; use parking_lot::Mutex; use sqlparser::ast::Ident; -use datafusion_common::logical_type::ExtensionType; /// test that casting happens on udfs. /// c11 is f32, but `custom_sqrt` requires f64. Casting happens but the logical plan and @@ -646,7 +646,11 @@ impl ScalarUDFImpl for TakeUDF { ); }; - arg_exprs.get(take_idx).unwrap().get_type(schema).map(|t| t.physical().clone()) + arg_exprs + .get(take_idx) + .unwrap() + .get_type(schema) + .map(|t| t.physical().clone()) } // The actual implementation @@ -837,7 +841,8 @@ impl TryFrom for ScalarFunctionWrapper { return_type: definition .return_type .expect("Return type has to be defined!") - .physical().clone(), + .physical() + .clone(), // TODO(@notfilippo): avoid conversion to physical type signature: Signature::exact( definition diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index 2052ea90be6a..457068d59b2f 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -29,13 +29,13 @@ use datafusion::execution::TaskContext; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::SessionContext; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; use std::fs::File; use std::io::Seek; use std::path::Path; use std::sync::Arc; -use datafusion_common::logical_type::schema::LogicalSchemaRef; /// test simple udtf with define read_csv with parameters #[tokio::test] diff --git a/datafusion/expr/src/conditional_expressions.rs b/datafusion/expr/src/conditional_expressions.rs index 4a96442bcc4c..15cc9967b64d 100644 --- a/datafusion/expr/src/conditional_expressions.rs +++ b/datafusion/expr/src/conditional_expressions.rs @@ -19,9 +19,9 @@ use crate::expr::Case; use crate::{expr_schema::ExprSchemable, Expr}; use arrow::datatypes::DataType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{plan_err, DFSchema, Result}; use std::collections::HashSet; -use datafusion_common::logical_type::TypeRelation; /// Helper struct for building [Expr::Case] pub struct CaseBuilder { diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index c7e94d8e0b83..8a2ac9bdde31 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -34,6 +34,8 @@ use crate::{ use crate::{window_frame, Volatility}; use arrow::datatypes::DataType; +use datafusion_common::logical_type::field::LogicalFieldRef; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -41,8 +43,6 @@ use datafusion_common::{ internal_err, plan_err, Column, DFSchema, Result, ScalarValue, TableReference, }; use sqlparser::ast::NullTreatment; -use datafusion_common::logical_type::field::LogicalFieldRef; -use datafusion_common::logical_type::TypeRelation; /// Represents logical expressions such as `A + 1`, or `CAST(c1 AS int)`. /// @@ -567,7 +567,10 @@ pub struct Cast { impl Cast { /// Create a new Cast expression pub fn new(expr: Box, data_type: impl Into) -> Self { - Self { expr, data_type: data_type.into() } + Self { + expr, + data_type: data_type.into(), + } } } @@ -583,7 +586,10 @@ pub struct TryCast { impl TryCast { /// Create a new TryCast expression pub fn new(expr: Box, data_type: impl Into) -> Self { - Self { expr, data_type: data_type.into() } + Self { + expr, + data_type: data_type.into(), + } } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 0821b475b183..dbd3fd78a2f7 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -35,12 +35,12 @@ use arrow::compute::kernels::cast_utils::{ parse_interval_day_time, parse_interval_month_day_nano, parse_interval_year_month, }; use arrow::datatypes::{DataType, Field}; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{Column, Result, ScalarValue}; use std::any::Any; use std::fmt::Debug; use std::ops::Not; use std::sync::Arc; -use datafusion_common::logical_type::TypeRelation; /// Create a column expression based on a qualified or unqualified column name. Will /// normalize unquoted identifiers according to SQL rules (identifiers will become lowercase). diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 7a3c556b3802..12784bfdec79 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -413,7 +413,8 @@ mod test { .map(|f| Arc::new(LogicalField::new(f.to_string(), DataType::Int8, false))) .collect::>(); let schema = Arc::new(LogicalSchema::new(fields)); - DFSchema::from_field_specific_qualified_schema(qualifiers, &schema.into()).unwrap() + DFSchema::from_field_specific_qualified_schema(qualifiers, &schema.into()) + .unwrap() } #[test] diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index dbf7236e8a85..994053c98c3e 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -153,11 +153,11 @@ mod test { use std::ops::Add; use std::sync::Arc; - use arrow::datatypes::{DataType, Field, Schema}; use crate::{ cast, col, lit, logical_plan::builder::LogicalTableSource, min, test::function_stub::avg, try_cast, LogicalPlanBuilder, }; + use arrow::datatypes::{DataType, Field, Schema}; use super::*; diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index e7f31ca6e584..e9ba37dcc005 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -27,15 +27,15 @@ use crate::type_coercion::functions::{ use crate::{utils, LogicalPlan, Projection, Subquery, WindowFunctionDefinition}; use arrow::compute::can_cast_types; use arrow::datatypes::DataType; +use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::{ internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, ExprSchema, Result, TableReference, }; use std::collections::HashMap; use std::sync::Arc; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; /// trait to allow expr to typable with respect to a schema pub trait ExprSchemable { @@ -55,11 +55,17 @@ pub trait ExprSchemable { ) -> Result<(Option, Arc)>; /// cast to a type with respect to a schema - fn cast_to(self, cast_to_type: &TypeRelation, schema: &dyn ExprSchema) -> Result; + fn cast_to( + self, + cast_to_type: &TypeRelation, + schema: &dyn ExprSchema, + ) -> Result; /// given a schema, return the type and nullability of the expr - fn data_type_and_nullable(&self, schema: &dyn ExprSchema) - -> Result<(TypeRelation, bool)>; + fn data_type_and_nullable( + &self, + schema: &dyn ExprSchema, + ) -> Result<(TypeRelation, bool)>; } impl ExprSchemable for Expr { @@ -454,7 +460,10 @@ impl ExprSchemable for Expr { let left = left.data_type_and_nullable(schema)?; let right = right.data_type_and_nullable(schema)?; // TODO(@notfilippo): do not convert to physical type - Ok((get_result_type(&left.0.physical(), op, &right.0.physical())?.into(), left.1 || right.1)) + Ok(( + get_result_type(&left.0.physical(), op, &right.0.physical())?.into(), + left.1 || right.1, + )) } _ => Ok((self.get_type(schema)?, self.nullable(schema)?)), } @@ -505,7 +514,11 @@ impl ExprSchemable for Expr { /// /// This function errors when it is impossible to cast the /// expression to the target [arrow::datatypes::DataType]. - fn cast_to(self, cast_to_type: &TypeRelation, schema: &dyn ExprSchema) -> Result { + fn cast_to( + self, + cast_to_type: &TypeRelation, + schema: &dyn ExprSchema, + ) -> Result { let this_type = self.get_type(schema)?; if this_type == *cast_to_type { return Ok(self); @@ -530,7 +543,10 @@ impl ExprSchemable for Expr { } /// cast subquery in InSubquery/ScalarSubquery to a given type. -pub fn cast_subquery(subquery: Subquery, cast_to_type: &TypeRelation) -> Result { +pub fn cast_subquery( + subquery: Subquery, + cast_to_type: &TypeRelation, +) -> Result { if subquery.subquery.schema().field(0).data_type() == cast_to_type { return Ok(subquery); } @@ -664,7 +680,8 @@ mod tests { assert_eq!( &LogicalType::Utf8, expr.get_type(&MockExprSchema::new().with_data_type(DataType::Utf8.into())) - .unwrap().logical() + .unwrap() + .logical() ); } @@ -692,8 +709,9 @@ mod tests { ); let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("foo", DataType::Int32, true).with_metadata(meta.clone())] - .into(), + vec![LogicalField::new("foo", DataType::Int32, true) + .with_metadata(meta.clone())] + .into(), HashMap::new(), ) .unwrap(); diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index cef839860a70..1d015e227718 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -51,12 +51,16 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::{get_target_functional_dependencies, internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, TableReference, UnnestOptions, ToDFSchema}; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::fields::LogicalFields; use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::{ + get_target_functional_dependencies, internal_err, not_impl_err, plan_datafusion_err, + plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, + TableReference, ToDFSchema, UnnestOptions, +}; /// Default table name for unnamed table pub const UNNAMED_TABLE: &str = "?table?"; @@ -194,7 +198,9 @@ impl LogicalPlanBuilder { } if let Some(prev_type) = common_type { // get common type of each column values. - let Some(new_type) = values_coercion(&data_type.physical(), &prev_type.physical()) else { + let Some(new_type) = + values_coercion(&data_type.physical(), &prev_type.physical()) + else { return plan_err!("Inconsistent data type across values list at row {i} column {j}. Was {prev_type} but found {data_type}"); }; common_type = Some(new_type.into()); @@ -1191,7 +1197,11 @@ pub fn change_redundant_column(fields: &LogicalFields) -> Vec { *counter += 1; if *counter > 1 { let new_name = format!("{}:{}", field.name(), *counter - 1); - LogicalField::new(new_name, field.data_type().clone(), field.is_nullable()) + LogicalField::new( + new_name, + field.data_type().clone(), + field.is_nullable(), + ) } else { field.as_ref().clone() } @@ -1220,53 +1230,54 @@ pub fn build_join_schema( let right_fields = right.iter(); let left_fields = left.iter(); - let qualified_fields: Vec<(Option, Arc)> = match join_type { - JoinType::Inner => { - // left then right - let left_fields = left_fields - .map(|(q, f)| (q.cloned(), f.clone())) - .collect::>(); - let right_fields = right_fields - .map(|(q, f)| (q.cloned(), f.clone())) - .collect::>(); - left_fields.into_iter().chain(right_fields).collect() - } - JoinType::Left => { - // left then right, right set to nullable in case of not matched scenario - let left_fields = left_fields - .map(|(q, f)| (q.cloned(), f.clone())) - .collect::>(); - left_fields - .into_iter() - .chain(nullify_fields(right_fields)) - .collect() - } - JoinType::Right => { - // left then right, left set to nullable in case of not matched scenario - let right_fields = right_fields - .map(|(q, f)| (q.cloned(), f.clone())) - .collect::>(); - nullify_fields(left_fields) - .into_iter() - .chain(right_fields) - .collect() - } - JoinType::Full => { - // left then right, all set to nullable in case of not matched scenario - nullify_fields(left_fields) - .into_iter() - .chain(nullify_fields(right_fields)) - .collect() - } - JoinType::LeftSemi | JoinType::LeftAnti => { - // Only use the left side for the schema - left_fields.map(|(q, f)| (q.cloned(), f.clone())).collect() - } - JoinType::RightSemi | JoinType::RightAnti => { - // Only use the right side for the schema - right_fields.map(|(q, f)| (q.cloned(), f.clone())).collect() - } - }; + let qualified_fields: Vec<(Option, Arc)> = + match join_type { + JoinType::Inner => { + // left then right + let left_fields = left_fields + .map(|(q, f)| (q.cloned(), f.clone())) + .collect::>(); + let right_fields = right_fields + .map(|(q, f)| (q.cloned(), f.clone())) + .collect::>(); + left_fields.into_iter().chain(right_fields).collect() + } + JoinType::Left => { + // left then right, right set to nullable in case of not matched scenario + let left_fields = left_fields + .map(|(q, f)| (q.cloned(), f.clone())) + .collect::>(); + left_fields + .into_iter() + .chain(nullify_fields(right_fields)) + .collect() + } + JoinType::Right => { + // left then right, left set to nullable in case of not matched scenario + let right_fields = right_fields + .map(|(q, f)| (q.cloned(), f.clone())) + .collect::>(); + nullify_fields(left_fields) + .into_iter() + .chain(right_fields) + .collect() + } + JoinType::Full => { + // left then right, all set to nullable in case of not matched scenario + nullify_fields(left_fields) + .into_iter() + .chain(nullify_fields(right_fields)) + .collect() + } + JoinType::LeftSemi | JoinType::LeftAnti => { + // Only use the left side for the schema + left_fields.map(|(q, f)| (q.cloned(), f.clone())).collect() + } + JoinType::RightSemi | JoinType::RightAnti => { + // Only use the right side for the schema + right_fields.map(|(q, f)| (q.cloned(), f.clone())).collect() + } + }; let func_dependencies = left.functional_dependencies().join( right.functional_dependencies(), join_type, @@ -1394,7 +1405,11 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result DFSchemaRef { Arc::new( - LogicalSchema::new(vec![LogicalField::new("count", DataType::UInt64, false)]).try_into().unwrap() + LogicalSchema::new(vec![LogicalField::new("count", DataType::UInt64, false)]) + .try_into() + .unwrap(), ) } diff --git a/datafusion/expr/src/logical_plan/extension.rs b/datafusion/expr/src/logical_plan/extension.rs index 182018cbaf65..05f0b59c3fda 100644 --- a/datafusion/expr/src/logical_plan/extension.rs +++ b/datafusion/expr/src/logical_plan/extension.rs @@ -349,5 +349,9 @@ impl UserDefinedLogicalNode for T { } fn get_all_columns_from_schema(schema: &DFSchema) -> HashSet { - schema.fields().iter().map(|f| f.name().to_string()).collect() + schema + .fields() + .iter() + .map(|f| f.name().to_string()) + .collect() } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index a10897a2d6a2..fdf6882f2c3b 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -55,10 +55,10 @@ use datafusion_common::{ use crate::display::PgJsonVisitor; use crate::logical_plan::tree_node::unwrap_arc; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; -pub use datafusion_common::{JoinConstraint, JoinType}; use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +pub use datafusion_common::{JoinConstraint, JoinType}; /// A `LogicalPlan` is a node in a tree of relational operators (such as /// Projection or Filter). @@ -356,10 +356,13 @@ impl LogicalPlan { /// Returns the (fixed) output schema for explain plans pub fn explain_schema() -> LogicalSchemaRef { - LogicalSchemaRef::new(Schema::new(vec![ - Field::new("plan_type", DataType::Utf8, false), - Field::new("plan", DataType::Utf8, false), - ]).into()) + LogicalSchemaRef::new( + Schema::new(vec![ + Field::new("plan_type", DataType::Utf8, false), + Field::new("plan", DataType::Utf8, false), + ]) + .into(), + ) } /// Returns the (fixed) output schema for `DESCRIBE` plans @@ -368,7 +371,8 @@ impl LogicalPlan { Field::new("column_name", DataType::Utf8, false), Field::new("data_type", DataType::Utf8, false), Field::new("is_nullable", DataType::Utf8, false), - ]).into() + ]) + .into() } /// Returns all expressions (non-recursively) evaluated by the current @@ -2382,9 +2386,7 @@ impl TableScan { let df_schema = DFSchema::new_with_metadata( p.iter() - .map(|i| { - (Some(table_name.clone()), schema.field(*i).clone()) - }) + .map(|i| (Some(table_name.clone()), schema.field(*i).clone())) .collect(), schema.metadata.clone(), )?; diff --git a/datafusion/expr/src/planner.rs b/datafusion/expr/src/planner.rs index b8c0378f46aa..a87c9d12c372 100644 --- a/datafusion/expr/src/planner.rs +++ b/datafusion/expr/src/planner.rs @@ -19,13 +19,13 @@ use std::sync::Arc; +use crate::{AggregateUDF, Expr, GetFieldAccess, ScalarUDF, TableSource, WindowUDF}; use arrow::datatypes::SchemaRef; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ config::ConfigOptions, file_options::file_type::FileType, not_impl_err, DFSchema, Result, TableReference, }; -use datafusion_common::logical_type::TypeRelation; -use crate::{AggregateUDF, Expr, GetFieldAccess, ScalarUDF, TableSource, WindowUDF}; /// Provides the `SQL` query planner meta-data about tables and /// functions referenced in SQL statements, without a direct dependency on other diff --git a/datafusion/expr/src/simplify.rs b/datafusion/expr/src/simplify.rs index 25b3384e2baf..440aca75f07a 100644 --- a/datafusion/expr/src/simplify.rs +++ b/datafusion/expr/src/simplify.rs @@ -17,10 +17,10 @@ //! Structs and traits to provide the information needed for expression simplification. +use crate::{execution_props::ExecutionProps, Expr, ExprSchemable}; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::{DFSchemaRef, DataFusionError, Result}; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; -use crate::{execution_props::ExecutionProps, Expr, ExprSchemable}; /// Provides the information necessary to apply algebraic simplification to an /// [Expr]. See [SimplifyContext] for one concrete implementation. @@ -75,7 +75,9 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { /// returns true if this Expr has boolean type fn is_boolean_type(&self, expr: &Expr) -> Result { for schema in &self.schema { - if let Ok(LogicalType::Boolean) = expr.get_type(schema).map(|t| t.logical().clone()) { + if let Ok(LogicalType::Boolean) = + expr.get_type(schema).map(|t| t.logical().clone()) + { return Ok(true); } } diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 729304097477..fa64a8e151b0 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -19,8 +19,8 @@ use crate::{Expr, LogicalPlan}; -use datafusion_common::{Constraints, Result}; use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::{Constraints, Result}; use std::any::Any; diff --git a/datafusion/expr/src/type_coercion/mod.rs b/datafusion/expr/src/type_coercion/mod.rs index 14bd82e18d8f..3e610c77e3c8 100644 --- a/datafusion/expr/src/type_coercion/mod.rs +++ b/datafusion/expr/src/type_coercion/mod.rs @@ -36,15 +36,16 @@ pub mod binary; pub mod functions; pub mod other; -use datafusion_common::logical_type::{signature::LogicalType, TypeRelation, ExtensionType}; +use datafusion_common::logical_type::{ + signature::LogicalType, ExtensionType, TypeRelation, +}; /// Determine whether the given data type `dt` represents signed numeric values. pub fn is_signed_numeric(dt: &TypeRelation) -> bool { use LogicalType::*; matches!( dt.logical(), - Int8 - | Int16 + Int8 | Int16 | Int32 | Int64 | Float16 @@ -85,5 +86,8 @@ pub fn is_utf8_or_large_utf8(dt: &TypeRelation) -> bool { /// Determine whether the given data type `dt` is a `Decimal`. pub fn is_decimal(dt: &TypeRelation) -> bool { - matches!(dt.logical(), LogicalType::Decimal128(_, _) | LogicalType::Decimal256(_, _)) + matches!( + dt.logical(), + LogicalType::Decimal128(_, _) | LogicalType::Decimal256(_, _) + ) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index df4da26cd818..048090d602c7 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -41,7 +41,7 @@ use datafusion_common::{ use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::schema::LogicalSchema; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; /// The value to which `COUNT(*)` is expanded to in @@ -838,8 +838,7 @@ pub fn can_hash(data_type: &TypeRelation) -> bool { use LogicalType::*; matches!( data_type.logical(), - Null - | Boolean + Null | Boolean | Int8 | Int16 | Int32 diff --git a/datafusion/expr/src/var_provider.rs b/datafusion/expr/src/var_provider.rs index a1f4c40843ee..5d2b198d6342 100644 --- a/datafusion/expr/src/var_provider.rs +++ b/datafusion/expr/src/var_provider.rs @@ -17,8 +17,8 @@ //! Variable provider -use datafusion_common::{Result, ScalarValue}; use datafusion_common::logical_type::TypeRelation; +use datafusion_common::{Result, ScalarValue}; /// Variable type, system/user defined #[derive(Debug, Clone, PartialEq, Eq, Hash)] diff --git a/datafusion/functions-array/src/planner.rs b/datafusion/functions-array/src/planner.rs index 5c464cc82844..4fc1152f8be1 100644 --- a/datafusion/functions-array/src/planner.rs +++ b/datafusion/functions-array/src/planner.rs @@ -17,8 +17,8 @@ //! SQL planning extensions like [`ArrayFunctionPlanner`] and [`FieldAccessPlanner`] -use datafusion_common::{utils::list_ndims, DFSchema, Result}; use datafusion_common::logical_type::ExtensionType; +use datafusion_common::{utils::list_ndims, DFSchema, Result}; use datafusion_expr::{ planner::{PlannerResult, RawBinaryExpr, RawFieldAccessExpr, UserDefinedSQLPlanner}, sqlparser, AggregateFunction, Expr, ExprSchemable, GetFieldAccess, diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index 4f4ca28a206c..44d32b97a7ea 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -21,13 +21,13 @@ use arrow::array::{ use arrow::datatypes::DataType; use datafusion_common::cast::{as_map_array, as_struct_array}; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{ exec_err, plan_datafusion_err, plan_err, ExprSchema, Result, ScalarValue, }; use datafusion_expr::{ColumnarValue, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; -use datafusion_common::logical_type::ExtensionType; #[derive(Debug)] pub struct GetFieldFunc { diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index 630bab95cf46..d7f0c6a7e54f 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -188,7 +188,9 @@ impl ScalarUDFImpl for LogFunc { }; match number { - Expr::Literal(value) if value == ScalarValue::new_one(&number_datatype.physical())? => { + Expr::Literal(value) + if value == ScalarValue::new_one(&number_datatype.physical())? => + { Ok(ExprSimplifyResult::Simplified(lit(ScalarValue::new_zero( &info.get_data_type(&base)?.physical(), )?))) diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index 9d47e27d8c65..ddc546552342 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -28,12 +28,12 @@ use datafusion_expr::expr::ScalarFunction; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{ColumnarValue, Expr, ScalarUDF}; +use super::log::LogFunc; use arrow::array::{ArrayRef, Float64Array, Int64Array}; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use std::any::Any; use std::sync::Arc; -use super::log::LogFunc; #[derive(Debug)] pub struct PowerFunc { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 4b15d279898b..0fbda671de58 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -107,8 +107,8 @@ mod tests { WindowFrameUnits, }; use datafusion_functions_aggregate::count::count_udaf; - use std::sync::Arc; use datafusion_functions_aggregate::expr_fn::{count, sum}; + use std::sync::Arc; fn assert_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { assert_analyzed_plan_eq_display_indent( diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index f5c8e98f0fc3..0eae65765a9d 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -23,12 +23,12 @@ use arrow::datatypes::{DataType, IntervalUnit}; use datafusion_common::config::ConfigOptions; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::expr::{ self, AggregateFunctionDefinition, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, ScalarFunction, WindowFunction, @@ -268,9 +268,12 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { ) })?; let expr = match left_type { - _ => Box::new(expr.cast_to(&coerced_type.clone().into(), self.schema)?), + _ => { + Box::new(expr.cast_to(&coerced_type.clone().into(), self.schema)?) + } }; - let pattern = Box::new(pattern.cast_to(&coerced_type.into(), self.schema)?); + let pattern = + Box::new(pattern.cast_to(&coerced_type.into(), self.schema)?); Ok(Transformed::yes(Expr::Like(Like::new( negated, expr, @@ -331,10 +334,16 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { let expr_data_type = expr.get_type(self.schema)?; let list_data_types = list .iter() - .map(|list_expr| list_expr.get_type(self.schema).map(|t| t.physical().clone())) + .map(|list_expr| { + list_expr + .get_type(self.schema) + .map(|t| t.physical().clone()) + }) .collect::>>()?; - let result_type = - get_coerce_type_for_list(&expr_data_type.physical(), &list_data_types); + let result_type = get_coerce_type_for_list( + &expr_data_type.physical(), + &list_data_types, + ); match result_type { None => plan_err!( "Can not find compatible types to compare {expr_data_type:?} with {list_data_types:?}" @@ -580,7 +589,8 @@ fn coerce_window_frame( } WindowFrameUnits::Rows | WindowFrameUnits::Groups => DataType::UInt64.into(), }; - window_frame.start_bound = coerce_frame_bound(&target_type, window_frame.start_bound)?; + window_frame.start_bound = + coerce_frame_bound(&target_type, window_frame.start_bound)?; window_frame.end_bound = coerce_frame_bound(&target_type, window_frame.end_bound)?; Ok(window_frame) } @@ -589,7 +599,11 @@ fn coerce_window_frame( // The above op will be rewrite to the binary op when creating the physical op. fn get_casted_expr_for_bool_op(expr: Expr, schema: &DFSchema) -> Result { let left_type = expr.get_type(schema)?; - get_input_types(&left_type.physical(), &Operator::IsDistinctFrom, &DataType::Boolean)?; + get_input_types( + &left_type.physical(), + &Operator::IsDistinctFrom, + &DataType::Boolean, + )?; expr.cast_to(&DataType::Boolean.into(), schema) } @@ -731,8 +745,10 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { .iter() .map(|(when, _then)| when.get_type(schema).map(|t| t.physical().clone())) .collect::>>()?; - let coerced_type = - get_coerce_type_for_case_expression(&when_types, Some(&case_type.physical())); + let coerced_type = get_coerce_type_for_case_expression( + &when_types, + Some(&case_type.physical()), + ); coerced_type.ok_or_else(|| { plan_datafusion_err!( "Failed to coerce case ({case_type:?}) and when ({when_types:?}) \ @@ -755,7 +771,9 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { let case_expr = case .expr .zip(case_when_coerce_type.as_ref()) - .map(|(case_expr, coercible_type)| case_expr.cast_to(&coercible_type.into(), schema)) + .map(|(case_expr, coercible_type)| { + case_expr.cast_to(&coercible_type.into(), schema) + }) .transpose()? .map(Box::new); let when_then = case @@ -793,10 +811,10 @@ mod test { use arrow::datatypes::DataType::Utf8; use arrow::datatypes::{DataType, Field, TimeUnit}; - use datafusion_common::tree_node::{TransformedResult, TreeNode}; - use datafusion_common::{DFSchema, DFSchemaRef, Result, ScalarValue}; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::TypeRelation; + use datafusion_common::tree_node::{TransformedResult, TreeNode}; + use datafusion_common::{DFSchema, DFSchemaRef, Result, ScalarValue}; use datafusion_expr::expr::{self, InSubquery, Like, ScalarFunction}; use datafusion_expr::logical_plan::{EmptyRelation, Projection}; use datafusion_expr::test::function_stub::avg_udaf; diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index d88491be1979..cba594325baa 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -65,12 +65,12 @@ mod tests { use super::*; use crate::test::*; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::ToDFSchema; use datafusion_expr::{ expr_rewriter::coerce_plan_expr_for_schema, logical_plan::table_scan, }; use std::sync::Arc; - use datafusion_common::logical_type::schema::LogicalSchema; - use datafusion_common::ToDFSchema; fn schema() -> Schema { Schema::new(vec![ diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 9f5a5ced741e..9446c3c959e0 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -807,10 +807,10 @@ mod tests { }; use crate::{OptimizerContext, OptimizerRule}; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::logical_type::field::LogicalField; use datafusion_common::{ Column, DFSchema, DFSchemaRef, JoinType, Result, TableReference, }; - use datafusion_common::logical_type::field::LogicalField; use datafusion_expr::AggregateExt; use datafusion_expr::{ binary_expr, build_join_schema, diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 5212b8df6e1b..6a3aa47de3be 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -248,8 +248,8 @@ mod tests { use arrow::datatypes::{DataType, Field, Fields, Schema}; - use datafusion_common::{Column, DFSchema, JoinType, ScalarValue}; use datafusion_common::logical_type::fields::LogicalFields; + use datafusion_common::{Column, DFSchema, JoinType, ScalarValue}; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, Expr, Operator, diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 94a500778da1..4c59e13a0fe7 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -2387,9 +2387,9 @@ mod tests { let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), filters: vec![], - projected_schema: Arc::new(DFSchema::try_from( - LogicalSchema::from((*test_provider.schema()).clone()), - )?), + projected_schema: Arc::new(DFSchema::try_from(LogicalSchema::from( + (*test_provider.schema()).clone(), + ))?), projection: None, source: Arc::new(test_provider), fetch: None, @@ -2459,9 +2459,9 @@ mod tests { let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), filters: vec![col("a").eq(lit(10i64)), col("b").gt(lit(11i64))], - projected_schema: Arc::new(DFSchema::try_from( - LogicalSchema::from((*test_provider.schema()).clone()), - )?), + projected_schema: Arc::new(DFSchema::try_from(LogicalSchema::from( + (*test_provider.schema()).clone(), + ))?), projection: Some(vec![0]), source: Arc::new(test_provider), fetch: None, @@ -2488,9 +2488,9 @@ mod tests { let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), filters: vec![], - projected_schema: Arc::new(DFSchema::try_from( - LogicalSchema::from((*test_provider.schema()).clone()), - )?), + projected_schema: Arc::new(DFSchema::try_from(LogicalSchema::from( + (*test_provider.schema()).clone(), + ))?), projection: Some(vec![0]), source: Arc::new(test_provider), fetch: None, diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index bf7cc3bd9001..ec4278a30330 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -28,7 +28,9 @@ use arrow::{ }; use datafusion_common::{ - cast::{as_large_list_array, as_list_array}, logical_type::ExtensionType, tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter} + cast::{as_large_list_array, as_list_array}, + logical_type::ExtensionType, + tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, }; use datafusion_common::{internal_err, DFSchema, DataFusionError, Result, ScalarValue}; use datafusion_expr::expr::{ @@ -1244,7 +1246,9 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if expr_contains(&left, &right, BitwiseXor) => { let expr = delete_xor_in_complex_expr(&left, &right, false); Transformed::yes(if expr == *right { - Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&right)?.physical())?) + Expr::Literal(ScalarValue::new_zero( + &info.get_data_type(&right)?.physical(), + )?) } else { expr }) @@ -1258,7 +1262,9 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { }) if expr_contains(&right, &left, BitwiseXor) => { let expr = delete_xor_in_complex_expr(&right, &left, true); Transformed::yes(if expr == *left { - Expr::Literal(ScalarValue::new_zero(&info.get_data_type(&left)?.physical())?) + Expr::Literal(ScalarValue::new_zero( + &info.get_data_type(&left)?.physical(), + )?) } else { expr }) @@ -1768,7 +1774,13 @@ fn inlist_except(mut l1: InList, l2: InList) -> Result { #[cfg(test)] mod tests { - use datafusion_common::{assert_contains, logical_type::signature::LogicalType, DFSchemaRef, ToDFSchema}; + use crate::simplify_expressions::SimplifyContext; + use crate::test::test_table_scan_with_name; + use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::{ + assert_contains, logical_type::signature::LogicalType, DFSchemaRef, ToDFSchema, + }; use datafusion_expr::{ function::{ AccumulatorArgs, AggregateFunctionSimplification, @@ -1782,10 +1794,6 @@ mod tests { ops::{BitAnd, BitOr, BitXor}, sync::Arc, }; - use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::schema::LogicalSchema; - use crate::simplify_expressions::SimplifyContext; - use crate::test::test_table_scan_with_name; use super::*; @@ -3102,7 +3110,10 @@ mod tests { #[test] fn simplify_expr_eq() { let schema = expr_test_schema(); - assert_eq!(col("c2").get_type(&schema).unwrap().logical(), &LogicalType::Boolean); + assert_eq!( + col("c2").get_type(&schema).unwrap().logical(), + &LogicalType::Boolean + ); // true = true -> true assert_eq!(simplify(lit(true).eq(lit(true))), lit(true)); @@ -3126,7 +3137,10 @@ mod tests { // expression to non-boolean. // // Make sure c1 column to be used in tests is not boolean type - assert_eq!(col("c1").get_type(&schema).unwrap().logical(), &LogicalType::Utf8); + assert_eq!( + col("c1").get_type(&schema).unwrap().logical(), + &LogicalType::Utf8 + ); // don't fold c1 = foo assert_eq!(simplify(col("c1").eq(lit("foo"))), col("c1").eq(lit("foo")),); @@ -3136,7 +3150,10 @@ mod tests { fn simplify_expr_not_eq() { let schema = expr_test_schema(); - assert_eq!(col("c2").get_type(&schema).unwrap().logical(), &LogicalType::Boolean); + assert_eq!( + col("c2").get_type(&schema).unwrap().logical(), + &LogicalType::Boolean + ); // c2 != true -> !c2 assert_eq!(simplify(col("c2").not_eq(lit(true))), col("c2").not(),); @@ -3157,7 +3174,10 @@ mod tests { // when one of the operand is not of boolean type, folding the // other boolean constant will change return type of // expression to non-boolean. - assert_eq!(col("c1").get_type(&schema).unwrap().logical(), &LogicalType::Utf8); + assert_eq!( + col("c1").get_type(&schema).unwrap().logical(), + &LogicalType::Utf8 + ); assert_eq!( simplify(col("c1").not_eq(lit("foo"))), diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 35e4e72c53f8..a083945b6a3e 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -149,9 +149,9 @@ impl SimplifyExpressions { mod tests { use std::ops::Not; + use crate::optimizer::Optimizer; use arrow::datatypes::{DataType, Field, Schema}; use chrono::{DateTime, Utc}; - use crate::optimizer::Optimizer; use datafusion_expr::logical_plan::builder::table_scan_with_filters; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ @@ -702,7 +702,8 @@ mod tests { let t1 = test_table_scan_with_name("t1")?; let t2 = test_table_scan_with_name("t2")?; - let left_key = col("t1.a") + lit(1i64).cast_to(&DataType::UInt32.into(), t1.schema())?; + let left_key = + col("t1.a") + lit(1i64).cast_to(&DataType::UInt32.into(), t1.schema())?; let right_key = col("t2.a") + lit(2i64).cast_to(&DataType::UInt32.into(), t2.schema())?; let plan = LogicalPlanBuilder::from(t1) diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index b0880c566c75..3d4a33a29458 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -26,13 +26,13 @@ use crate::{OptimizerConfig, OptimizerRule}; use crate::utils::NamePreserver; use arrow::datatypes::{ - DataType, TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION + DataType, TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION, }; use arrow::temporal_conversions::{MICROSECONDS, MILLISECONDS, NANOSECONDS}; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{internal_err, DFSchema, DFSchemaRef, Result, ScalarValue}; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::expr::{BinaryExpr, Cast, InList, TryCast}; use datafusion_expr::utils::merge_schema; use datafusion_expr::{lit, Expr, ExprSchemable, LogicalPlan, Operator}; @@ -278,8 +278,7 @@ fn is_comparison_op(op: &Operator) -> bool { /// Returns true if [UnwrapCastExprRewriter] supports this data type fn is_supported_type(data_type: &TypeRelation) -> bool { - is_supported_numeric_type(data_type) - || is_supported_string_type(data_type) + is_supported_numeric_type(data_type) || is_supported_string_type(data_type) } /// Returns true if [[UnwrapCastExprRewriter]] suppors this numeric type @@ -479,7 +478,11 @@ fn try_cast_string_literal( } /// Cast a timestamp value from one unit to another -fn cast_between_timestamp(from: impl Into, to: impl Into, value: i128) -> Option { +fn cast_between_timestamp( + from: impl Into, + to: impl Into, + value: i128, +) -> Option { let value = value as i64; let from_scale = match from.into().logical() { LogicalType::Timestamp(TimeUnit::Second, _) => 1, @@ -510,7 +513,10 @@ mod tests { use super::*; - use arrow::{compute::{cast_with_options, CastOptions}, datatypes::DataType}; + use arrow::{ + compute::{cast_with_options, CastOptions}, + datatypes::DataType, + }; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::tree_node::TransformedResult; use datafusion_expr::{cast, col, in_list, try_cast}; @@ -1138,8 +1144,10 @@ mod tests { if let ( DataType::Timestamp(left_unit, left_tz), DataType::Timestamp(right_unit, right_tz), - ) = (actual_value.data_type().into(), expected_value.data_type().into()) - { + ) = ( + actual_value.data_type().into(), + expected_value.data_type().into(), + ) { assert_eq!(left_unit, right_unit); assert_eq!(left_tz, right_tz); } diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 4843cca9c633..9714f26d9fb7 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -22,9 +22,9 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use datafusion_common::config::ConfigOptions; -use datafusion_common::{plan_err, Result}; use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::logical_type::TypeRelation; +use datafusion_common::{plan_err, Result}; use datafusion_expr::test::function_stub::sum_udaf; use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF}; use datafusion_functions_aggregate::average::avg_udaf; diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 98d501c97854..7f809cdccbf6 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -429,7 +429,10 @@ pub fn in_list( let expr_data_type = expr.data_type(schema)?; for list_expr in list.iter() { let list_expr_data_type = list_expr.data_type(schema)?; - if !DFSchema::datatype_is_logically_equal(&expr_data_type.clone().into(), &list_expr_data_type.clone().into()) { + if !DFSchema::datatype_is_logically_equal( + &expr_data_type.clone().into(), + &list_expr_data_type.clone().into(), + ) { return internal_err!( "The data type inlist should be same, the value type is {expr_data_type}, one of list expr type is {list_expr_data_type}" ); diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 8bac68e108d4..8b521a7acae5 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -24,9 +24,11 @@ use crate::{ }; use arrow::datatypes::Schema; -use datafusion_common::logical_type::ExtensionType; -use datafusion_common::{exec_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, ToDFSchema}; use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::ExtensionType; +use datafusion_common::{ + exec_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, ToDFSchema, +}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction}; use datafusion_expr::var_provider::is_system_variables; @@ -378,7 +380,8 @@ mod tests { let expr = col("letter").eq(lit("A")); let schema = Schema::new(vec![Field::new("letter", DataType::Utf8, false)]); - let df_schema = DFSchema::try_from_qualified_schema("data", &schema.clone().into())?; + let df_schema = + DFSchema::try_from_qualified_schema("data", &schema.clone().into())?; let p = create_physical_expr(&expr, &df_schema, &ExecutionProps::new())?; let batch = RecordBatch::try_new( diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index 67d20837afdc..ad1c15d9ea13 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -419,10 +419,10 @@ impl<'a> ColOpLit<'a> { #[cfg(test)] mod test { - use std::sync::OnceLock; - use arrow_schema::{DataType, Field, Schema, SchemaRef}; use super::*; use crate::planner::logical2physical; + use arrow_schema::{DataType, Field, Schema, SchemaRef}; + use std::sync::OnceLock; use datafusion_expr::expr_fn::*; use datafusion_expr::{lit, Expr}; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index abb4f3d814f9..d91fa2300203 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -30,6 +30,8 @@ use arrow::datatypes::{ }; use arrow::ipc::{reader::read_record_batch, root_as_message}; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ arrow_datafusion_err, config::{ @@ -42,8 +44,6 @@ use datafusion_common::{ Column, ColumnStatistics, Constraint, Constraints, DFSchema, DFSchemaRef, DataFusionError, JoinSide, ScalarValue, Statistics, TableReference, }; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::TypeRelation; #[derive(Debug)] pub enum Error { @@ -160,19 +160,21 @@ impl TryFrom<&protobuf::DfSchema> for DFSchema { df_schema: &protobuf::DfSchema, ) -> datafusion_common::Result { let df_fields = df_schema.columns.clone(); - let qualifiers_and_fields: Vec<(Option, Arc)> = df_fields - .iter() - .map(|df_field| { - let field: LogicalField = df_field.field.as_ref().required("field")?; - Ok(( - df_field - .qualifier - .as_ref() - .map(|q| q.relation.clone().into()), - Arc::new(field), - )) - }) - .collect::, Error>>()?; + let qualifiers_and_fields: Vec<(Option, Arc)> = + df_fields + .iter() + .map(|df_field| { + let field: LogicalField = + df_field.field.as_ref().required("field")?; + Ok(( + df_field + .qualifier + .as_ref() + .map(|q| q.relation.clone().into()), + Arc::new(field), + )) + }) + .collect::, Error>>()?; Ok(DFSchema::new_with_metadata( qualifiers_and_fields, @@ -344,7 +346,6 @@ impl TryFrom<&protobuf::Field> for Field { } } - impl TryFrom<&protobuf::Field> for LogicalField { type Error = Error; fn try_from(field: &protobuf::Field) -> Result { diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 2b611a8ffce6..598e8b63a8ee 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -23,8 +23,12 @@ use crate::protobuf_common::{ }; use arrow::array::{ArrayRef, RecordBatch}; use arrow::csv::WriterBuilder; -use arrow::datatypes::{DataType, Field, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionMode}; +use arrow::datatypes::{ + DataType, Field, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, + SchemaRef, TimeUnit, UnionMode, +}; use arrow::ipc::writer::{DictionaryTracker, IpcDataGenerator}; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::{ config::{ ColumnOptions, CsvOptions, JsonOptions, ParquetOptions, TableParquetOptions, @@ -36,7 +40,6 @@ use datafusion_common::{ Column, ColumnStatistics, Constraint, Constraints, DFSchema, DFSchemaRef, DataFusionError, JoinSide, ScalarValue, Statistics, }; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; #[derive(Debug)] pub enum Error { diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 0842b3f4cbc9..4e8e2d36406a 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -558,7 +558,8 @@ pub fn parse_expr( "expr", codec, )?); - let data_type: TypeRelation = cast.arrow_type.as_ref().required("arrow_type")?; + let data_type: TypeRelation = + cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::Cast(Cast::new(expr, data_type))) } ExprType::TryCast(cast) => { @@ -568,7 +569,8 @@ pub fn parse_expr( "expr", codec, )?); - let data_type: TypeRelation = cast.arrow_type.as_ref().required("arrow_type")?; + let data_type: TypeRelation = + cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::TryCast(TryCast::new(expr, data_type))) } ExprType::Sort(sort) => Ok(Expr::Sort(Sort::new( diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 05412f02de33..2ed27c61fbe6 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -63,11 +63,11 @@ use datafusion_expr::{ DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, }; +use self::to_proto::serialize_expr; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_proto_common::ArrowType; use prost::bytes::BufMut; use prost::Message; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; -use datafusion_proto_common::ArrowType; -use self::to_proto::serialize_expr; pub mod file_formats; pub mod from_proto; @@ -935,7 +935,7 @@ impl AsLogicalPlan for LogicalPlanNode { .. }) => { let provider = source_as_provider(source)?; - let schema= SchemaRef::new(provider.schema().as_ref().clone().into()); + let schema = SchemaRef::new(provider.schema().as_ref().clone().into()); let source = provider.as_any(); let projection = match projection { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 9a7010284be4..a9261dbfec21 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -26,11 +26,11 @@ use arrow::datatypes::{ DataType, Field, Fields, Int32Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, }; -use datafusion_common::logical_type::field::LogicalField; use datafusion::datasource::file_format::arrow::ArrowFormatFactory; use datafusion::datasource::file_format::csv::CsvFormatFactory; use datafusion::datasource::file_format::format_as_file_type; use datafusion::datasource::file_format::parquet::ParquetFormatFactory; +use datafusion_common::logical_type::field::LogicalField; use datafusion_proto::logical_plan::file_formats::{ ArrowLogicalExtensionCodec, CsvLogicalExtensionCodec, ParquetLogicalExtensionCodec, }; @@ -591,7 +591,10 @@ async fn roundtrip_expr_api() -> Result<()> { // list of expressions to round trip let expr_list = vec![ - encode(col("a").cast_to(&DataType::Utf8.into(), &schema)?, lit("hex")), + encode( + col("a").cast_to(&DataType::Utf8.into(), &schema)?, + lit("hex"), + ), decode(lit("1234"), lit("hex")), array_to_string(make_array(vec![lit(1), lit(2), lit(3)]), lit(",")), array_dims(make_array(vec![lit(1), lit(2), lit(3)])), @@ -1560,13 +1563,18 @@ fn roundtrip_schema() { fn roundtrip_dfschema() { let dfschema = DFSchema::new_with_metadata( vec![ - (None, Arc::new(LogicalField::new("a", DataType::Int64, false))), + ( + None, + Arc::new(LogicalField::new("a", DataType::Int64, false)), + ), ( Some("t".into()), Arc::new( - LogicalField::new("b", DataType::Decimal128(15, 2), true).with_metadata( - HashMap::from([(String::from("k1"), String::from("v1"))]), - ), + LogicalField::new("b", DataType::Decimal128(15, 2), true) + .with_metadata(HashMap::from([( + String::from("k1"), + String::from("v1"), + )])), ), ), ], diff --git a/datafusion/sql/examples/sql.rs b/datafusion/sql/examples/sql.rs index 9c27433cd129..f2f83447b56f 100644 --- a/datafusion/sql/examples/sql.rs +++ b/datafusion/sql/examples/sql.rs @@ -17,6 +17,7 @@ use arrow_schema::{DataType, Field, Schema}; use datafusion_common::config::ConfigOptions; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{plan_err, Result}; use datafusion_expr::WindowUDF; use datafusion_expr::{ @@ -30,7 +31,6 @@ use datafusion_sql::{ TableReference, }; use std::{collections::HashMap, sync::Arc}; -use datafusion_common::logical_type::TypeRelation; fn main() { let sql = "SELECT \ diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index d9f2578725d6..c5656266ae9e 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -17,6 +17,7 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::ExtensionType; use datafusion_common::{ internal_datafusion_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, Dependency, Result, @@ -36,7 +37,6 @@ use sqlparser::ast::{ }; use std::str::FromStr; use strum::IntoEnumIterator; -use datafusion_common::logical_type::ExtensionType; /// Suggest a valid function based on an invalid input function name pub fn suggest_valid_function( @@ -475,8 +475,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn check_unnest_arg(arg: &Expr, schema: &DFSchema) -> Result<()> { // Check argument type, array types are supported match arg.get_type(schema)?.logical() { - LogicalType::List(_) - | LogicalType::Struct(_) => Ok(()), + LogicalType::List(_) | LogicalType::Struct(_) => Ok(()), LogicalType::Null => { not_impl_err!("unnest() does not support null yet") } diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index 87419d97351e..39d0da8cd206 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -16,13 +16,13 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; +use datafusion_common::logical_type::field::LogicalField; use datafusion_common::{ internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, ScalarValue, TableReference, }; use datafusion_expr::{expr::ScalarFunction, lit, Case, Expr}; use sqlparser::ast::{Expr as SQLExpr, Ident}; -use datafusion_common::logical_type::field::LogicalField; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(super) fn sql_identifier_to_expr( diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index a564cb12d924..b585dfd6256e 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -21,11 +21,11 @@ use datafusion_expr::planner::PlannerResult; use datafusion_expr::planner::RawFieldAccessExpr; use sqlparser::ast::{CastKind, Expr as SQLExpr, Subscript, TrimWhereField, Value}; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, }; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::expr::InList; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ @@ -332,7 +332,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { { Expr::Cast(Cast::new( Box::new(expr), - TypeRelation::from(DataType::Timestamp(TimeUnit::Second, tz.clone())), + TypeRelation::from(DataType::Timestamp( + TimeUnit::Second, + tz.clone(), + )), )) } _ => expr, @@ -610,9 +613,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context, )?), match *time_zone { - SQLExpr::Value(Value::SingleQuotedString(s)) => { - TypeRelation::from(DataType::Timestamp(TimeUnit::Nanosecond, Some(s.into()))) - } + SQLExpr::Value(Value::SingleQuotedString(s)) => TypeRelation::from( + DataType::Timestamp(TimeUnit::Nanosecond, Some(s.into())), + ), _ => { return not_impl_err!( "Unsupported ast node in sqltorel: {time_zone:?}" @@ -759,7 +762,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let pattern = self.sql_expr_to_logical_expr(pattern, schema, planner_context)?; let pattern_type = pattern.get_type(schema)?; - if !matches!(pattern_type.logical(), LogicalType::Utf8 | LogicalType::Null) { + if !matches!( + pattern_type.logical(), + LogicalType::Utf8 | LogicalType::Null + ) { return plan_err!("Invalid pattern in LIKE expression"); } let escape_char = if let Some(char) = escape_char { @@ -790,7 +796,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let pattern = self.sql_expr_to_logical_expr(pattern, schema, planner_context)?; let pattern_type = pattern.get_type(schema)?; - if !matches!(pattern_type.logical(), LogicalType::Utf8 | LogicalType::Null) { + if !matches!( + pattern_type.logical(), + LogicalType::Utf8 | LogicalType::Null + ) { return plan_err!("Invalid pattern in SIMILAR TO expression"); } let escape_char = if let Some(char) = escape_char { diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index cfae655117ee..2bf50cec24e1 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -18,6 +18,7 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; use arrow::datatypes::DECIMAL128_MAX_PRECISION; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ internal_err, not_impl_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -28,7 +29,6 @@ use log::debug; use sqlparser::ast::{BinaryOperator, Expr as SQLExpr, Interval, Value}; use sqlparser::parser::ParserError::ParserError; use std::borrow::Cow; -use datafusion_common::logical_type::TypeRelation; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn parse_value( diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 74237b55b71a..c8d1f93bcea1 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -30,15 +30,15 @@ use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo}; use sqlparser::ast::{ColumnDef as SQLColumnDef, ColumnOption}; use sqlparser::ast::{DataType as SQLDataType, Ident, ObjectName, TableAlias}; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::TableReference; use datafusion_common::{ not_impl_err, plan_err, unqualified_field_not_found, DFSchema, DataFusionError, Result, }; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::TypeRelation; -use datafusion_common::logical_type::schema::LogicalSchema; use datafusion_expr::logical_plan::{LogicalPlan, LogicalPlanBuilder}; use datafusion_expr::utils::find_column_exprs; use datafusion_expr::{col, Expr}; @@ -341,7 +341,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) } - pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { + pub(crate) fn convert_data_type( + &self, + sql_type: &SQLDataType, + ) -> Result { match sql_type { SQLDataType::Array(ArrayElemTypeDef::AngleBracket(inner_sql_type)) | SQLDataType::Array(ArrayElemTypeDef::SquareBracket(inner_sql_type, _)) => { diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index d501588c68af..93b302f03e50 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -15,11 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::collections::{BTreeMap, HashMap, HashSet}; -use std::path::Path; -use std::str::FromStr; -use std::sync::Arc; -use arrow_schema::SchemaRef; use crate::parser::{ CopyToSource, CopyToStatement, CreateExternalTable, DFParser, ExplainStatement, LexOrdering, Statement as DFStatement, @@ -28,7 +23,15 @@ use crate::planner::{ object_name_to_qualifier, ContextProvider, PlannerContext, SqlToRel, }; use crate::utils::normalize_ident; +use arrow_schema::SchemaRef; +use std::collections::{BTreeMap, HashMap, HashSet}; +use std::path::Path; +use std::str::FromStr; +use std::sync::Arc; +use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ exec_err, not_impl_err, plan_datafusion_err, plan_err, schema_err, @@ -59,9 +62,6 @@ use sqlparser::ast::{ TableConstraint, TableFactor, TableWithJoins, TransactionMode, UnaryOperator, Value, }; use sqlparser::parser::ParserError::ParserError; -use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::TypeRelation; -use datafusion_common::logical_type::schema::LogicalSchema; fn ident_to_string(ident: &Ident) -> String { normalize_ident(ident.to_owned()) diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 19b9f2a61895..c085aa0e528d 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -34,11 +34,11 @@ use sqlparser::ast::{ UnaryOperator, }; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, Column, Result, ScalarValue, }; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; use datafusion_expr::{ expr::{Alias, Exists, InList, ScalarFunction, Sort, WindowFunction}, Between, BinaryExpr, Case, Cast, Expr, GroupingSet, Like, Operator, TryCast, @@ -958,7 +958,10 @@ impl Unparser<'_> { } } - fn arrow_dtype_to_ast_dtype(&self, data_type: &TypeRelation) -> Result { + fn arrow_dtype_to_ast_dtype( + &self, + data_type: &TypeRelation, + ) -> Result { match data_type.logical() { LogicalType::Null => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 3cd6ff0a7248..84858cb20616 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -20,9 +20,10 @@ use std::collections::HashMap; use arrow_schema::{ - DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE + DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, }; use datafusion_common::logical_type::signature::LogicalType; +use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ exec_err, internal_err, plan_err, Column, DataFusionError, Result, ScalarValue, @@ -32,7 +33,6 @@ use datafusion_expr::expr::{Alias, GroupingSet, Unnest, WindowFunction}; use datafusion_expr::utils::{expr_as_column_expr, find_column_exprs}; use datafusion_expr::{expr_vec_fmt, Expr, ExprSchemable, LogicalPlan}; use sqlparser::ast::Ident; -use datafusion_common::logical_type::{TypeRelation, ExtensionType}; /// Make a best-effort attempt at resolving all columns in the expression tree pub(crate) fn resolve_columns(expr: &Expr, plan: &LogicalPlan) -> Result { diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index eefa9e2b3471..e672d228cf4c 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -28,9 +28,9 @@ use datafusion_sql::unparser::dialect::{ }; use datafusion_sql::unparser::{expr_to_sql, plan_to_sql, Unparser}; +use crate::common::MockContextProvider; use sqlparser::dialect::{Dialect, GenericDialect, MySqlDialect}; use sqlparser::parser::Parser; -use crate::common::MockContextProvider; #[test] fn roundtrip_expr() { diff --git a/datafusion/sql/tests/common/mod.rs b/datafusion/sql/tests/common/mod.rs index 9c002a339338..d49b5f348b80 100644 --- a/datafusion/sql/tests/common/mod.rs +++ b/datafusion/sql/tests/common/mod.rs @@ -24,9 +24,9 @@ use std::{sync::Arc, vec}; use arrow_schema::*; use datafusion_common::config::ConfigOptions; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::{plan_err, GetExt, Result, TableReference}; use datafusion_common::logical_type::schema::LogicalSchemaRef; use datafusion_common::logical_type::TypeRelation; +use datafusion_common::{plan_err, GetExt, Result, TableReference}; use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource, WindowUDF}; use datafusion_sql::planner::ContextProvider; diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index e85614ec555b..cd8e49e68ede 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -37,13 +37,13 @@ use datafusion_sql::{ planner::{ParserOptions, SqlToRel}, }; +use datafusion_common::logical_type::TypeRelation; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::{ approx_median::approx_median_udaf, count::count_udaf, }; use rstest::rstest; use sqlparser::dialect::{Dialect, GenericDialect, HiveDialect, MySqlDialect}; -use datafusion_common::logical_type::TypeRelation; mod cases; mod common; @@ -3836,7 +3836,8 @@ Projection: person.id, orders.order_id let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); + let expected_types = + HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3868,7 +3869,8 @@ Projection: person.id, person.age let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); + let expected_types = + HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3901,7 +3903,7 @@ Projection: person.id, person.age let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), - ("$2".to_string(), Some(TypeRelation::from(DataType::Int32))), + ("$2".to_string(), Some(TypeRelation::from(DataType::Int32))), ]); assert_eq!(actual_types, expected_types); @@ -3939,7 +3941,8 @@ Projection: person.id, person.age let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); let actual_types = plan.get_parameter_types().unwrap(); - let expected_types = HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::UInt32)))]); + let expected_types = + HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::UInt32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3978,7 +3981,7 @@ Dml: op=[Update] table=[person] let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), - ("$2".to_string(), Some(TypeRelation::from(DataType::UInt32))), + ("$2".to_string(), Some(TypeRelation::from(DataType::UInt32))), ]); assert_eq!(actual_types, expected_types); diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index f9cca1ee06b2..9e54b5e9bde3 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -265,9 +265,9 @@ pub(crate) fn convert_schema_to_types(columns: &LogicalFields) -> Vec DFColumnType::Float, LogicalType::Utf8 => DFColumnType::Text, - LogicalType::Date - | LogicalType::Time32(_) - | LogicalType::Time64(_) => DFColumnType::DateTime, + LogicalType::Date | LogicalType::Time32(_) | LogicalType::Time64(_) => { + DFColumnType::DateTime + } LogicalType::Timestamp(_, _) => DFColumnType::Timestamp, _ => DFColumnType::Another, }) diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index ba834f225deb..281f82da5544 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -40,9 +40,9 @@ use datafusion_common::cast::as_float64_array; use datafusion_common::DataFusionError; use async_trait::async_trait; +use datafusion_common::logical_type::schema::LogicalSchemaRef; use log::info; use tempfile::TempDir; -use datafusion_common::logical_type::schema::LogicalSchemaRef; /// Context for running tests pub struct TestContext { diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 258d4dd6a7d3..6d2b3967bedb 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -17,11 +17,11 @@ use async_recursion::async_recursion; use datafusion::arrow::datatypes::{ - DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit + DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, }; use datafusion::common::logical_type::field::LogicalField; use datafusion::common::logical_type::schema::LogicalSchema; -use datafusion::common::logical_type::{TypeRelation, ExtensionType}; +use datafusion::common::logical_type::{ExtensionType, TypeRelation}; use datafusion::common::{ not_impl_datafusion_err, not_impl_err, plan_datafusion_err, plan_err, substrait_datafusion_err, substrait_err, DFSchema, DFSchemaRef, @@ -366,14 +366,14 @@ fn make_renamed_schema( let name = next_struct_field_name(0, dfs_names, &mut name_idx)?; Ok(( q.cloned(), - (**f) - .to_owned() - .with_name(name) - .with_data_type(rename_inner_fields( + (**f).to_owned().with_name(name).with_data_type( + rename_inner_fields( f.data_type().physical(), dfs_names, &mut name_idx, - )?.into()), + )? + .into(), + ), )) }) .collect::>>()? @@ -1462,7 +1462,9 @@ fn from_substrait_struct_type( for (i, f) in s.types.iter().enumerate() { let field = Field::new( next_struct_field_name(i, dfs_names, name_idx)?, - from_substrait_type(f, dfs_names, name_idx)?.physical().clone(), + from_substrait_type(f, dfs_names, name_idx)? + .physical() + .clone(), true, // We assume everything to be nullable since that's easier than ensuring it matches ); fields.push(field); @@ -1670,9 +1672,9 @@ fn from_substrait_literal( name_idx, )?; match lit.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => { - ScalarValue::List(ScalarValue::new_list_nullable(&[], &element_type.physical())) - } + DEFAULT_CONTAINER_TYPE_VARIATION_REF => ScalarValue::List( + ScalarValue::new_list_nullable(&[], &element_type.physical()), + ), LARGE_CONTAINER_TYPE_VARIATION_REF => ScalarValue::LargeList( ScalarValue::new_large_list(&[], &element_type.physical()), ), @@ -1861,19 +1863,25 @@ fn from_substrait_null( true, ); match l.type_variation_reference { - DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(ScalarValue::List( - Arc::new(GenericListArray::new_null(FieldRef::new(field.into()), 1)), - )), - LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(ScalarValue::LargeList( - Arc::new(GenericListArray::new_null(FieldRef::new(field.into()), 1)), - )), + DEFAULT_CONTAINER_TYPE_VARIATION_REF => { + Ok(ScalarValue::List(Arc::new(GenericListArray::new_null( + FieldRef::new(field.into()), + 1, + )))) + } + LARGE_CONTAINER_TYPE_VARIATION_REF => { + Ok(ScalarValue::LargeList(Arc::new( + GenericListArray::new_null(FieldRef::new(field.into()), 1), + ))) + } v => not_impl_err!( "Unsupported Substrait type variation {v} of type {kind:?}" ), } } r#type::Kind::Struct(s) => { - let fields: Fields = from_substrait_struct_type(s, dfs_names, name_idx)?.into(); + let fields: Fields = + from_substrait_struct_type(s, dfs_names, name_idx)?.into(); Ok(ScalarStructBuilder::new_null(fields)) } _ => not_impl_err!("Unsupported Substrait type for null: {kind:?}"), diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 940f25234c02..2e92ca8c5b20 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -1440,56 +1440,56 @@ fn to_substrait_type(dt: &DataType, nullable: bool) -> Result internal_err!("Null cast is not valid"), - DataType::Boolean => Ok(substrait::proto::Type { + DataType::Null => internal_err!("Null cast is not valid"), + DataType::Boolean => Ok(substrait::proto::Type { kind: Some(r#type::Kind::Bool(r#type::Boolean { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int8 => Ok(substrait::proto::Type { + DataType::Int8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I8(r#type::I8 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt8 => Ok(substrait::proto::Type { + DataType::UInt8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I8(r#type::I8 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int16 => Ok(substrait::proto::Type { + DataType::Int16 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I16(r#type::I16 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt16 => Ok(substrait::proto::Type { + DataType::UInt16 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I16(r#type::I16 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int32 => Ok(substrait::proto::Type { + DataType::Int32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I32(r#type::I32 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt32 => Ok(substrait::proto::Type { + DataType::UInt32 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I32(r#type::I32 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, })), }), - DataType::Int64 => Ok(substrait::proto::Type { + DataType::Int64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I64(r#type::I64 { type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability, })), }), - DataType::UInt64 => Ok(substrait::proto::Type { + DataType::UInt64 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::I64(r#type::I64 { type_variation_reference: UNSIGNED_INTEGER_TYPE_VARIATION_REF, nullability, From 9fd793afe5405731fc4470b3f70fd305470a17ba Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Mon, 8 Jul 2024 15:17:02 +0200 Subject: [PATCH 05/10] Format --- datafusion-examples/examples/sql_frontend.rs | 6 +++--- datafusion/common/src/logical_type/field.rs | 6 +----- datafusion/expr/src/expr_schema.rs | 15 ++++++++++----- datafusion/expr/src/logical_plan/builder.rs | 8 ++++---- datafusion/optimizer/src/optimizer.rs | 4 +++- datafusion/sql/src/utils.rs | 2 +- 6 files changed, 22 insertions(+), 19 deletions(-) diff --git a/datafusion-examples/examples/sql_frontend.rs b/datafusion-examples/examples/sql_frontend.rs index ee8b0b38806f..28ccb86ecd19 100644 --- a/datafusion-examples/examples/sql_frontend.rs +++ b/datafusion-examples/examples/sql_frontend.rs @@ -17,6 +17,9 @@ use arrow::datatypes::DataType; use datafusion_common::config::ConfigOptions; +use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{plan_err, Result}; use datafusion_expr::{ AggregateUDF, Expr, LogicalPlan, ScalarUDF, TableProviderFilterPushDown, TableSource, @@ -31,9 +34,6 @@ use datafusion_sql::sqlparser::parser::Parser; use datafusion_sql::TableReference; use std::any::Any; use std::sync::Arc; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; -use datafusion_common::logical_type::TypeRelation; /// This example shows how to use DataFusion's SQL planner to parse SQL text and /// build `LogicalPlan`s without executing them. diff --git a/datafusion/common/src/logical_type/field.rs b/datafusion/common/src/logical_type/field.rs index 0144cc4c6f4e..e59384110322 100644 --- a/datafusion/common/src/logical_type/field.rs +++ b/datafusion/common/src/logical_type/field.rs @@ -35,11 +35,7 @@ pub struct LogicalField { impl From<&Field> for LogicalField { fn from(value: &Field) -> Self { - Self::new( - value.name().clone(), - value.data_type(), - value.is_nullable(), - ) + Self::new(value.name().clone(), value.data_type(), value.is_nullable()) } } diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index fcba0e7684bc..26361b0c6ed7 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -128,9 +128,7 @@ impl ExprSchemable for Expr { let arg_data_type = expr.get_type(schema)?; // Unnest's output type is the inner type of the list match arg_data_type.logical() { - LogicalType::List(field) =>{ - Ok(field.data_type().clone()) - } + LogicalType::List(field) => Ok(field.data_type().clone()), LogicalType::Struct(_) => Ok(arg_data_type), LogicalType::Null => { not_impl_err!("unnest() does not support null yet") @@ -169,7 +167,9 @@ impl ExprSchemable for Expr { // perform additional function arguments validation (due to limited // expressiveness of `TypeSignature`), then infer return type - Ok(func.return_type_from_exprs(args, schema, &arg_data_types)?.into()) + Ok(func + .return_type_from_exprs(args, schema, &arg_data_types)? + .into()) } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { let data_types = args @@ -260,7 +260,12 @@ impl ExprSchemable for Expr { ref right, ref op, // TODO(@notfilippo): do not convert to physical type - }) => Ok(get_result_type(&left.get_type(schema)?.physical(), op, &right.get_type(schema)?.physical())?.into()), + }) => Ok(get_result_type( + &left.get_type(schema)?.physical(), + op, + &right.get_type(schema)?.physical(), + )? + .into()), Expr::Like { .. } | Expr::SimilarTo { .. } => Ok(DataType::Boolean.into()), Expr::Placeholder(Placeholder { data_type, .. }) => { data_type.clone().ok_or_else(|| { diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 0662e785afcd..e7a8d494e637 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1272,14 +1272,14 @@ pub fn build_join_schema( JoinType::LeftSemi | JoinType::LeftAnti => { // Only use the left side for the schema left_fields - .map(|(q, f)| (q.cloned(), Arc::clone(f))) - .collect() + .map(|(q, f)| (q.cloned(), Arc::clone(f))) + .collect() } JoinType::RightSemi | JoinType::RightAnti => { // Only use the right side for the schema right_fields - .map(|(q, f)| (q.cloned(), Arc::clone(f))) - .collect() + .map(|(q, f)| (q.cloned(), Arc::clone(f))) + .collect() } }; let func_dependencies = left.functional_dependencies().join( diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 339bf75e0de3..b1b43884e9a3 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -469,7 +469,9 @@ pub(crate) fn assert_schema_is_the_same( new_plan: &LogicalPlan, ) -> Result<()> { // TODO(@notfilippo): this was changed from equivalent_names_and_types because of arrow_cast. Is it ok? - let equivalent = new_plan.schema().logically_equivalent_names_and_types(prev_schema); + let equivalent = new_plan + .schema() + .logically_equivalent_names_and_types(prev_schema); if !equivalent { let e = DataFusionError::Internal(format!( diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 08a530347ef3..578ed0a929ca 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -22,6 +22,7 @@ use std::collections::HashMap; use arrow_schema::{ DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, }; +use datafusion_common::logical_type::signature::LogicalType; use datafusion_common::logical_type::{ExtensionType, TypeRelation}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, @@ -34,7 +35,6 @@ use datafusion_expr::expr::{Alias, GroupingSet, Unnest, WindowFunction}; use datafusion_expr::utils::{expr_as_column_expr, find_column_exprs}; use datafusion_expr::{expr_vec_fmt, Expr, ExprSchemable, LogicalPlan}; use sqlparser::ast::Ident; -use datafusion_common::logical_type::signature::LogicalType; /// Make a best-effort attempt at resolving all columns in the expression tree pub(crate) fn resolve_columns(expr: &Expr, plan: &LogicalPlan) -> Result { From 8b9715daf25c877f3c7311f706637b63e1473609 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Mon, 8 Jul 2024 17:30:37 +0200 Subject: [PATCH 06/10] Union mode --- datafusion-examples/examples/logical_type.rs | 4 +- datafusion/common/src/logical_type/fields.rs | 12 +++++- datafusion/common/src/logical_type/mod.rs | 2 +- .../common/src/logical_type/signature.rs | 42 ++++++++++++++++++- datafusion/sql/src/unparser/expr.rs | 2 +- 5 files changed, 55 insertions(+), 7 deletions(-) diff --git a/datafusion-examples/examples/logical_type.rs b/datafusion-examples/examples/logical_type.rs index 809b45d3b30e..6790ec726982 100644 --- a/datafusion-examples/examples/logical_type.rs +++ b/datafusion-examples/examples/logical_type.rs @@ -8,7 +8,7 @@ use datafusion::prelude::SessionContext; use datafusion_common::logical_type::field::LogicalField; use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::{ExtensionType, ExtensionTypeRef}; use datafusion_expr::{Expr, TableType}; use std::any::Any; use std::sync::Arc; @@ -64,7 +64,7 @@ impl TableProvider for ExampleTableSource { fn schema(&self) -> LogicalSchemaRef { // TODO: ugly? - let custom_magical_type: Arc = + let custom_magical_type: ExtensionTypeRef = Arc::new(CustomMagicalType::default()); // This schema will be equivalent to: diff --git a/datafusion/common/src/logical_type/fields.rs b/datafusion/common/src/logical_type/fields.rs index 3aa2445aa153..927c67c7617e 100644 --- a/datafusion/common/src/logical_type/fields.rs +++ b/datafusion/common/src/logical_type/fields.rs @@ -18,7 +18,7 @@ use std::ops::Deref; use std::sync::Arc; -use arrow_schema::{Field, Fields, UnionFields}; +use arrow_schema::{Field, FieldRef, Fields, UnionFields}; use super::field::{LogicalField, LogicalFieldRef}; @@ -153,3 +153,13 @@ impl From for LogicalUnionFields { Self::from(&value) } } + +impl Into for LogicalUnionFields { + fn into(self) -> UnionFields { + UnionFields::from_iter( + self.0 + .into_iter() + .map(|(i, f)| (*i, FieldRef::new(f.as_ref().clone().into()))), + ) + } +} diff --git a/datafusion/common/src/logical_type/mod.rs b/datafusion/common/src/logical_type/mod.rs index bb93727e838e..aed404c5c6c6 100644 --- a/datafusion/common/src/logical_type/mod.rs +++ b/datafusion/common/src/logical_type/mod.rs @@ -196,7 +196,7 @@ impl From<&DataType> for LogicalType { sorted.clone(), ), DataType::RunEndEncoded(_, f) => f.data_type().into(), - DataType::Union(f, _) => LogicalType::Union(f.into()), + DataType::Union(f, mode) => LogicalType::Union(f.into(), mode.clone()), } } } diff --git a/datafusion/common/src/logical_type/signature.rs b/datafusion/common/src/logical_type/signature.rs index d01063a67305..92126d466109 100644 --- a/datafusion/common/src/logical_type/signature.rs +++ b/datafusion/common/src/logical_type/signature.rs @@ -18,7 +18,7 @@ use core::fmt; use std::sync::Arc; -use arrow_schema::{IntervalUnit, TimeUnit}; +use arrow_schema::{DataType, FieldRef, IntervalUnit, TimeUnit, UnionMode}; use super::{ field::LogicalFieldRef, @@ -53,7 +53,7 @@ pub enum LogicalType { Map(LogicalFieldRef, bool), Decimal128(u8, i8), Decimal256(u8, i8), - Union(LogicalUnionFields), // TODO: extension signatures? + Union(LogicalUnionFields, UnionMode), // TODO: extension signatures? } impl fmt::Display for LogicalType { @@ -135,3 +135,41 @@ impl LogicalType { matches!(self, Null) } } + +impl Into for LogicalType { + fn into(self) -> DataType { + match self { + LogicalType::Null => DataType::Null, + LogicalType::Int8 => DataType::Int8, + LogicalType::Int16 => DataType::Int16, + LogicalType::Int32 => DataType::Int32, + LogicalType::Int64 => DataType::Int64, + LogicalType::UInt8 => DataType::UInt8, + LogicalType::UInt16 => DataType::UInt16, + LogicalType::UInt32 => DataType::UInt32, + LogicalType::UInt64 => DataType::UInt64, + LogicalType::Boolean => DataType::Boolean, + LogicalType::Float16 => DataType::Float16, + LogicalType::Float32 => DataType::Float32, + LogicalType::Float64 => DataType::Float64, + LogicalType::Utf8 => DataType::Utf8, + LogicalType::Binary => DataType::Binary, + LogicalType::Date => DataType::Date32, + LogicalType::Time32(tu) => DataType::Time32(tu), + LogicalType::Time64(tu) => DataType::Time64(tu), + LogicalType::Timestamp(tu, tz) => DataType::Timestamp(tu, tz), + LogicalType::Duration(tu) => DataType::Duration(tu), + LogicalType::Interval(iu) => DataType::Interval(iu), + LogicalType::List(field) => { + DataType::List(FieldRef::new(field.as_ref().clone().into())) + } + LogicalType::Struct(fields) => DataType::Struct(fields.into()), + LogicalType::Map(field, v) => { + DataType::Map(FieldRef::new(field.as_ref().clone().into()), v) + } + LogicalType::Decimal128(a, b) => DataType::Decimal128(a, b), + LogicalType::Decimal256(a, b) => DataType::Decimal256(a, b), + LogicalType::Union(union, mode) => DataType::Union(union.into(), mode), + } + } +} diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 33dceb342ca9..811bc085ec0a 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1012,7 +1012,7 @@ impl Unparser<'_> { LogicalType::Struct(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } - LogicalType::Union(_) => { + LogicalType::Union(_, _) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") } LogicalType::Decimal128(precision, scale) From 981486ae54983a85ad06d97b428f882bc3f8cd7c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 8 Jul 2024 19:45:09 -0600 Subject: [PATCH 07/10] Prepare 40 release --- Cargo.toml | 36 +-- datafusion-cli/Cargo.lock | 96 ++++---- datafusion-cli/Cargo.toml | 4 +- dev/changelog/40.0.0.md | 372 ++++++++++++++++++++++++++++++ docs/source/user-guide/configs.md | 2 +- 5 files changed, 441 insertions(+), 69 deletions(-) create mode 100644 dev/changelog/40.0.0.md diff --git a/Cargo.toml b/Cargo.toml index 968a74e37f10..f87205f0d067 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -53,7 +53,7 @@ license = "Apache-2.0" readme = "README.md" repository = "https://github.com/apache/datafusion" rust-version = "1.76" -version = "39.0.0" +version = "40.0.0" [workspace.dependencies] # We turn off default-features for some dependencies here so the workspaces which inherit them can @@ -86,23 +86,23 @@ bytes = "1.4" chrono = { version = "0.4.34", default-features = false } ctor = "0.2.0" dashmap = "5.5.0" -datafusion = { path = "datafusion/core", version = "39.0.0", default-features = false } -datafusion-common = { path = "datafusion/common", version = "39.0.0", default-features = false } -datafusion-common-runtime = { path = "datafusion/common-runtime", version = "39.0.0" } -datafusion-execution = { path = "datafusion/execution", version = "39.0.0" } -datafusion-expr = { path = "datafusion/expr", version = "39.0.0" } -datafusion-functions = { path = "datafusion/functions", version = "39.0.0" } -datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "39.0.0" } -datafusion-functions-array = { path = "datafusion/functions-array", version = "39.0.0" } -datafusion-optimizer = { path = "datafusion/optimizer", version = "39.0.0", default-features = false } -datafusion-physical-expr = { path = "datafusion/physical-expr", version = "39.0.0", default-features = false } -datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "39.0.0", default-features = false } -datafusion-physical-plan = { path = "datafusion/physical-plan", version = "39.0.0" } -datafusion-proto = { path = "datafusion/proto", version = "39.0.0" } -datafusion-proto-common = { path = "datafusion/proto-common", version = "39.0.0" } -datafusion-sql = { path = "datafusion/sql", version = "39.0.0" } -datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "39.0.0" } -datafusion-substrait = { path = "datafusion/substrait", version = "39.0.0" } +datafusion = { path = "datafusion/core", version = "40.0.0", default-features = false } +datafusion-common = { path = "datafusion/common", version = "40.0.0", default-features = false } +datafusion-common-runtime = { path = "datafusion/common-runtime", version = "40.0.0" } +datafusion-execution = { path = "datafusion/execution", version = "40.0.0" } +datafusion-expr = { path = "datafusion/expr", version = "40.0.0" } +datafusion-functions = { path = "datafusion/functions", version = "40.0.0" } +datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "40.0.0" } +datafusion-functions-array = { path = "datafusion/functions-array", version = "40.0.0" } +datafusion-optimizer = { path = "datafusion/optimizer", version = "40.0.0", default-features = false } +datafusion-physical-expr = { path = "datafusion/physical-expr", version = "40.0.0", default-features = false } +datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "40.0.0", default-features = false } +datafusion-physical-plan = { path = "datafusion/physical-plan", version = "40.0.0" } +datafusion-proto = { path = "datafusion/proto", version = "40.0.0" } +datafusion-proto-common = { path = "datafusion/proto-common", version = "40.0.0" } +datafusion-sql = { path = "datafusion/sql", version = "40.0.0" } +datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "40.0.0" } +datafusion-substrait = { path = "datafusion/substrait", version = "40.0.0" } doc-comment = "0.3" env_logger = "0.11" futures = "0.3" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 500e731a5b4f..42ec5922a73f 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -381,13 +381,13 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -875,9 +875,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.104" +version = "1.0.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74b6a57f98764a267ff415d50a25e6e166f3831a5071af4995296ea97d210490" +checksum = "066fce287b1d4eafef758e89e09d724a24808a9196fe9756b8ca90e86d0719a2" dependencies = [ "jobserver", "libc", @@ -1099,7 +1099,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -1123,7 +1123,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "apache-avro", @@ -1177,7 +1177,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "assert_cmd", @@ -1204,7 +1204,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "apache-avro", @@ -1225,14 +1225,14 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "39.0.0" +version = "40.0.0" dependencies = [ "tokio", ] [[package]] name = "datafusion-execution" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "chrono", @@ -1251,7 +1251,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1268,7 +1268,7 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "base64 0.22.1", @@ -1292,7 +1292,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1308,7 +1308,7 @@ dependencies = [ [[package]] name = "datafusion-functions-array" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "arrow-array", @@ -1327,7 +1327,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "async-trait", @@ -1345,7 +1345,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1373,7 +1373,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1385,7 +1385,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1417,7 +1417,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "arrow-array", @@ -1686,7 +1686,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2008,7 +2008,7 @@ dependencies = [ "http 1.1.0", "hyper 1.4.0", "hyper-util", - "rustls 0.23.10", + "rustls 0.23.11", "rustls-native-certs 0.7.1", "rustls-pki-types", "tokio", @@ -2699,7 +2699,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2822,7 +2822,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.10", + "rustls 0.23.11", "thiserror", "tokio", "tracing", @@ -2838,7 +2838,7 @@ dependencies = [ "rand", "ring 0.17.8", "rustc-hash", - "rustls 0.23.10", + "rustls 0.23.11", "slab", "thiserror", "tinyvec", @@ -2987,7 +2987,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.10", + "rustls 0.23.11", "rustls-native-certs 0.7.1", "rustls-pemfile 2.1.2", "rustls-pki-types", @@ -3117,9 +3117,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.10" +version = "0.23.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402" +checksum = "4828ea528154ae444e5a642dbb7d5623354030dc9822b83fd9bb79683c7399d0" dependencies = [ "once_cell", "ring 0.17.8", @@ -3296,22 +3296,22 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3446,7 +3446,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3492,7 +3492,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3505,7 +3505,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3527,9 +3527,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.68" +version = "2.0.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "901fa70d88b9d6c98022e23b4136f9f3e54e4662c3bc1bd1d84a42a9a0f0c1e9" +checksum = "2f0209b68b3613b093e0ec905354eccaedcfe83b8cb37cbdeae64026c3064c16" dependencies = [ "proc-macro2", "quote", @@ -3592,7 +3592,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3647,9 +3647,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce6b6a2fb3a985e99cebfaefa9faa3024743da73304ca1c683a36429613d3d22" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -3687,7 +3687,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3707,7 +3707,7 @@ version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" dependencies = [ - "rustls 0.23.10", + "rustls 0.23.11", "rustls-pki-types", "tokio", ] @@ -3784,7 +3784,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3829,7 +3829,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3983,7 +3983,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-shared", ] @@ -4017,7 +4017,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4282,7 +4282,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index bcacf1d52a9b..860dc123fa94 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-cli" description = "Command Line Client for DataFusion query engine." -version = "39.0.0" +version = "40.0.0" authors = ["Apache DataFusion "] edition = "2021" keywords = ["arrow", "datafusion", "query", "sql"] @@ -35,7 +35,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "39.0.0", features = [ +datafusion = { path = "../datafusion/core", version = "40.0.0", features = [ "avro", "crypto_expressions", "datetime_expressions", diff --git a/dev/changelog/40.0.0.md b/dev/changelog/40.0.0.md new file mode 100644 index 000000000000..163f8ea876a6 --- /dev/null +++ b/dev/changelog/40.0.0.md @@ -0,0 +1,372 @@ + + +# Apache DataFusion 40.0.0 Changelog + +This release consists of 261 commits from 64 contributors. See credits at the end of this changelog for more information. + +**Breaking changes:** + +- fix: remove the Sized requirement on ExecutionPlan::name() [#11047](https://github.com/apache/datafusion/pull/11047) (waynexia) +- Return `&Arc` reference to inner trait object [#11103](https://github.com/apache/datafusion/pull/11103) (linhr) +- remove `derive(Copy)` from `Operator` [#11132](https://github.com/apache/datafusion/pull/11132) (samuelcolvin) +- Support COPY TO Externally Defined File Formats, add FileType trait [#11060](https://github.com/apache/datafusion/pull/11060) (devinjdangelo) +- expose table name in proto extension codec [#11139](https://github.com/apache/datafusion/pull/11139) (leoyvens) +- fix(typo): unqualifed to unqualified [#11159](https://github.com/apache/datafusion/pull/11159) (waynexia) +- Consolidate `Filter::remove_aliases` into `Expr::unalias_nested` [#11001](https://github.com/apache/datafusion/pull/11001) (alamb) + +**Implemented enhancements:** + +- feat: Add support for Int8 and Int16 data types in data page statistics [#10931](https://github.com/apache/datafusion/pull/10931) (Weijun-H) +- feat: add CliSessionContext trait for cli [#10890](https://github.com/apache/datafusion/pull/10890) (tshauck) +- feat(optimizer): handle partial anchored regex cases and improve doc [#10977](https://github.com/apache/datafusion/pull/10977) (waynexia) +- feat: support uint data page extraction [#11018](https://github.com/apache/datafusion/pull/11018) (tshauck) +- feat: propagate EmptyRelation for more join types [#10963](https://github.com/apache/datafusion/pull/10963) (tshauck) +- feat: Add method to add analyzer rules to SessionContext [#10849](https://github.com/apache/datafusion/pull/10849) (pingsutw) +- feat: Support duplicate column names in Joins in Substrait consumer [#11049](https://github.com/apache/datafusion/pull/11049) (Blizzara) +- feat: Add support for Timestamp data types in data page statistics. [#11123](https://github.com/apache/datafusion/pull/11123) (efredine) +- feat: Add support for `Binary`/`LargeBinary`/`Utf8`/`LargeUtf8` data types in data page statistics [#11136](https://github.com/apache/datafusion/pull/11136) (PsiACE) +- feat: Support Map type in Substrait conversions [#11129](https://github.com/apache/datafusion/pull/11129) (Blizzara) +- feat: Conditionally allow to keep partition_by columns when using PARTITIONED BY enhancement [#11107](https://github.com/apache/datafusion/pull/11107) (hveiga) +- feat: enable "substring" as a UDF in addition to "substr" [#11277](https://github.com/apache/datafusion/pull/11277) (Blizzara) + +**Fixed bugs:** + +- fix: use total ordering in the min & max accumulator for floats [#10627](https://github.com/apache/datafusion/pull/10627) (westonpace) +- fix: Support double quotes in `date_part` [#10833](https://github.com/apache/datafusion/pull/10833) (Weijun-H) +- fix: Ignore nullability of list elements when consuming Substrait [#10874](https://github.com/apache/datafusion/pull/10874) (Blizzara) +- fix: Support `NOT IN ()` via anti join [#10936](https://github.com/apache/datafusion/pull/10936) (akoshchiy) +- fix: CTEs defined in a subquery can escape their scope [#10954](https://github.com/apache/datafusion/pull/10954) (jonahgao) +- fix: Fix the incorrect null joined rows for SMJ outer join with join filter [#10892](https://github.com/apache/datafusion/pull/10892) (viirya) +- fix: gcd returns negative results [#11099](https://github.com/apache/datafusion/pull/11099) (jonahgao) +- fix: LCM panicked due to overflow [#11131](https://github.com/apache/datafusion/pull/11131) (jonahgao) +- fix: Support dictionary type in parquet metadata statistics. [#11169](https://github.com/apache/datafusion/pull/11169) (efredine) +- fix: Ignore nullability in Substrait structs [#11130](https://github.com/apache/datafusion/pull/11130) (Blizzara) +- fix: typo in comment about FinalPhysicalPlan [#11181](https://github.com/apache/datafusion/pull/11181) (c8ef) +- fix: Support Substrait's compound names also for window functions [#11163](https://github.com/apache/datafusion/pull/11163) (Blizzara) +- fix: Incorrect LEFT JOIN evaluation result on OR conditions [#11203](https://github.com/apache/datafusion/pull/11203) (viirya) +- fix: Be more lenient in interpreting input args for builtin window functions [#11199](https://github.com/apache/datafusion/pull/11199) (Blizzara) +- fix: correctly handle Substrait windows with rows bounds (and validate executability of test plans) [#11278](https://github.com/apache/datafusion/pull/11278) (Blizzara) +- fix: When consuming Substrait, temporarily rename clashing duplicate columns [#11329](https://github.com/apache/datafusion/pull/11329) (Blizzara) + +**Documentation updates:** + +- Minor: Clarify `SessionContext::state` docs [#10847](https://github.com/apache/datafusion/pull/10847) (alamb) +- Minor: Update SIGMOD paper reference url [#10860](https://github.com/apache/datafusion/pull/10860) (alamb) +- docs(variance): Correct typos in comments [#10844](https://github.com/apache/datafusion/pull/10844) (pingsutw) +- Add missing code close tick in LiteralGuarantee docs [#10859](https://github.com/apache/datafusion/pull/10859) (adriangb) +- Minor: Add more docs and examples for `Transformed` and `TransformedResult` [#11003](https://github.com/apache/datafusion/pull/11003) (alamb) +- doc: Update links in the documantation [#11044](https://github.com/apache/datafusion/pull/11044) (Weijun-H) +- Minor: Examples cleanup + more docs in pruning example [#11086](https://github.com/apache/datafusion/pull/11086) (alamb) +- Minor: refine documentation pointing to examples [#11110](https://github.com/apache/datafusion/pull/11110) (alamb) +- Fix running in Docker instructions [#11141](https://github.com/apache/datafusion/pull/11141) (findepi) +- docs: add example for custom file format with `COPY TO` [#11174](https://github.com/apache/datafusion/pull/11174) (tshauck) +- Fix docs wordings [#11226](https://github.com/apache/datafusion/pull/11226) (findepi) +- Fix count() docs around including null values [#11293](https://github.com/apache/datafusion/pull/11293) (findepi) + +**Other:** + +- chore: Prepare 39.0.0-rc1 [#10828](https://github.com/apache/datafusion/pull/10828) (andygrove) +- Remove expr_fn::sum and replace them with function stub [#10816](https://github.com/apache/datafusion/pull/10816) (jayzhan211) +- Debug print as many fields as possible for `SessionState` [#10818](https://github.com/apache/datafusion/pull/10818) (lewiszlw) +- Prune Parquet RowGroup in a single call to `PruningPredicate::prune`, update StatisticsExtractor API [#10802](https://github.com/apache/datafusion/pull/10802) (alamb) +- Remove Built-in sum and Rename to lowercase `sum` [#10831](https://github.com/apache/datafusion/pull/10831) (jayzhan211) +- Convert `stddev` and `stddev_pop` to UDAF [#10834](https://github.com/apache/datafusion/pull/10834) (goldmedal) +- Introduce expr builder for aggregate function [#10560](https://github.com/apache/datafusion/pull/10560) (jayzhan211) +- chore: Improve change log generator [#10841](https://github.com/apache/datafusion/pull/10841) (andygrove) +- Support user defined `ParquetAccessPlan` in `ParquetExec`, validation to `ParquetAccessPlan::select` [#10813](https://github.com/apache/datafusion/pull/10813) (alamb) +- Convert `VariancePopulation` to UDAF [#10836](https://github.com/apache/datafusion/pull/10836) (mknaw) +- Convert `approx_median` to UDAF [#10840](https://github.com/apache/datafusion/pull/10840) (goldmedal) +- MINOR: use workspace deps in proto-common (upgrade object store dependency) [#10848](https://github.com/apache/datafusion/pull/10848) (waynexia) +- Minor: add `Window::try_new_with_schema` constructor [#10850](https://github.com/apache/datafusion/pull/10850) (sadboy) +- Add support for reading CSV files with comments [#10467](https://github.com/apache/datafusion/pull/10467) (bbannier) +- Convert approx_distinct to UDAF [#10851](https://github.com/apache/datafusion/pull/10851) (Lordworms) +- minor: add proto-common crate to release instructions [#10858](https://github.com/apache/datafusion/pull/10858) (andygrove) +- Implement TPCH substrait integration teset, support tpch_1 [#10842](https://github.com/apache/datafusion/pull/10842) (Lordworms) +- Remove unecessary passing around of `suffix: &str` in `pruning.rs`'s `RequiredColumns` [#10863](https://github.com/apache/datafusion/pull/10863) (adriangb) +- chore: Make DFSchema::datatype_is_logically_equal function public [#10867](https://github.com/apache/datafusion/pull/10867) (advancedxy) +- Bump braces from 3.0.2 to 3.0.3 in /datafusion/wasmtest/datafusion-wasm-app [#10865](https://github.com/apache/datafusion/pull/10865) (dependabot[bot]) +- Docs: Add `unnest` to SQL Reference [#10839](https://github.com/apache/datafusion/pull/10839) (gloomweaver) +- Support correct output column names and struct field names when consuming/producing Substrait [#10829](https://github.com/apache/datafusion/pull/10829) (Blizzara) +- Make Logical Plans more readable by removing extra aliases [#10832](https://github.com/apache/datafusion/pull/10832) (MohamedAbdeen21) +- Minor: Improve `ListingTable` documentation [#10854](https://github.com/apache/datafusion/pull/10854) (alamb) +- Extending join fuzz tests to support join filtering [#10728](https://github.com/apache/datafusion/pull/10728) (edmondop) +- replace and(_, not(_)) with and_not(\*) [#10885](https://github.com/apache/datafusion/pull/10885) (RTEnzyme) +- Disabling test for semi join with filters [#10887](https://github.com/apache/datafusion/pull/10887) (edmondop) +- Minor: Update `min_statistics` and `max_statistics` to be helpers, update docs [#10866](https://github.com/apache/datafusion/pull/10866) (alamb) +- Remove `Interval` column test // parquet extraction [#10888](https://github.com/apache/datafusion/pull/10888) (marvinlanhenke) +- Minor: SMJ fuzz tests fix for rowcounts [#10891](https://github.com/apache/datafusion/pull/10891) (comphead) +- Move `Count` to `functions-aggregate`, update MSRV to rust 1.75 [#10484](https://github.com/apache/datafusion/pull/10484) (jayzhan211) +- refactor: fetch statistics for a given ParquetMetaData [#10880](https://github.com/apache/datafusion/pull/10880) (NGA-TRAN) +- Move FileSinkExec::metrics to the correct place [#10901](https://github.com/apache/datafusion/pull/10901) (joroKr21) +- Refine ParquetAccessPlan comments and tests [#10896](https://github.com/apache/datafusion/pull/10896) (alamb) +- ci: fix clippy failures on main [#10903](https://github.com/apache/datafusion/pull/10903) (jonahgao) +- Minor: disable flaky fuzz test [#10904](https://github.com/apache/datafusion/pull/10904) (comphead) +- Remove builtin count [#10893](https://github.com/apache/datafusion/pull/10893) (jayzhan211) +- Move Regr\_\* functions to use UDAF [#10898](https://github.com/apache/datafusion/pull/10898) (eejbyfeldt) +- Docs: clarify when the parquet reader will read from object store when using cached metadata [#10909](https://github.com/apache/datafusion/pull/10909) (alamb) +- Minor: Fix `bench.sh tpch data` [#10905](https://github.com/apache/datafusion/pull/10905) (alamb) +- Minor: use venv in benchmark compare [#10894](https://github.com/apache/datafusion/pull/10894) (tmi) +- Support explicit type and name during table creation [#10273](https://github.com/apache/datafusion/pull/10273) (duongcongtoai) +- Simplify Join Partition Rules [#10911](https://github.com/apache/datafusion/pull/10911) (berkaysynnada) +- Move `Literal` to `physical-expr-common` [#10910](https://github.com/apache/datafusion/pull/10910) (lewiszlw) +- chore: update some error messages for clarity [#10916](https://github.com/apache/datafusion/pull/10916) (jeffreyssmith2nd) +- Initial Extract parquet data page statistics API [#10852](https://github.com/apache/datafusion/pull/10852) (marvinlanhenke) +- Add contains function, and support in datafusion substrait consumer [#10879](https://github.com/apache/datafusion/pull/10879) (Lordworms) +- Minor: Improve `arrow_statistics` tests [#10927](https://github.com/apache/datafusion/pull/10927) (alamb) +- Minor: Remove `prefer_hash_join` env variable for clickbench [#10933](https://github.com/apache/datafusion/pull/10933) (jayzhan211) +- Convert ApproxPercentileCont and ApproxPercentileContWithWeight to UDAF [#10917](https://github.com/apache/datafusion/pull/10917) (goldmedal) +- refactor: remove extra default in max rows [#10941](https://github.com/apache/datafusion/pull/10941) (tshauck) +- chore: Improve performance of Parquet statistics conversion [#10932](https://github.com/apache/datafusion/pull/10932) (Weijun-H) +- Add catalog::resolve_table_references [#10876](https://github.com/apache/datafusion/pull/10876) (leoyvens) +- Convert BitAnd, BitOr, BitXor to UDAF [#10930](https://github.com/apache/datafusion/pull/10930) (dharanad) +- refactor: improve PoolType argument handling for CLI [#10940](https://github.com/apache/datafusion/pull/10940) (tshauck) +- Minor: remove potential string copy from Column::from_qualified_name [#10947](https://github.com/apache/datafusion/pull/10947) (alamb) +- Fix: StatisticsConverter `counts` for missing columns [#10946](https://github.com/apache/datafusion/pull/10946) (marvinlanhenke) +- Add initial support for Utf8View and BinaryView types [#10925](https://github.com/apache/datafusion/pull/10925) (XiangpengHao) +- Use shorter aliases in CSE [#10939](https://github.com/apache/datafusion/pull/10939) (peter-toth) +- Substrait support for ParquetExec round trip for simple select [#10949](https://github.com/apache/datafusion/pull/10949) (xinlifoobar) +- Support to unparse `ScalarValue::IntervalMonthDayNano` to String [#10956](https://github.com/apache/datafusion/pull/10956) (goldmedal) +- Convert `StringAgg` to UDAF [#10945](https://github.com/apache/datafusion/pull/10945) (lewiszlw) +- Minor: Return option from row_group_row_count [#10973](https://github.com/apache/datafusion/pull/10973) (marvinlanhenke) +- Minor: Add routine to debug join fuzz tests [#10970](https://github.com/apache/datafusion/pull/10970) (comphead) +- Support to unparse `ScalarValue::TimestampNanosecond` to String [#10984](https://github.com/apache/datafusion/pull/10984) (goldmedal) +- build(deps-dev): bump ws from 8.14.2 to 8.17.1 in /datafusion/wasmtest/datafusion-wasm-app [#10988](https://github.com/apache/datafusion/pull/10988) (dependabot[bot]) +- Minor: reuse Rows buffer in GroupValuesRows [#10980](https://github.com/apache/datafusion/pull/10980) (alamb) +- Add example for writing SQL analysis using DataFusion structures [#10938](https://github.com/apache/datafusion/pull/10938) (LorrensP-2158466) +- Push down filter for Unnest plan [#10974](https://github.com/apache/datafusion/pull/10974) (jayzhan211) +- Add parquet page stats for float{16, 32, 64} [#10982](https://github.com/apache/datafusion/pull/10982) (tmi) +- Fix `file_stream_provider` example compilation failure on windows [#10975](https://github.com/apache/datafusion/pull/10975) (lewiszlw) +- Stop copying LogicalPlan and Exprs in `CommonSubexprEliminate` (2-3% planning speed improvement) [#10835](https://github.com/apache/datafusion/pull/10835) (alamb) +- chore: Update documentation link in `PhysicalOptimizerRule` comment [#11002](https://github.com/apache/datafusion/pull/11002) (Weijun-H) +- Push down filter plan for unnest on non-unnest column only [#10991](https://github.com/apache/datafusion/pull/10991) (jayzhan211) +- Minor: add test for pushdown past unnest [#11017](https://github.com/apache/datafusion/pull/11017) (alamb) +- Update docs for `protoc` minimum installed version [#11006](https://github.com/apache/datafusion/pull/11006) (jcsherin) +- Convert `bool_and` & `bool_or` to UDAF [#11009](https://github.com/apache/datafusion/pull/11009) (jcsherin) +- propagate error instead of panicking on out of bounds in physical-expr/src/analysis.rs [#10992](https://github.com/apache/datafusion/pull/10992) (LorrensP-2158466) +- Add drop_columns to dataframe api [#11010](https://github.com/apache/datafusion/pull/11010) (Omega359) +- Push down filter plan for non-unnest column [#11019](https://github.com/apache/datafusion/pull/11019) (jayzhan211) +- Consider timezones with `UTC` and `+00:00` to be the same [#10960](https://github.com/apache/datafusion/pull/10960) (marvinlanhenke) +- Deprecate `OptimizerRule::try_optimize` [#11022](https://github.com/apache/datafusion/pull/11022) (lewiszlw) +- Relax combine partial final rule [#10913](https://github.com/apache/datafusion/pull/10913) (mustafasrepo) +- Compute gcd with u64 instead of i64 because of overflows [#11036](https://github.com/apache/datafusion/pull/11036) (LorrensP-2158466) +- Add distinct_on to dataframe api [#11012](https://github.com/apache/datafusion/pull/11012) (Omega359) +- chore: add test to show current behavior of `AT TIME ZONE` for string vs. timestamp [#11056](https://github.com/apache/datafusion/pull/11056) (appletreeisyellow) +- Boolean parquet get datapage stat [#11054](https://github.com/apache/datafusion/pull/11054) (LorrensP-2158466) +- Using display_name for Expr::Aggregation [#11020](https://github.com/apache/datafusion/pull/11020) (Lordworms) +- Minor: Convert `Count`'s name to lowercase [#11028](https://github.com/apache/datafusion/pull/11028) (jayzhan211) +- Minor: Move `function::Hint` to `datafusion-expr` crate to avoid physical-expr dependency for `datafusion-function` crate [#11061](https://github.com/apache/datafusion/pull/11061) (jayzhan211) +- Support to unparse ScalarValue::TimestampMillisecond to String [#11046](https://github.com/apache/datafusion/pull/11046) (pingsutw) +- Support to unparse IntervalYearMonth and IntervalDayTime to String [#11065](https://github.com/apache/datafusion/pull/11065) (goldmedal) +- SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join [#11041](https://github.com/apache/datafusion/pull/11041) (comphead) +- Add `advanced_parquet_index.rs` example of index in into parquet files [#10701](https://github.com/apache/datafusion/pull/10701) (alamb) +- Add Expr::column_refs to find column references without copying [#10948](https://github.com/apache/datafusion/pull/10948) (alamb) +- Give `OptimizerRule::try_optimize` default implementation and cleanup duplicated custom implementations [#11059](https://github.com/apache/datafusion/pull/11059) (lewiszlw) +- Fix `FormatOptions::CSV` propagation [#10912](https://github.com/apache/datafusion/pull/10912) (svranesevic) +- Support parsing SQL strings to Exprs [#10995](https://github.com/apache/datafusion/pull/10995) (xinlifoobar) +- Support dictionary data type in array_to_string [#10908](https://github.com/apache/datafusion/pull/10908) (EduardoVega) +- Implement min/max for interval types [#11015](https://github.com/apache/datafusion/pull/11015) (maxburke) +- Convert Average to UDAF #10942 [#10964](https://github.com/apache/datafusion/pull/10964) (dharanad) +- Improve LIKE performance for Dictionary arrays [#11058](https://github.com/apache/datafusion/pull/11058) (Lordworms) +- handle overflow in gcd and return this as an error [#11057](https://github.com/apache/datafusion/pull/11057) (LorrensP-2158466) +- Convert Correlation to UDAF [#11064](https://github.com/apache/datafusion/pull/11064) (pingsutw) +- Migrate more code from `Expr::to_columns` to `Expr::column_refs` [#11067](https://github.com/apache/datafusion/pull/11067) (alamb) +- decimal support for unparser [#11092](https://github.com/apache/datafusion/pull/11092) (y-f-u) +- Improve `CommonSubexprEliminate` identifier management (10% faster planning) [#10473](https://github.com/apache/datafusion/pull/10473) (peter-toth) +- Change wildcard qualifier type from `String` to `TableReference` [#11073](https://github.com/apache/datafusion/pull/11073) (linhr) +- Allow access to UDTF in `SessionContext` [#11071](https://github.com/apache/datafusion/pull/11071) (linhr) +- Strip table qualifiers from schema in `UNION ALL` for unparser [#11082](https://github.com/apache/datafusion/pull/11082) (phillipleblanc) +- Update ListingTable to use StatisticsConverter [#11068](https://github.com/apache/datafusion/pull/11068) (xinlifoobar) +- to_timestamp functions should preserve timezone [#11038](https://github.com/apache/datafusion/pull/11038) (maxburke) +- Rewrite array operator to function in parser [#11101](https://github.com/apache/datafusion/pull/11101) (jayzhan211) +- Resolve empty relation opt for join types [#11066](https://github.com/apache/datafusion/pull/11066) (LorrensP-2158466) +- Add composed extension codec example [#11095](https://github.com/apache/datafusion/pull/11095) (lewiszlw) +- Minor: Avoid some repetition in to_timestamp [#11116](https://github.com/apache/datafusion/pull/11116) (alamb) +- Minor: fix ScalarValue::new_ten error message (cites one not ten) [#11126](https://github.com/apache/datafusion/pull/11126) (gstvg) +- Deprecate Expr::column_refs [#11115](https://github.com/apache/datafusion/pull/11115) (alamb) +- Overflow in negate operator [#11084](https://github.com/apache/datafusion/pull/11084) (LorrensP-2158466) +- Minor: Add Architectural Goals to the docs [#11109](https://github.com/apache/datafusion/pull/11109) (alamb) +- Fix overflow in pow [#11124](https://github.com/apache/datafusion/pull/11124) (LorrensP-2158466) +- Support to unparse Time scalar value to String [#11121](https://github.com/apache/datafusion/pull/11121) (goldmedal) +- Support to unparse `TimestampSecond` and `TimestampMicrosecond` to String [#11120](https://github.com/apache/datafusion/pull/11120) (goldmedal) +- Add standalone example for `OptimizerRule` [#11087](https://github.com/apache/datafusion/pull/11087) (alamb) +- Fix overflow in factorial [#11134](https://github.com/apache/datafusion/pull/11134) (LorrensP-2158466) +- Temporary Fix: Query error when grouping by case expressions [#11133](https://github.com/apache/datafusion/pull/11133) (jonahgao) +- Fix nullability of return value of array_agg [#11093](https://github.com/apache/datafusion/pull/11093) (eejbyfeldt) +- Support filter for List [#11091](https://github.com/apache/datafusion/pull/11091) (jayzhan211) +- [MINOR]: Fix some minor silent bugs [#11127](https://github.com/apache/datafusion/pull/11127) (mustafasrepo) +- Minor Fix for Logical and Physical Expr Conversions [#11142](https://github.com/apache/datafusion/pull/11142) (berkaysynnada) +- Support Date Parquet Data Page Statistics [#11135](https://github.com/apache/datafusion/pull/11135) (dharanad) +- fix flaky array query slt test [#11140](https://github.com/apache/datafusion/pull/11140) (leoyvens) +- Support Decimal and Decimal256 Parquet Data Page Statistics [#11138](https://github.com/apache/datafusion/pull/11138) (Lordworms) +- Implement comparisons on nested data types such that distinct/except would work [#11117](https://github.com/apache/datafusion/pull/11117) (rtyler) +- Minor: dont panic with bad arguments to round [#10899](https://github.com/apache/datafusion/pull/10899) (tmi) +- Minor: reduce replication for nested comparison [#11149](https://github.com/apache/datafusion/pull/11149) (alamb) +- [Minor]: Remove datafusion-functions-aggregate dependency from physical-expr crate [#11158](https://github.com/apache/datafusion/pull/11158) (mustafasrepo) +- adding config to control Varchar behavior [#11090](https://github.com/apache/datafusion/pull/11090) (Lordworms) +- minor: consolidate `gcd` related tests [#11164](https://github.com/apache/datafusion/pull/11164) (jonahgao) +- Minor: move batch spilling methods to `lib.rs` to make it reusable [#11154](https://github.com/apache/datafusion/pull/11154) (comphead) +- Move schema projection to where it's used in ListingTable [#11167](https://github.com/apache/datafusion/pull/11167) (adriangb) +- Make running in docker instruction be copy-pastable [#11148](https://github.com/apache/datafusion/pull/11148) (findepi) +- Rewrite `array @> array` and `array <@ array` in sql_expr_to_logical_expr [#11155](https://github.com/apache/datafusion/pull/11155) (jayzhan211) +- Minor: make some physical_optimizer rules public [#11171](https://github.com/apache/datafusion/pull/11171) (askalt) +- Remove pr_benchmarks.yml [#11165](https://github.com/apache/datafusion/pull/11165) (alamb) +- Optionally display schema in explain plan [#11177](https://github.com/apache/datafusion/pull/11177) (alamb) +- Minor: Add more support for ScalarValue::Float16 [#11156](https://github.com/apache/datafusion/pull/11156) (Lordworms) +- Minor: fix SQLOptions::with_allow_ddl comments [#11166](https://github.com/apache/datafusion/pull/11166) (alamb) +- Update sqllogictest requirement from 0.20.0 to 0.21.0 [#11189](https://github.com/apache/datafusion/pull/11189) (dependabot[bot]) +- Support Time Parquet Data Page Statistics [#11187](https://github.com/apache/datafusion/pull/11187) (dharanad) +- Adds support for Dictionary data type statistics from parquet data pages. [#11195](https://github.com/apache/datafusion/pull/11195) (efredine) +- [Minor]: Make sort_batch public [#11191](https://github.com/apache/datafusion/pull/11191) (mustafasrepo) +- Introduce user defined SQL planner API [#11180](https://github.com/apache/datafusion/pull/11180) (jayzhan211) +- Covert grouping to udaf [#11147](https://github.com/apache/datafusion/pull/11147) (Rachelint) +- Make statistics_from_parquet_meta a sync function [#11205](https://github.com/apache/datafusion/pull/11205) (adriangb) +- Allow user defined SQL planners to be registered [#11208](https://github.com/apache/datafusion/pull/11208) (samuelcolvin) +- Recursive `unnest` [#11062](https://github.com/apache/datafusion/pull/11062) (duongcongtoai) +- Document how to test examples in user guide, add some more coverage [#11178](https://github.com/apache/datafusion/pull/11178) (alamb) +- Minor: Move MemoryCatalog\*Provider into a module, improve comments [#11183](https://github.com/apache/datafusion/pull/11183) (alamb) +- Add standalone example of using the SQL frontend [#11088](https://github.com/apache/datafusion/pull/11088) (alamb) +- Add Optimizer Sanity Checker, improve sortedness equivalence properties [#11196](https://github.com/apache/datafusion/pull/11196) (mustafasrepo) +- Implement user defined planner for extract [#11215](https://github.com/apache/datafusion/pull/11215) (xinlifoobar) +- Move basic SQL query examples to user guide [#11217](https://github.com/apache/datafusion/pull/11217) (alamb) +- Support FixedSizedBinaryArray Parquet Data Page Statistics [#11200](https://github.com/apache/datafusion/pull/11200) (dharanad) +- Implement ScalarValue::Map [#11224](https://github.com/apache/datafusion/pull/11224) (goldmedal) +- Remove unmaintained python pre-commit configuration [#11255](https://github.com/apache/datafusion/pull/11255) (findepi) +- Enable `clone_on_ref_ptr` clippy lint on execution crate [#11239](https://github.com/apache/datafusion/pull/11239) (lewiszlw) +- Minor: Improve documentation about pushdown join predicates [#11209](https://github.com/apache/datafusion/pull/11209) (alamb) +- Minor: clean up data page statistics tests and fix bugs [#11236](https://github.com/apache/datafusion/pull/11236) (efredine) +- Replacing pattern matching through downcast with trait method [#11257](https://github.com/apache/datafusion/pull/11257) (edmondop) +- Update substrait requirement from 0.34.0 to 0.35.0 [#11206](https://github.com/apache/datafusion/pull/11206) (dependabot[bot]) +- Enhance short circuit handling in `CommonSubexprEliminate` [#11197](https://github.com/apache/datafusion/pull/11197) (peter-toth) +- Add bench for data page statistics parquet extraction [#10950](https://github.com/apache/datafusion/pull/10950) (marvinlanhenke) +- Register SQL planners in `SessionState` constructor [#11253](https://github.com/apache/datafusion/pull/11253) (dharanad) +- Support DuckDB style struct syntax [#11214](https://github.com/apache/datafusion/pull/11214) (jayzhan211) +- Enable `clone_on_ref_ptr` clippy lint on expr crate [#11238](https://github.com/apache/datafusion/pull/11238) (lewiszlw) +- Optimize PushDownFilter to avoid recreating schema columns [#11211](https://github.com/apache/datafusion/pull/11211) (alamb) +- Remove outdated `rewrite_expr.rs` example [#11085](https://github.com/apache/datafusion/pull/11085) (alamb) +- Implement TPCH substrait integration teset, support tpch_2 [#11234](https://github.com/apache/datafusion/pull/11234) (Lordworms) +- Enable `clone_on_ref_ptr` clippy lint on physical-expr crate [#11240](https://github.com/apache/datafusion/pull/11240) (lewiszlw) +- Add standalone `AnalyzerRule` example that implements row level access control [#11089](https://github.com/apache/datafusion/pull/11089) (alamb) +- Replace println! with assert! if possible in DataFusion examples [#11237](https://github.com/apache/datafusion/pull/11237) (Nishi46) +- minor: format `Expr::get_type()` [#11267](https://github.com/apache/datafusion/pull/11267) (jonahgao) +- Fix hash join for nested types [#11232](https://github.com/apache/datafusion/pull/11232) (eejbyfeldt) +- Infer count() aggregation is not null [#11256](https://github.com/apache/datafusion/pull/11256) (findepi) +- Remove unnecessary qualified names [#11292](https://github.com/apache/datafusion/pull/11292) (findepi) +- Fix running examples readme [#11225](https://github.com/apache/datafusion/pull/11225) (findepi) +- Minor: Add `ConstExpr::from` and use in physical optimizer [#11283](https://github.com/apache/datafusion/pull/11283) (alamb) +- Implement TPCH substrait integration teset, support tpch_3 [#11298](https://github.com/apache/datafusion/pull/11298) (Lordworms) +- Implement user defined planner for position [#11243](https://github.com/apache/datafusion/pull/11243) (xinlifoobar) +- Upgrade to arrow 52.1.0 (and fix clippy issues on main) [#11302](https://github.com/apache/datafusion/pull/11302) (alamb) +- AggregateExec: Take grouping sets into account for InputOrderMode [#11301](https://github.com/apache/datafusion/pull/11301) (thinkharderdev) +- Add user_defined_sql_planners(..) to FunctionRegistry [#11296](https://github.com/apache/datafusion/pull/11296) (Omega359) +- use safe cast in propagate_constraints [#11297](https://github.com/apache/datafusion/pull/11297) (Lordworms) +- Minor: Remove clone in optimizer [#11315](https://github.com/apache/datafusion/pull/11315) (jayzhan211) +- minor: Add `PhysicalSortExpr::new` [#11310](https://github.com/apache/datafusion/pull/11310) (andygrove) +- Fix data page statistics when all rows are null in a data page [#11295](https://github.com/apache/datafusion/pull/11295) (efredine) +- Made UserDefinedFunctionPlanner to uniform the usages [#11318](https://github.com/apache/datafusion/pull/11318) (xinlifoobar) +- Implement user defined planner for `create_struct` & `create_named_struct` [#11273](https://github.com/apache/datafusion/pull/11273) (dharanad) +- Improve stats convert performance for Binary/String/Boolean arrays [#11319](https://github.com/apache/datafusion/pull/11319) (Rachelint) +- Fix typos in datafusion-examples/datafusion-cli/docs [#11259](https://github.com/apache/datafusion/pull/11259) (lewiszlw) +- Minor: Fix Failing TPC-DS Test [#11331](https://github.com/apache/datafusion/pull/11331) (berkaysynnada) +- HashJoin can preserve the right ordering when join type is Right [#11276](https://github.com/apache/datafusion/pull/11276) (berkaysynnada) +- Update substrait requirement from 0.35.0 to 0.36.0 [#11328](https://github.com/apache/datafusion/pull/11328) (dependabot[bot]) +- Support to uparse logical plans with timestamp cast to string [#11326](https://github.com/apache/datafusion/pull/11326) (sgrebnov) +- Implement user defined planner for sql_substring_to_expr [#11327](https://github.com/apache/datafusion/pull/11327) (xinlifoobar) +- Improve volatile expression handling in `CommonSubexprEliminate` [#11265](https://github.com/apache/datafusion/pull/11265) (peter-toth) +- Support `IS NULL` and `IS NOT NULL` on Unions [#11321](https://github.com/apache/datafusion/pull/11321) (samuelcolvin) +- Convert `nth_value` to UDAF [#11287](https://github.com/apache/datafusion/pull/11287) (jcsherin) +- Implement TPCH substrait integration test, support tpch_4 and tpch_5 [#11311](https://github.com/apache/datafusion/pull/11311) (Lordworms) +- Enable `clone_on_ref_ptr` clippy lint on physical-plan crate [#11241](https://github.com/apache/datafusion/pull/11241) (lewiszlw) +- Remove any aliases in `Filter::try_new` rather than erroring [#11307](https://github.com/apache/datafusion/pull/11307) (samuelcolvin) +- Improve `DataFrame` Users Guide [#11324](https://github.com/apache/datafusion/pull/11324) (alamb) +- chore: Rename UserDefinedSQLPlanner to ExprPlanner [#11338](https://github.com/apache/datafusion/pull/11338) (andygrove) +- Revert "remove `derive(Copy)` from `Operator` (#11132)" [#11341](https://github.com/apache/datafusion/pull/11341) (alamb) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 40 Andrew Lamb + 17 Jay Zhan + 12 Lordworms + 12 张林伟 + 10 Arttu + 9 Jax Liu + 9 Lorrens Pantelis + 8 Piotr Findeisen + 7 Dharan Aditya + 7 Jonah Gao + 7 Xin Li + 6 Marvin Lanhenke + 6 Trent Hauck + 5 Alex Huang + 5 Andy Grove + 5 Eric Fredine + 5 Mustafa Akur + 5 Oleks V + 5 dependabot[bot] + 4 Adrian Garcia Badaracco + 4 Berkay Åžahin + 4 Kevin Su + 4 Peter Toth + 4 Ruihang Xia + 4 Samuel Colvin + 3 Bruce Ritchie + 3 Edmondo Porcu + 3 Emil Ejbyfeldt + 3 Heran Lin + 3 Leonardo Yvens + 3 jcsherin + 3 tmi + 2 Duong Cong Toai + 2 Liang-Chi Hsieh + 2 Max Burke + 2 kamille + 1 Albert Skalt + 1 Andrey Koshchiy + 1 Benjamin Bannier + 1 Bo Lin + 1 Chojan Shang + 1 Chunchun Ye + 1 Dan Harris + 1 Devin D'Angelo + 1 Eduardo Vega + 1 Georgi Krastev + 1 Hector Veiga + 1 Jeffrey Smith II + 1 Kirill Khramkov + 1 Matt Nawara + 1 Mohamed Abdeen + 1 Nga Tran + 1 Nishi + 1 Phillip LeBlanc + 1 R. Tyler Croy + 1 RT_Enzyme + 1 Sava VraneÅ¡ević + 1 Sergei Grebnov + 1 Weston Pace + 1 Xiangpeng Hao + 1 advancedxy + 1 c8ef + 1 gstvg + 1 yfu +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 303caef57700..579088f991ef 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 39.0.0 | Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 40.0.0 | Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | From 43fb645dda9534b0e7cad1a29d3fc307150e9fa0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 8 Jul 2024 19:48:57 -0600 Subject: [PATCH 08/10] regenerate changelog --- dev/changelog/40.0.0.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dev/changelog/40.0.0.md b/dev/changelog/40.0.0.md index 163f8ea876a6..30d4cd5d16fb 100644 --- a/dev/changelog/40.0.0.md +++ b/dev/changelog/40.0.0.md @@ -19,10 +19,13 @@ under the License. # Apache DataFusion 40.0.0 Changelog -This release consists of 261 commits from 64 contributors. See credits at the end of this changelog for more information. +This release consists of 263 commits from 64 contributors. See credits at the end of this changelog for more information. **Breaking changes:** +- Convert `StringAgg` to UDAF [#10945](https://github.com/apache/datafusion/pull/10945) (lewiszlw) +- Convert `bool_and` & `bool_or` to UDAF [#11009](https://github.com/apache/datafusion/pull/11009) (jcsherin) +- Convert Average to UDAF #10942 [#10964](https://github.com/apache/datafusion/pull/10964) (dharanad) - fix: remove the Sized requirement on ExecutionPlan::name() [#11047](https://github.com/apache/datafusion/pull/11047) (waynexia) - Return `&Arc` reference to inner trait object [#11103](https://github.com/apache/datafusion/pull/11103) (linhr) - remove `derive(Copy)` from `Operator` [#11132](https://github.com/apache/datafusion/pull/11132) (samuelcolvin) @@ -30,6 +33,7 @@ This release consists of 261 commits from 64 contributors. See credits at the en - expose table name in proto extension codec [#11139](https://github.com/apache/datafusion/pull/11139) (leoyvens) - fix(typo): unqualifed to unqualified [#11159](https://github.com/apache/datafusion/pull/11159) (waynexia) - Consolidate `Filter::remove_aliases` into `Expr::unalias_nested` [#11001](https://github.com/apache/datafusion/pull/11001) (alamb) +- Convert `nth_value` to UDAF [#11287](https://github.com/apache/datafusion/pull/11287) (jcsherin) **Implemented enhancements:** @@ -143,7 +147,6 @@ This release consists of 261 commits from 64 contributors. See credits at the en - Use shorter aliases in CSE [#10939](https://github.com/apache/datafusion/pull/10939) (peter-toth) - Substrait support for ParquetExec round trip for simple select [#10949](https://github.com/apache/datafusion/pull/10949) (xinlifoobar) - Support to unparse `ScalarValue::IntervalMonthDayNano` to String [#10956](https://github.com/apache/datafusion/pull/10956) (goldmedal) -- Convert `StringAgg` to UDAF [#10945](https://github.com/apache/datafusion/pull/10945) (lewiszlw) - Minor: Return option from row_group_row_count [#10973](https://github.com/apache/datafusion/pull/10973) (marvinlanhenke) - Minor: Add routine to debug join fuzz tests [#10970](https://github.com/apache/datafusion/pull/10970) (comphead) - Support to unparse `ScalarValue::TimestampNanosecond` to String [#10984](https://github.com/apache/datafusion/pull/10984) (goldmedal) @@ -158,7 +161,6 @@ This release consists of 261 commits from 64 contributors. See credits at the en - Push down filter plan for unnest on non-unnest column only [#10991](https://github.com/apache/datafusion/pull/10991) (jayzhan211) - Minor: add test for pushdown past unnest [#11017](https://github.com/apache/datafusion/pull/11017) (alamb) - Update docs for `protoc` minimum installed version [#11006](https://github.com/apache/datafusion/pull/11006) (jcsherin) -- Convert `bool_and` & `bool_or` to UDAF [#11009](https://github.com/apache/datafusion/pull/11009) (jcsherin) - propagate error instead of panicking on out of bounds in physical-expr/src/analysis.rs [#10992](https://github.com/apache/datafusion/pull/10992) (LorrensP-2158466) - Add drop_columns to dataframe api [#11010](https://github.com/apache/datafusion/pull/11010) (Omega359) - Push down filter plan for non-unnest column [#11019](https://github.com/apache/datafusion/pull/11019) (jayzhan211) @@ -182,7 +184,6 @@ This release consists of 261 commits from 64 contributors. See credits at the en - Support parsing SQL strings to Exprs [#10995](https://github.com/apache/datafusion/pull/10995) (xinlifoobar) - Support dictionary data type in array_to_string [#10908](https://github.com/apache/datafusion/pull/10908) (EduardoVega) - Implement min/max for interval types [#11015](https://github.com/apache/datafusion/pull/11015) (maxburke) -- Convert Average to UDAF #10942 [#10964](https://github.com/apache/datafusion/pull/10964) (dharanad) - Improve LIKE performance for Dictionary arrays [#11058](https://github.com/apache/datafusion/pull/11058) (Lordworms) - handle overflow in gcd and return this as an error [#11057](https://github.com/apache/datafusion/pull/11057) (LorrensP-2158466) - Convert Correlation to UDAF [#11064](https://github.com/apache/datafusion/pull/11064) (pingsutw) @@ -290,7 +291,6 @@ This release consists of 261 commits from 64 contributors. See credits at the en - Implement user defined planner for sql_substring_to_expr [#11327](https://github.com/apache/datafusion/pull/11327) (xinlifoobar) - Improve volatile expression handling in `CommonSubexprEliminate` [#11265](https://github.com/apache/datafusion/pull/11265) (peter-toth) - Support `IS NULL` and `IS NOT NULL` on Unions [#11321](https://github.com/apache/datafusion/pull/11321) (samuelcolvin) -- Convert `nth_value` to UDAF [#11287](https://github.com/apache/datafusion/pull/11287) (jcsherin) - Implement TPCH substrait integration test, support tpch_4 and tpch_5 [#11311](https://github.com/apache/datafusion/pull/11311) (Lordworms) - Enable `clone_on_ref_ptr` clippy lint on physical-plan crate [#11241](https://github.com/apache/datafusion/pull/11241) (lewiszlw) - Remove any aliases in `Filter::try_new` rather than erroring [#11307](https://github.com/apache/datafusion/pull/11307) (samuelcolvin) @@ -303,7 +303,7 @@ This release consists of 261 commits from 64 contributors. See credits at the en Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. ``` - 40 Andrew Lamb + 41 Andrew Lamb 17 Jay Zhan 12 Lordworms 12 张林伟 @@ -314,10 +314,10 @@ Thank you to everyone who contributed to this release. Here is a breakdown of co 7 Dharan Aditya 7 Jonah Gao 7 Xin Li + 6 Andy Grove 6 Marvin Lanhenke 6 Trent Hauck 5 Alex Huang - 5 Andy Grove 5 Eric Fredine 5 Mustafa Akur 5 Oleks V From 4cae81363e29f011c6602a7a7a54e1aaee841046 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 8 Jul 2024 19:52:06 -0600 Subject: [PATCH 09/10] manually remove a reverted PR from the breaking change section --- dev/changelog/40.0.0.md | 1 - 1 file changed, 1 deletion(-) diff --git a/dev/changelog/40.0.0.md b/dev/changelog/40.0.0.md index 30d4cd5d16fb..72143ae48b28 100644 --- a/dev/changelog/40.0.0.md +++ b/dev/changelog/40.0.0.md @@ -28,7 +28,6 @@ This release consists of 263 commits from 64 contributors. See credits at the en - Convert Average to UDAF #10942 [#10964](https://github.com/apache/datafusion/pull/10964) (dharanad) - fix: remove the Sized requirement on ExecutionPlan::name() [#11047](https://github.com/apache/datafusion/pull/11047) (waynexia) - Return `&Arc` reference to inner trait object [#11103](https://github.com/apache/datafusion/pull/11103) (linhr) -- remove `derive(Copy)` from `Operator` [#11132](https://github.com/apache/datafusion/pull/11132) (samuelcolvin) - Support COPY TO Externally Defined File Formats, add FileType trait [#11060](https://github.com/apache/datafusion/pull/11060) (devinjdangelo) - expose table name in proto extension codec [#11139](https://github.com/apache/datafusion/pull/11139) (leoyvens) - fix(typo): unqualifed to unqualified [#11159](https://github.com/apache/datafusion/pull/11159) (waynexia) From 5be9029e0148d4742072399bf3ac0a928c389d20 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Wed, 17 Jul 2024 14:13:26 +0200 Subject: [PATCH 10/10] Rename types to reflect proposal --- .../examples/advanced_parquet_index.rs | 6 +- .../examples/custom_datasource.rs | 12 +- datafusion-examples/examples/expr_api.rs | 16 +-- .../examples/function_factory.rs | 2 +- datafusion-examples/examples/logical_type.rs | 20 +-- datafusion-examples/examples/parquet_index.rs | 6 +- datafusion-examples/examples/simple_udtf.rs | 6 +- datafusion-examples/examples/sql_frontend.rs | 18 +-- datafusion/common/src/column.rs | 10 +- datafusion/common/src/dfschema.rs | 118 +++++++++--------- datafusion/common/src/logical_type/field.rs | 38 +++--- datafusion/common/src/logical_type/fields.rs | 60 ++++----- datafusion/common/src/logical_type/mod.rs | 48 +++---- datafusion/common/src/logical_type/schema.rs | 72 +++++------ .../common/src/logical_type/signature.rs | 10 +- datafusion/common/src/param_value.rs | 4 +- datafusion/core/src/dataframe/mod.rs | 6 +- .../core/src/datasource/cte_worktable.rs | 6 +- .../src/datasource/default_table_source.rs | 4 +- datafusion/core/src/datasource/empty.rs | 6 +- .../core/src/datasource/listing/helpers.rs | 4 +- .../core/src/datasource/listing/table.rs | 8 +- datafusion/core/src/datasource/memory.rs | 6 +- datafusion/core/src/datasource/provider.rs | 4 +- datafusion/core/src/datasource/stream.rs | 6 +- datafusion/core/src/datasource/streaming.rs | 6 +- datafusion/core/src/datasource/view.rs | 6 +- .../core/src/execution/session_state.rs | 6 +- datafusion/core/src/physical_planner.rs | 4 +- datafusion/core/src/test/variable.rs | 6 +- datafusion/core/src/test_util/mod.rs | 6 +- datafusion/core/src/test_util/parquet.rs | 4 +- .../core/tests/custom_sources_cases/mod.rs | 6 +- .../provider_filter_pushdown.rs | 6 +- .../tests/custom_sources_cases/statistics.rs | 6 +- datafusion/core/tests/dataframe/mod.rs | 4 +- .../core/tests/expr_api/parse_sql_expr.rs | 12 +- .../core/tests/expr_api/simplification.rs | 34 ++--- datafusion/core/tests/memory_limit/mod.rs | 6 +- .../core/tests/optimizer_integration.rs | 12 +- datafusion/core/tests/parquet/page_pruning.rs | 4 +- datafusion/core/tests/sql/create_drop.rs | 2 +- .../user_defined_scalar_functions.rs | 2 +- .../user_defined_table_functions.rs | 6 +- .../expr/src/conditional_expressions.rs | 6 +- datafusion/expr/src/expr.rs | 24 ++-- datafusion/expr/src/expr_fn.rs | 8 +- datafusion/expr/src/expr_rewriter/mod.rs | 8 +- datafusion/expr/src/expr_schema.rs | 38 +++--- datafusion/expr/src/logical_plan/builder.rs | 66 +++++----- datafusion/expr/src/logical_plan/ddl.rs | 6 +- datafusion/expr/src/logical_plan/dml.rs | 6 +- datafusion/expr/src/logical_plan/plan.rs | 20 +-- datafusion/expr/src/planner.rs | 4 +- datafusion/expr/src/simplify.rs | 6 +- datafusion/expr/src/table_source.rs | 4 +- datafusion/expr/src/type_coercion/mod.rs | 16 +-- datafusion/expr/src/utils.rs | 12 +- datafusion/expr/src/var_provider.rs | 4 +- datafusion/functions-array/src/planner.rs | 2 +- datafusion/functions/src/core/getfield.rs | 2 +- datafusion/functions/src/core/named_struct.rs | 2 +- datafusion/functions/src/math/log.rs | 2 +- datafusion/functions/src/math/power.rs | 2 +- .../src/analyzer/inline_table_scan.rs | 14 +-- .../optimizer/src/analyzer/type_coercion.rs | 50 ++++---- .../optimizer/src/common_subexpr_eliminate.rs | 14 +-- .../optimizer/src/eliminate_one_union.rs | 4 +- .../optimizer/src/optimize_projections/mod.rs | 20 +-- .../optimizer/src/propagate_empty_relation.rs | 4 +- datafusion/optimizer/src/push_down_filter.rs | 18 +-- .../simplify_expressions/expr_simplifier.rs | 28 ++--- .../src/unwrap_cast_in_comparison.rs | 50 ++++---- .../optimizer/tests/optimizer_integration.rs | 10 +- datafusion/physical-expr-common/src/utils.rs | 2 +- datafusion/physical-expr/src/planner.rs | 6 +- datafusion/proto-common/src/from_proto/mod.rs | 12 +- datafusion/proto-common/src/to_proto/mod.rs | 6 +- .../proto/src/logical_plan/from_proto.rs | 6 +- datafusion/proto/src/logical_plan/mod.rs | 4 +- .../tests/cases/roundtrip_logical_plan.rs | 6 +- datafusion/sql/examples/sql.rs | 4 +- datafusion/sql/src/expr/function.rs | 2 +- datafusion/sql/src/expr/identifier.rs | 4 +- datafusion/sql/src/expr/mod.rs | 8 +- datafusion/sql/src/expr/value.rs | 6 +- datafusion/sql/src/planner.rs | 30 ++--- datafusion/sql/src/statement.rs | 14 +-- datafusion/sql/src/unparser/expr.rs | 4 +- datafusion/sql/src/utils.rs | 4 +- datafusion/sql/tests/common/mod.rs | 10 +- datafusion/sql/tests/sql_integration.rs | 28 ++--- .../engines/datafusion_engine/normalize.rs | 6 +- datafusion/sqllogictest/src/test_context.rs | 4 +- .../substrait/src/logical_plan/consumer.rs | 16 +-- .../substrait/src/logical_plan/producer.rs | 2 +- 96 files changed, 634 insertions(+), 634 deletions(-) diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 2e50850c3723..c60649ca7c9d 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -42,7 +42,7 @@ use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -454,8 +454,8 @@ impl TableProvider for IndexTableProvider { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.indexed_file.schema.as_ref().clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.indexed_file.schema.as_ref().clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index e5b44cb25f31..be3dd70de3d1 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -37,8 +37,8 @@ use datafusion_expr::LogicalPlanBuilder; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; use tokio::time::timeout; /// This example demonstrates executing a simple query against a custom datasource @@ -164,10 +164,10 @@ impl TableProvider for CustomDataSource { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(LogicalSchema::new(vec![ - LogicalField::new("id", DataType::UInt8, false), - LogicalField::new("bank_account", DataType::UInt64, true), + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("id", DataType::UInt8, false), + LogicalPhysicalField::new("bank_account", DataType::UInt64, true), ])) } diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 5f3e8af3843e..16ec02515b07 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -28,8 +28,8 @@ use datafusion::functions_aggregate::first_last::first_value_udaf; use datafusion::optimizer::simplify_expressions::ExprSimplifier; use datafusion::physical_expr::{analyze, AnalysisContext, ExprBoundaries}; use datafusion::prelude::*; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr::BinaryExpr; @@ -159,7 +159,7 @@ fn simplify_demo() -> Result<()> { // you need to tell DataFusion the type of column "ts": let schema = - LogicalSchema::from(Schema::new(vec![make_ts_field("ts")])).to_dfschema_ref()?; + LogicalPhysicalSchema::from(Schema::new(vec![make_ts_field("ts")])).to_dfschema_ref()?; // And then build a simplifier // the ExecutionProps carries information needed to simplify @@ -180,7 +180,7 @@ fn simplify_demo() -> Result<()> { ); // here are some other examples of what DataFusion is capable of - let schema = LogicalSchema::from(Schema::new(vec![ + let schema = LogicalPhysicalSchema::from(Schema::new(vec![ make_field("i", DataType::Int64), make_field("b", DataType::Boolean), ])) @@ -297,14 +297,14 @@ fn expression_type_demo() -> Result<()> { // a schema. In this case we create a schema where the column `c` is of // type Utf8 (a String / VARCHAR) let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("c", DataType::Utf8, true)].into(), + vec![LogicalPhysicalField::new("c", DataType::Utf8, true)].into(), HashMap::new(), )?; assert_eq!("Utf8", format!("{}", expr.get_type(&schema).unwrap())); // Using a schema where the column `foo` is of type Int32 let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("c", DataType::Int32, true)].into(), + vec![LogicalPhysicalField::new("c", DataType::Int32, true)].into(), HashMap::new(), )?; assert_eq!("Int32", format!("{}", expr.get_type(&schema).unwrap())); @@ -314,8 +314,8 @@ fn expression_type_demo() -> Result<()> { let expr = col("c1") + col("c2"); let schema = DFSchema::from_unqualified_fields( vec![ - LogicalField::new("c1", DataType::Int32, true), - LogicalField::new("c2", DataType::Float32, true), + LogicalPhysicalField::new("c1", DataType::Int32, true), + LogicalPhysicalField::new("c2", DataType::Float32, true), ] .into(), HashMap::new(), diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index a19f36fd6ac7..e187cfab3859 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -22,7 +22,7 @@ use datafusion::error::Result; use datafusion::execution::context::{ FunctionFactory, RegisterFunction, SessionContext, SessionState, }; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{exec_err, internal_err, DataFusionError}; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; diff --git a/datafusion-examples/examples/logical_type.rs b/datafusion-examples/examples/logical_type.rs index 6790ec726982..19f6e445f9b5 100644 --- a/datafusion-examples/examples/logical_type.rs +++ b/datafusion-examples/examples/logical_type.rs @@ -5,10 +5,10 @@ use datafusion::error::Result; use datafusion::execution::context::SessionState; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, ExtensionTypeRef}; +use datafusion_common::logical_type::{TypeRelation, TypeRelationRef}; use datafusion_expr::{Expr, TableType}; use std::any::Any; use std::sync::Arc; @@ -41,7 +41,7 @@ impl Default for CustomMagicalType { } } -impl ExtensionType for CustomMagicalType { +impl TypeRelation for CustomMagicalType { fn logical(&self) -> &LogicalType { &self.logical } @@ -62,17 +62,17 @@ impl TableProvider for ExampleTableSource { self } - fn schema(&self) -> LogicalSchemaRef { + fn schema(&self) -> LogicalPhysicalSchemaRef { // TODO: ugly? - let custom_magical_type: ExtensionTypeRef = + let custom_magical_type: TypeRelationRef = Arc::new(CustomMagicalType::default()); // This schema will be equivalent to: // a -> Timestamp(Microsecond, None) // b -> Utf8 // c -> Int64 - Arc::new(LogicalSchema::new(vec![ - LogicalField::new( + Arc::new(LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new( "a", DataType::RunEndEncoded( Arc::new(Field::new("run_ends", DataType::Int64, false)), @@ -84,8 +84,8 @@ impl TableProvider for ExampleTableSource { ), false, ), - LogicalField::new("b", custom_magical_type, false), - LogicalField::new("c", DataType::Int64, true), + LogicalPhysicalField::new("b", custom_magical_type, false), + LogicalPhysicalField::new("c", DataType::Int64, true), ])) } diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 3e8d2cfc9a71..884392972bae 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -37,7 +37,7 @@ use datafusion::parquet::arrow::{ use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -213,8 +213,8 @@ impl TableProvider for IndexTableProvider { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.index.schema().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.index.schema().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index df7f708c5650..75bcce48eb85 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -27,7 +27,7 @@ use datafusion::execution::context::{ExecutionProps, SessionState}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{plan_err, ScalarValue}; use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::{Expr, TableType}; @@ -85,8 +85,8 @@ impl TableProvider for LocalCsvTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion-examples/examples/sql_frontend.rs b/datafusion-examples/examples/sql_frontend.rs index 28ccb86ecd19..b32261774f86 100644 --- a/datafusion-examples/examples/sql_frontend.rs +++ b/datafusion-examples/examples/sql_frontend.rs @@ -17,9 +17,9 @@ use arrow::datatypes::DataType; use datafusion_common::config::ConfigOptions; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{plan_err, Result}; use datafusion_expr::{ AggregateUDF, Expr, LogicalPlan, ScalarUDF, TableProviderFilterPushDown, TableSource, @@ -142,9 +142,9 @@ impl ContextProvider for MyContextProvider { fn get_table_source(&self, name: TableReference) -> Result> { if name.table() == "person" { Ok(Arc::new(MyTableSource { - schema: Arc::new(LogicalSchema::new(vec![ - LogicalField::new("name", DataType::Utf8, false), - LogicalField::new("age", DataType::UInt8, false), + schema: Arc::new(LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("name", DataType::Utf8, false), + LogicalPhysicalField::new("age", DataType::UInt8, false), ])), })) } else { @@ -160,7 +160,7 @@ impl ContextProvider for MyContextProvider { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -187,7 +187,7 @@ impl ContextProvider for MyContextProvider { /// TableSource is the part of TableProvider needed for creating a LogicalPlan. struct MyTableSource { - schema: LogicalSchemaRef, + schema: LogicalPhysicalSchemaRef, } impl TableSource for MyTableSource { @@ -195,7 +195,7 @@ impl TableSource for MyTableSource { self } - fn schema(&self) -> LogicalSchemaRef { + fn schema(&self) -> LogicalPhysicalSchemaRef { self.schema.clone() } diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index ca77df4ec95e..24b24db887fd 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -18,7 +18,7 @@ //! Column use crate::error::_schema_err; -use crate::logical_type::field::{LogicalField, LogicalFieldRef}; +use crate::logical_type::field::{LogicalPhysicalField, LogicalPhysicalFieldRef}; use crate::utils::{parse_identifiers_normalized, quote_identifier}; use crate::{DFSchema, DataFusionError, Result, SchemaError, TableReference}; use std::collections::HashSet; @@ -348,15 +348,15 @@ impl From for Column { } /// Create a column, use qualifier and field name -impl From<(Option<&TableReference>, &LogicalField)> for Column { - fn from((relation, field): (Option<&TableReference>, &LogicalField)) -> Self { +impl From<(Option<&TableReference>, &LogicalPhysicalField)> for Column { + fn from((relation, field): (Option<&TableReference>, &LogicalPhysicalField)) -> Self { Self::new(relation.cloned(), field.name()) } } /// Create a column, use qualifier and field name -impl From<(Option<&TableReference>, &LogicalFieldRef)> for Column { - fn from((relation, field): (Option<&TableReference>, &LogicalFieldRef)) -> Self { +impl From<(Option<&TableReference>, &LogicalPhysicalFieldRef)> for Column { + fn from((relation, field): (Option<&TableReference>, &LogicalPhysicalFieldRef)) -> Self { Self::new(relation.cloned(), field.name()) } } diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index c2c90c32735d..9f82bc3dee57 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -29,12 +29,12 @@ use crate::{ SchemaError, TableReference, }; -use crate::logical_type::field::{LogicalField, LogicalFieldRef}; -use crate::logical_type::fields::LogicalFields; +use crate::logical_type::field::{LogicalPhysicalField, LogicalPhysicalFieldRef}; +use crate::logical_type::fields::LogicalPhysicalFields; use crate::logical_type::schema::{ - LogicalSchema, LogicalSchemaBuilder, LogicalSchemaRef, + LogicalPhysicalSchema, LogicalPhysicalSchemaBuilder, LogicalPhysicalSchemaRef, }; -use crate::logical_type::{ExtensionType, TypeRelation}; +use crate::logical_type::{TypeRelation, LogicalPhysicalType}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; @@ -112,7 +112,7 @@ pub type DFSchemaRef = Arc; #[derive(Debug, Clone, PartialEq, Eq)] pub struct DFSchema { /// Inner Arrow schema reference. - inner: LogicalSchemaRef, + inner: LogicalPhysicalSchemaRef, /// Optional qualifiers for each column in this schema. In the same order as /// the `self.inner.fields` field_qualifiers: Vec>, @@ -124,7 +124,7 @@ impl DFSchema { /// Creates an empty `DFSchema` pub fn empty() -> Self { Self { - inner: Arc::new(LogicalSchema::new([])), + inner: Arc::new(LogicalPhysicalSchema::new([])), field_qualifiers: vec![], functional_dependencies: FunctionalDependencies::empty(), } @@ -133,26 +133,26 @@ impl DFSchema { /// Return a reference to the inner Arrow [`Schema`] /// /// Note this does not have the qualifier information - pub fn as_arrow(&self) -> &LogicalSchema { + pub fn as_arrow(&self) -> &LogicalPhysicalSchema { self.inner.as_ref() } /// Return a reference to the inner Arrow [`SchemaRef`] /// /// Note this does not have the qualifier information - pub fn inner(&self) -> &LogicalSchemaRef { + pub fn inner(&self) -> &LogicalPhysicalSchemaRef { &self.inner } /// Create a `DFSchema` from an Arrow schema where all the fields have a given qualifier pub fn new_with_metadata( - qualified_fields: Vec<(Option, Arc)>, + qualified_fields: Vec<(Option, Arc)>, metadata: HashMap, ) -> Result { - let (qualifiers, fields): (Vec>, Vec>) = + let (qualifiers, fields): (Vec>, Vec>) = qualified_fields.into_iter().unzip(); - let schema = Arc::new(LogicalSchema::new_with_metadata(fields, metadata)); + let schema = Arc::new(LogicalPhysicalSchema::new_with_metadata(fields, metadata)); let dfschema = Self { inner: schema, @@ -166,7 +166,7 @@ impl DFSchema { /// Create a new `DFSchema` from a list of Arrow [Field]s #[allow(deprecated)] pub fn from_unqualified_fields( - fields: LogicalFields, + fields: LogicalPhysicalFields, metadata: HashMap, ) -> Result { Self::from_unqualifed_fields(fields, metadata) @@ -178,11 +178,11 @@ impl DFSchema { note = "Please use `from_unqualified_fields` instead (this one's name is a typo). This method is subject to be removed soon" )] pub fn from_unqualifed_fields( - fields: LogicalFields, + fields: LogicalPhysicalFields, metadata: HashMap, ) -> Result { let field_count = fields.len(); - let schema = Arc::new(LogicalSchema::new_with_metadata(fields, metadata)); + let schema = Arc::new(LogicalPhysicalSchema::new_with_metadata(fields, metadata)); let dfschema = Self { inner: schema, field_qualifiers: vec![None; field_count], @@ -198,7 +198,7 @@ impl DFSchema { /// `DFSchema::try_from`. pub fn try_from_qualified_schema( qualifier: impl Into, - schema: &LogicalSchema, + schema: &LogicalPhysicalSchema, ) -> Result { let qualifier = qualifier.into(); let schema = DFSchema { @@ -213,7 +213,7 @@ impl DFSchema { /// Create a `DFSchema` from an Arrow schema where all the fields have a given qualifier pub fn from_field_specific_qualified_schema( qualifiers: Vec>, - schema: &LogicalSchemaRef, + schema: &LogicalPhysicalSchemaRef, ) -> Result { let dfschema = Self { inner: schema.clone(), @@ -268,7 +268,7 @@ impl DFSchema { /// 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 { - let mut schema_builder = LogicalSchemaBuilder::new(); + let mut schema_builder = LogicalPhysicalSchemaBuilder::new(); schema_builder.extend(self.inner.fields.iter().cloned()); schema_builder.extend(schema.fields().iter().cloned()); let new_schema = schema_builder.finish(); @@ -296,12 +296,12 @@ impl DFSchema { return; } - let self_fields: HashSet<(Option<&TableReference>, &LogicalFieldRef)> = + let self_fields: HashSet<(Option<&TableReference>, &LogicalPhysicalFieldRef)> = self.iter().collect(); let self_unqualified_names: HashSet<&str> = self.inner.fields.iter().map(|field| field.name()).collect(); - let mut schema_builder = LogicalSchemaBuilder::from(self.inner.fields.clone()); + let mut schema_builder = LogicalPhysicalSchemaBuilder::from(self.inner.fields.clone()); let mut qualifiers = Vec::new(); for (qualifier, field) in other_schema.iter() { // skip duplicate columns @@ -326,19 +326,19 @@ impl DFSchema { } /// Get a list of fields - pub fn fields(&self) -> &LogicalFields { + pub fn fields(&self) -> &LogicalPhysicalFields { &self.inner.fields } /// Returns an immutable reference of a specific `Field` instance selected using an /// offset within the internal `fields` vector - pub fn field(&self, i: usize) -> &LogicalField { + pub fn field(&self, i: usize) -> &LogicalPhysicalField { &self.inner.fields[i] } /// Returns an immutable reference of a specific `Field` instance selected using an /// offset within the internal `fields` vector and its qualifier - pub fn qualified_field(&self, i: usize) -> (Option<&TableReference>, &LogicalField) { + pub fn qualified_field(&self, i: usize) -> (Option<&TableReference>, &LogicalPhysicalField) { (self.field_qualifiers[i].as_ref(), self.field(i)) } @@ -405,7 +405,7 @@ impl DFSchema { &self, qualifier: Option<&TableReference>, name: &str, - ) -> Result<&LogicalField> { + ) -> Result<&LogicalPhysicalField> { if let Some(qualifier) = qualifier { self.field_with_qualified_name(qualifier, name) } else { @@ -418,7 +418,7 @@ impl DFSchema { &self, qualifier: Option<&TableReference>, name: &str, - ) -> Result<(Option<&TableReference>, &LogicalField)> { + ) -> Result<(Option<&TableReference>, &LogicalPhysicalField)> { if let Some(qualifier) = qualifier { let idx = self .index_of_column_by_name(Some(qualifier), name) @@ -433,7 +433,7 @@ impl DFSchema { pub fn fields_with_qualified( &self, qualifier: &TableReference, - ) -> Vec<&LogicalField> { + ) -> Vec<&LogicalPhysicalField> { self.iter() .filter(|(q, _)| q.map(|q| q.eq(qualifier)).unwrap_or(false)) .map(|(_, f)| f.as_ref()) @@ -452,7 +452,7 @@ impl DFSchema { } /// Find all fields that match the given name - pub fn fields_with_unqualified_name(&self, name: &str) -> Vec<&LogicalField> { + pub fn fields_with_unqualified_name(&self, name: &str) -> Vec<&LogicalPhysicalField> { self.fields() .iter() .filter(|field| field.name() == name) @@ -464,7 +464,7 @@ impl DFSchema { pub fn qualified_fields_with_unqualified_name( &self, name: &str, - ) -> Vec<(Option<&TableReference>, &LogicalField)> { + ) -> Vec<(Option<&TableReference>, &LogicalPhysicalField)> { self.iter() .filter(|(_, field)| field.name() == name) .map(|(qualifier, field)| (qualifier, field.as_ref())) @@ -490,7 +490,7 @@ impl DFSchema { pub fn qualified_field_with_unqualified_name( &self, name: &str, - ) -> Result<(Option<&TableReference>, &LogicalField)> { + ) -> Result<(Option<&TableReference>, &LogicalPhysicalField)> { let matches = self.qualified_fields_with_unqualified_name(name); match matches.len() { 0 => Err(unqualified_field_not_found(name, self)), @@ -522,7 +522,7 @@ impl DFSchema { } /// Find the field with the given name - pub fn field_with_unqualified_name(&self, name: &str) -> Result<&LogicalField> { + pub fn field_with_unqualified_name(&self, name: &str) -> Result<&LogicalPhysicalField> { let matches = self.qualified_fields_with_unqualified_name(name); match matches.len() { 0 => Err(unqualified_field_not_found(name, self)), @@ -558,7 +558,7 @@ impl DFSchema { &self, qualifier: &TableReference, name: &str, - ) -> Result<&LogicalField> { + ) -> Result<&LogicalPhysicalField> { let idx = self .index_of_column_by_name(Some(qualifier), name) .ok_or_else(|| field_not_found(Some(qualifier.clone()), name, self))?; @@ -567,7 +567,7 @@ impl DFSchema { } /// Find the field with the given qualified column - pub fn field_from_column(&self, column: &Column) -> Result<&LogicalField> { + pub fn field_from_column(&self, column: &Column) -> Result<&LogicalPhysicalField> { match &column.relation { Some(r) => self.field_with_qualified_name(r, &column.name), None => self.field_with_unqualified_name(&column.name), @@ -578,7 +578,7 @@ impl DFSchema { pub fn qualified_field_from_column( &self, column: &Column, - ) -> Result<(Option<&TableReference>, &LogicalField)> { + ) -> Result<(Option<&TableReference>, &LogicalPhysicalField)> { self.qualified_field_with_name(column.relation.as_ref(), &column.name) } @@ -684,7 +684,7 @@ impl DFSchema { /// than datatype_is_semantically_equal in that a Dictionary type is logically /// equal to a plain V type, but not semantically equal. Dictionary is also /// logically equal to Dictionary. - pub fn datatype_is_logically_equal(dt1: &TypeRelation, dt2: &TypeRelation) -> bool { + pub fn datatype_is_logically_equal(dt1: &LogicalPhysicalType, dt2: &LogicalPhysicalType) -> bool { dt1 == dt2 } @@ -779,7 +779,7 @@ impl DFSchema { /// Iterate over the qualifiers and fields in the DFSchema pub fn iter( &self, - ) -> impl Iterator, &LogicalFieldRef)> { + ) -> impl Iterator, &LogicalPhysicalFieldRef)> { self.field_qualifiers .iter() .zip(self.inner.fields.iter()) @@ -804,16 +804,16 @@ impl From<&DFSchema> for Schema { } /// Create a `DFSchema` from an Arrow schema -impl TryFrom for DFSchema { +impl TryFrom for DFSchema { type Error = DataFusionError; - fn try_from(schema: LogicalSchema) -> Result { + fn try_from(schema: LogicalPhysicalSchema) -> Result { Self::try_from(Arc::new(schema)) } } -impl TryFrom for DFSchema { +impl TryFrom for DFSchema { type Error = DataFusionError; - fn try_from(schema: LogicalSchemaRef) -> Result { + fn try_from(schema: LogicalPhysicalSchemaRef) -> Result { let field_count = schema.fields.len(); let dfschema = Self { inner: schema, @@ -828,7 +828,7 @@ impl TryFrom for DFSchema { impl TryFrom for DFSchema { type Error = DataFusionError; fn try_from(schema: Schema) -> Result { - Self::try_from(LogicalSchema::from(schema)) + Self::try_from(LogicalPhysicalSchema::from(schema)) } } @@ -867,22 +867,22 @@ where } } -impl ToDFSchema for LogicalSchema { +impl ToDFSchema for LogicalPhysicalSchema { fn to_dfschema(self) -> Result { DFSchema::try_from(self) } } -impl ToDFSchema for LogicalSchemaRef { +impl ToDFSchema for LogicalPhysicalSchemaRef { fn to_dfschema(self) -> Result { DFSchema::try_from(self) } } -impl ToDFSchema for Vec { +impl ToDFSchema for Vec { fn to_dfschema(self) -> Result { let field_count = self.len(); - let schema = LogicalSchema { + let schema = LogicalPhysicalSchema { fields: self.into(), metadata: HashMap::new(), }; @@ -919,13 +919,13 @@ pub trait ExprSchema: std::fmt::Debug { fn nullable(&self, col: &Column) -> Result; /// What is the datatype of this column? - fn data_type(&self, col: &Column) -> Result<&TypeRelation>; + fn data_type(&self, col: &Column) -> Result<&LogicalPhysicalType>; /// Returns the column's optional metadata. fn metadata(&self, col: &Column) -> Result<&HashMap>; /// Return the coulmn's datatype and nullability - fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)>; + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalPhysicalType, bool)>; } // Implement `ExprSchema` for `Arc` @@ -934,7 +934,7 @@ impl + std::fmt::Debug> ExprSchema for P { self.as_ref().nullable(col) } - fn data_type(&self, col: &Column) -> Result<&TypeRelation> { + fn data_type(&self, col: &Column) -> Result<&LogicalPhysicalType> { self.as_ref().data_type(col) } @@ -942,7 +942,7 @@ impl + std::fmt::Debug> ExprSchema for P { ExprSchema::metadata(self.as_ref(), col) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalPhysicalType, bool)> { self.as_ref().data_type_and_nullable(col) } } @@ -952,7 +952,7 @@ impl ExprSchema for DFSchema { Ok(self.field_from_column(col)?.is_nullable()) } - fn data_type(&self, col: &Column) -> Result<&TypeRelation> { + fn data_type(&self, col: &Column) -> Result<&LogicalPhysicalType> { Ok(self.field_from_column(col)?.data_type()) } @@ -960,7 +960,7 @@ impl ExprSchema for DFSchema { Ok(self.field_from_column(col)?.metadata()) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalPhysicalType, bool)> { let field = self.field_from_column(col)?; Ok((field.data_type(), field.is_nullable())) } @@ -1083,9 +1083,9 @@ mod tests { fn test_from_field_specific_qualified_schema() -> Result<()> { let schema = DFSchema::from_field_specific_qualified_schema( vec![Some("t1".into()), None], - &Arc::new(LogicalSchema::new(vec![ - LogicalField::new("c0", DataType::Boolean, true), - LogicalField::new("c1", DataType::Boolean, true), + &Arc::new(LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("c0", DataType::Boolean, true), + LogicalPhysicalField::new("c1", DataType::Boolean, true), ])), )?; assert_eq!("fields:[t1.c0, c1], metadata:{}", schema.to_string()); @@ -1247,7 +1247,7 @@ mod tests { vec![Field::new("c0", DataType::Int64, true)], metadata.clone(), ); - let logical_schema = LogicalSchema::from(arrow_schema); + let logical_schema = LogicalPhysicalSchema::from(arrow_schema); let logical_schema_ref = Arc::new(logical_schema.clone()); let df_schema = DFSchema { @@ -1278,7 +1278,7 @@ mod tests { assert_eq!(df_schema_ref, logical_schema_ref.to_dfschema_ref().unwrap()); } - fn test_schema_1() -> LogicalSchema { + fn test_schema_1() -> LogicalPhysicalSchema { Schema::new(vec![ Field::new("c0", DataType::Boolean, true), Field::new("c1", DataType::Boolean, true), @@ -1295,7 +1295,7 @@ mod tests { b_metadata.insert("key".to_string(), "value".to_string()); let b_field = Field::new("b", DataType::Int64, false).with_metadata(b_metadata); - let schema = LogicalSchemaRef::new(Schema::new(vec![a_field, b_field]).into()); + let schema = LogicalPhysicalSchemaRef::new(Schema::new(vec![a_field, b_field]).into()); let df_schema = DFSchema { inner: schema.clone(), @@ -1339,10 +1339,10 @@ mod tests { Ok(()) } - fn test_schema_2() -> LogicalSchema { - LogicalSchema::new(vec![ - LogicalField::new("c100", DataType::Boolean, true), - LogicalField::new("c101", DataType::Boolean, true), + fn test_schema_2() -> LogicalPhysicalSchema { + LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("c100", DataType::Boolean, true), + LogicalPhysicalField::new("c101", DataType::Boolean, true), ]) } diff --git a/datafusion/common/src/logical_type/field.rs b/datafusion/common/src/logical_type/field.rs index e59384110322..5220b788a243 100644 --- a/datafusion/common/src/logical_type/field.rs +++ b/datafusion/common/src/logical_type/field.rs @@ -21,50 +21,50 @@ use std::sync::Arc; use arrow_schema::{Field, FieldRef}; -use super::{ExtensionType, TypeRelation}; +use super::{TypeRelation, LogicalPhysicalType}; -pub type LogicalFieldRef = Arc; +pub type LogicalPhysicalFieldRef = Arc; #[derive(Debug, Clone)] -pub struct LogicalField { +pub struct LogicalPhysicalField { name: String, - data_type: TypeRelation, + data_type: LogicalPhysicalType, nullable: bool, metadata: HashMap, } -impl From<&Field> for LogicalField { +impl From<&Field> for LogicalPhysicalField { fn from(value: &Field) -> Self { Self::new(value.name().clone(), value.data_type(), value.is_nullable()) } } -impl From for LogicalField { +impl From for LogicalPhysicalField { fn from(value: Field) -> Self { Self::from(&value) } } -impl From<&FieldRef> for LogicalField { +impl From<&FieldRef> for LogicalPhysicalField { fn from(value: &FieldRef) -> Self { Self::from(value.as_ref()) } } -impl From for LogicalField { +impl From for LogicalPhysicalField { fn from(value: FieldRef) -> Self { Self::from(value.as_ref()) } } -impl Into for LogicalField { +impl Into for LogicalPhysicalField { fn into(self) -> Field { Field::new(self.name, self.data_type.physical().clone(), self.nullable) .with_metadata(self.metadata) } } -impl PartialEq for LogicalField { +impl PartialEq for LogicalPhysicalField { fn eq(&self, other: &Self) -> bool { self.name == other.name && self.data_type == other.data_type @@ -73,9 +73,9 @@ impl PartialEq for LogicalField { } } -impl Eq for LogicalField {} +impl Eq for LogicalPhysicalField {} -impl Hash for LogicalField { +impl Hash for LogicalPhysicalField { fn hash(&self, state: &mut H) { self.name.hash(state); self.data_type.hash(state); @@ -91,13 +91,13 @@ impl Hash for LogicalField { } } -impl LogicalField { +impl LogicalPhysicalField { pub fn new( name: impl Into, - data_type: impl Into, + data_type: impl Into, nullable: bool, ) -> Self { - LogicalField { + LogicalPhysicalField { name: name.into(), data_type: data_type.into(), nullable, @@ -105,7 +105,7 @@ impl LogicalField { } } - pub fn new_list_field(data_type: impl Into, nullable: bool) -> Self { + pub fn new_list_field(data_type: impl Into, nullable: bool) -> Self { Self::new("item", data_type, nullable) } @@ -113,7 +113,7 @@ impl LogicalField { &self.name } - pub fn data_type(&self) -> &TypeRelation { + pub fn data_type(&self) -> &LogicalPhysicalType { &self.data_type } @@ -144,13 +144,13 @@ impl LogicalField { } #[inline] - pub fn with_data_type(mut self, data_type: TypeRelation) -> Self { + pub fn with_data_type(mut self, data_type: LogicalPhysicalType) -> Self { self.data_type = data_type; self } } -impl std::fmt::Display for LogicalField { +impl std::fmt::Display for LogicalPhysicalField { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "{self:?}") } diff --git a/datafusion/common/src/logical_type/fields.rs b/datafusion/common/src/logical_type/fields.rs index 927c67c7617e..4e0dd2291c60 100644 --- a/datafusion/common/src/logical_type/fields.rs +++ b/datafusion/common/src/logical_type/fields.rs @@ -20,35 +20,35 @@ use std::sync::Arc; use arrow_schema::{Field, FieldRef, Fields, UnionFields}; -use super::field::{LogicalField, LogicalFieldRef}; +use super::field::{LogicalPhysicalField, LogicalPhysicalFieldRef}; #[derive(Clone, Eq, PartialEq, Hash)] -pub struct LogicalFields(Arc<[LogicalFieldRef]>); +pub struct LogicalPhysicalFields(Arc<[LogicalPhysicalFieldRef]>); -impl std::fmt::Debug for LogicalFields { +impl std::fmt::Debug for LogicalPhysicalFields { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { self.0.as_ref().fmt(f) } } -impl From<&Fields> for LogicalFields { +impl From<&Fields> for LogicalPhysicalFields { fn from(value: &Fields) -> Self { Self( value .iter() - .map(|v| LogicalFieldRef::new(v.into())) + .map(|v| LogicalPhysicalFieldRef::new(v.into())) .collect(), ) } } -impl From for LogicalFields { +impl From for LogicalPhysicalFields { fn from(value: Fields) -> Self { Self::from(&value) } } -impl Into for LogicalFields { +impl Into for LogicalPhysicalFields { fn into(self) -> Fields { Fields::from( self.iter() @@ -58,73 +58,73 @@ impl Into for LogicalFields { } } -impl Default for LogicalFields { +impl Default for LogicalPhysicalFields { fn default() -> Self { Self::empty() } } -impl FromIterator for LogicalFields { - fn from_iter>(iter: T) -> Self { +impl FromIterator for LogicalPhysicalFields { + fn from_iter>(iter: T) -> Self { iter.into_iter().map(Arc::new).collect() } } -impl FromIterator for LogicalFields { - fn from_iter>(iter: T) -> Self { +impl FromIterator for LogicalPhysicalFields { + fn from_iter>(iter: T) -> Self { Self(iter.into_iter().collect()) } } -impl From> for LogicalFields { - fn from(value: Vec) -> Self { +impl From> for LogicalPhysicalFields { + fn from(value: Vec) -> Self { value.into_iter().collect() } } -impl From> for LogicalFields { - fn from(value: Vec) -> Self { +impl From> for LogicalPhysicalFields { + fn from(value: Vec) -> Self { Self(value.into()) } } -impl From<&[LogicalFieldRef]> for LogicalFields { - fn from(value: &[LogicalFieldRef]) -> Self { +impl From<&[LogicalPhysicalFieldRef]> for LogicalPhysicalFields { + fn from(value: &[LogicalPhysicalFieldRef]) -> Self { Self(value.into()) } } -impl From<[LogicalFieldRef; N]> for LogicalFields { - fn from(value: [LogicalFieldRef; N]) -> Self { +impl From<[LogicalPhysicalFieldRef; N]> for LogicalPhysicalFields { + fn from(value: [LogicalPhysicalFieldRef; N]) -> Self { Self(Arc::new(value)) } } -impl Deref for LogicalFields { - type Target = [LogicalFieldRef]; +impl Deref for LogicalPhysicalFields { + type Target = [LogicalPhysicalFieldRef]; fn deref(&self) -> &Self::Target { self.0.as_ref() } } -impl<'a> IntoIterator for &'a LogicalFields { - type Item = &'a LogicalFieldRef; - type IntoIter = std::slice::Iter<'a, LogicalFieldRef>; +impl<'a> IntoIterator for &'a LogicalPhysicalFields { + type Item = &'a LogicalPhysicalFieldRef; + type IntoIter = std::slice::Iter<'a, LogicalPhysicalFieldRef>; fn into_iter(self) -> Self::IntoIter { self.0.iter() } } -impl LogicalFields { +impl LogicalPhysicalFields { pub fn empty() -> Self { Self(Arc::new([])) } } #[derive(Clone, Eq, PartialEq, Hash)] -pub struct LogicalUnionFields(Arc<[(i8, LogicalFieldRef)]>); +pub struct LogicalUnionFields(Arc<[(i8, LogicalPhysicalFieldRef)]>); impl std::fmt::Debug for LogicalUnionFields { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { @@ -132,8 +132,8 @@ impl std::fmt::Debug for LogicalUnionFields { } } -impl FromIterator<(i8, LogicalFieldRef)> for LogicalUnionFields { - fn from_iter>(iter: T) -> Self { +impl FromIterator<(i8, LogicalPhysicalFieldRef)> for LogicalUnionFields { + fn from_iter>(iter: T) -> Self { Self(iter.into_iter().collect()) } } @@ -143,7 +143,7 @@ impl From<&UnionFields> for LogicalUnionFields { Self::from_iter( value .iter() - .map(|(i, f)| (i, LogicalFieldRef::new(f.into()))), + .map(|(i, f)| (i, LogicalPhysicalFieldRef::new(f.into()))), ) } } diff --git a/datafusion/common/src/logical_type/mod.rs b/datafusion/common/src/logical_type/mod.rs index aed404c5c6c6..c59946719a21 100644 --- a/datafusion/common/src/logical_type/mod.rs +++ b/datafusion/common/src/logical_type/mod.rs @@ -19,8 +19,8 @@ use std::{fmt::Display, sync::Arc}; use arrow_schema::DataType; -use field::{LogicalField, LogicalFieldRef}; -use fields::LogicalFields; +use field::{LogicalPhysicalField, LogicalPhysicalFieldRef}; +use fields::LogicalPhysicalFields; use signature::LogicalType; pub mod field; @@ -28,22 +28,22 @@ pub mod fields; pub mod schema; pub mod signature; -pub type ExtensionTypeRef = Arc; +pub type TypeRelationRef = Arc; -pub trait ExtensionType: std::fmt::Debug { +pub trait TypeRelation: std::fmt::Debug { fn logical(&self) -> &LogicalType; fn physical(&self) -> &DataType; } #[derive(Clone, Debug)] -pub struct TypeRelation(ExtensionTypeRef); +pub struct LogicalPhysicalType(TypeRelationRef); -impl TypeRelation { - pub fn new_list(inner: TypeRelation, nullable: bool) -> Self { +impl LogicalPhysicalType { + pub fn new_list(inner: LogicalPhysicalType, nullable: bool) -> Self { Self(Arc::new(NativeType::new_list(inner, nullable))) } - pub fn new_struct(fields: LogicalFields) -> Self { + pub fn new_struct(fields: LogicalPhysicalFields) -> Self { Self(Arc::new(NativeType::new_struct(fields))) } } @@ -56,7 +56,7 @@ pub struct NativeType { physical: DataType, } -impl ExtensionType for NativeType { +impl TypeRelation for NativeType { fn logical(&self) -> &LogicalType { &self.logical } @@ -67,16 +67,16 @@ impl ExtensionType for NativeType { } impl NativeType { - pub fn new_list(inner: TypeRelation, nullable: bool) -> Self { + pub fn new_list(inner: LogicalPhysicalType, nullable: bool) -> Self { Self { physical: DataType::new_list(inner.physical().clone(), nullable), - logical: LogicalType::List(LogicalFieldRef::new( - LogicalField::new_list_field(inner, nullable), + logical: LogicalType::List(LogicalPhysicalFieldRef::new( + LogicalPhysicalField::new_list_field(inner, nullable), )), } } - pub fn new_struct(fields: LogicalFields) -> Self { + pub fn new_struct(fields: LogicalPhysicalFields) -> Self { Self { physical: DataType::Struct(fields.clone().into()), logical: LogicalType::Struct(fields), @@ -84,7 +84,7 @@ impl NativeType { } } -impl ExtensionType for TypeRelation { +impl TypeRelation for LogicalPhysicalType { fn logical(&self) -> &LogicalType { self.0.logical() } @@ -94,21 +94,21 @@ impl ExtensionType for TypeRelation { } } -impl PartialEq for TypeRelation { +impl PartialEq for LogicalPhysicalType { fn eq(&self, other: &Self) -> bool { self.logical() == other.logical() } } -impl Eq for TypeRelation {} +impl Eq for LogicalPhysicalType {} -impl std::hash::Hash for TypeRelation { +impl std::hash::Hash for LogicalPhysicalType { fn hash(&self, state: &mut H) { self.logical().hash(state) } } -impl Display for TypeRelation { +impl Display for LogicalPhysicalType { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{self:?}") } @@ -123,19 +123,19 @@ impl From for NativeType { } } -impl From for TypeRelation { - fn from(value: ExtensionTypeRef) -> Self { +impl From for LogicalPhysicalType { + fn from(value: TypeRelationRef) -> Self { Self(value) } } -impl From<&DataType> for TypeRelation { +impl From<&DataType> for LogicalPhysicalType { fn from(value: &DataType) -> Self { value.clone().into() } } -impl From for TypeRelation { +impl From for LogicalPhysicalType { fn from(value: DataType) -> Self { Self(NativeTypeRef::new(value.into())) } @@ -181,7 +181,7 @@ impl From<&DataType> for LogicalType { | DataType::FixedSizeList(f, _) | DataType::LargeList(f) | DataType::LargeListView(f) => { - LogicalType::List(LogicalFieldRef::new(f.as_ref().clone().into())) + LogicalType::List(LogicalPhysicalFieldRef::new(f.as_ref().clone().into())) } DataType::Struct(f) => LogicalType::Struct(f.clone().into()), DataType::Dictionary(_, t) => t.as_ref().into(), @@ -192,7 +192,7 @@ impl From<&DataType> for LogicalType { LogicalType::Decimal256(precision.clone(), scale.clone()) } DataType::Map(f, sorted) => LogicalType::Map( - LogicalFieldRef::new(f.as_ref().clone().into()), + LogicalPhysicalFieldRef::new(f.as_ref().clone().into()), sorted.clone(), ), DataType::RunEndEncoded(_, f) => f.data_type().into(), diff --git a/datafusion/common/src/logical_type/schema.rs b/datafusion/common/src/logical_type/schema.rs index 99cf5754c559..856060162253 100644 --- a/datafusion/common/src/logical_type/schema.rs +++ b/datafusion/common/src/logical_type/schema.rs @@ -20,16 +20,16 @@ use std::sync::Arc; use arrow_schema::{Schema, SchemaRef}; -use super::field::{LogicalField, LogicalFieldRef}; -use super::fields::LogicalFields; +use super::field::{LogicalPhysicalField, LogicalPhysicalFieldRef}; +use super::fields::LogicalPhysicalFields; #[derive(Debug, Default)] -pub struct LogicalSchemaBuilder { - fields: Vec, +pub struct LogicalPhysicalSchemaBuilder { + fields: Vec, metadata: HashMap, } -impl LogicalSchemaBuilder { +impl LogicalPhysicalSchemaBuilder { pub fn new() -> Self { Self::default() } @@ -41,19 +41,19 @@ impl LogicalSchemaBuilder { } } - pub fn push(&mut self, field: impl Into) { + pub fn push(&mut self, field: impl Into) { self.fields.push(field.into()) } - pub fn remove(&mut self, idx: usize) -> LogicalFieldRef { + pub fn remove(&mut self, idx: usize) -> LogicalPhysicalFieldRef { self.fields.remove(idx) } - pub fn field(&mut self, idx: usize) -> &LogicalFieldRef { + pub fn field(&mut self, idx: usize) -> &LogicalPhysicalFieldRef { &mut self.fields[idx] } - pub fn field_mut(&mut self, idx: usize) -> &mut LogicalFieldRef { + pub fn field_mut(&mut self, idx: usize) -> &mut LogicalPhysicalFieldRef { &mut self.fields[idx] } @@ -69,16 +69,16 @@ impl LogicalSchemaBuilder { self.fields.reverse(); } - pub fn finish(self) -> LogicalSchema { - LogicalSchema { + pub fn finish(self) -> LogicalPhysicalSchema { + LogicalPhysicalSchema { fields: self.fields.into(), metadata: self.metadata, } } } -impl From<&LogicalFields> for LogicalSchemaBuilder { - fn from(value: &LogicalFields) -> Self { +impl From<&LogicalPhysicalFields> for LogicalPhysicalSchemaBuilder { + fn from(value: &LogicalPhysicalFields) -> Self { Self { fields: value.to_vec(), metadata: Default::default(), @@ -86,8 +86,8 @@ impl From<&LogicalFields> for LogicalSchemaBuilder { } } -impl From for LogicalSchemaBuilder { - fn from(value: LogicalFields) -> Self { +impl From for LogicalPhysicalSchemaBuilder { + fn from(value: LogicalPhysicalFields) -> Self { Self { fields: value.to_vec(), metadata: Default::default(), @@ -95,14 +95,14 @@ impl From for LogicalSchemaBuilder { } } -impl From<&LogicalSchema> for LogicalSchemaBuilder { - fn from(value: &LogicalSchema) -> Self { +impl From<&LogicalPhysicalSchema> for LogicalPhysicalSchemaBuilder { + fn from(value: &LogicalPhysicalSchema) -> Self { Self::from(value.clone()) } } -impl From for LogicalSchemaBuilder { - fn from(value: LogicalSchema) -> Self { +impl From for LogicalPhysicalSchemaBuilder { + fn from(value: LogicalPhysicalSchema) -> Self { Self { fields: value.fields.to_vec(), metadata: value.metadata, @@ -110,8 +110,8 @@ impl From for LogicalSchemaBuilder { } } -impl Extend for LogicalSchemaBuilder { - fn extend>(&mut self, iter: T) { +impl Extend for LogicalPhysicalSchemaBuilder { + fn extend>(&mut self, iter: T) { let iter = iter.into_iter(); self.fields.reserve(iter.size_hint().0); for f in iter { @@ -120,8 +120,8 @@ impl Extend for LogicalSchemaBuilder { } } -impl Extend for LogicalSchemaBuilder { - fn extend>(&mut self, iter: T) { +impl Extend for LogicalPhysicalSchemaBuilder { + fn extend>(&mut self, iter: T) { let iter = iter.into_iter(); self.fields.reserve(iter.size_hint().0); for f in iter { @@ -130,15 +130,15 @@ impl Extend for LogicalSchemaBuilder { } } -pub type LogicalSchemaRef = Arc; +pub type LogicalPhysicalSchemaRef = Arc; #[derive(Debug, Clone, PartialEq, Eq)] -pub struct LogicalSchema { - pub fields: LogicalFields, +pub struct LogicalPhysicalSchema { + pub fields: LogicalPhysicalFields, pub metadata: HashMap, } -impl From for LogicalSchema { +impl From for LogicalPhysicalSchema { fn from(value: Schema) -> Self { Self { fields: value.fields.into(), @@ -147,25 +147,25 @@ impl From for LogicalSchema { } } -impl From<&Schema> for LogicalSchema { +impl From<&Schema> for LogicalPhysicalSchema { fn from(value: &Schema) -> Self { Self::from(value.clone()) } } -impl From for LogicalSchema { +impl From for LogicalPhysicalSchema { fn from(value: SchemaRef) -> Self { Self::from(value.as_ref()) } } -impl From<&SchemaRef> for LogicalSchema { +impl From<&SchemaRef> for LogicalPhysicalSchema { fn from(value: &SchemaRef) -> Self { Self::from(value.as_ref()) } } -impl Into for LogicalSchema { +impl Into for LogicalPhysicalSchema { fn into(self) -> Schema { Schema { fields: self.fields.into(), @@ -174,14 +174,14 @@ impl Into for LogicalSchema { } } -impl LogicalSchema { - pub fn new(fields: impl Into) -> Self { +impl LogicalPhysicalSchema { + pub fn new(fields: impl Into) -> Self { Self::new_with_metadata(fields, HashMap::new()) } #[inline] pub fn new_with_metadata( - fields: impl Into, + fields: impl Into, metadata: HashMap, ) -> Self { Self { @@ -200,11 +200,11 @@ impl LogicalSchema { &self.metadata } - pub fn field(&self, i: usize) -> &LogicalFieldRef { + pub fn field(&self, i: usize) -> &LogicalPhysicalFieldRef { &self.fields[i] } - pub fn fields(&self) -> &LogicalFields { + pub fn fields(&self) -> &LogicalPhysicalFields { &self.fields } } diff --git a/datafusion/common/src/logical_type/signature.rs b/datafusion/common/src/logical_type/signature.rs index 92126d466109..5e35870cc6aa 100644 --- a/datafusion/common/src/logical_type/signature.rs +++ b/datafusion/common/src/logical_type/signature.rs @@ -21,8 +21,8 @@ use std::sync::Arc; use arrow_schema::{DataType, FieldRef, IntervalUnit, TimeUnit, UnionMode}; use super::{ - field::LogicalFieldRef, - fields::{LogicalFields, LogicalUnionFields}, + field::LogicalPhysicalFieldRef, + fields::{LogicalPhysicalFields, LogicalUnionFields}, }; #[derive(Clone, Debug, PartialEq, Eq, Hash)] @@ -48,9 +48,9 @@ pub enum LogicalType { Timestamp(TimeUnit, Option>), Duration(TimeUnit), Interval(IntervalUnit), - List(LogicalFieldRef), - Struct(LogicalFields), - Map(LogicalFieldRef, bool), + List(LogicalPhysicalFieldRef), + Struct(LogicalPhysicalFields), + Map(LogicalPhysicalFieldRef, bool), Decimal128(u8, i8), Decimal256(u8, i8), Union(LogicalUnionFields, UnionMode), // TODO: extension signatures? diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index 66908d04626d..8042f0663ba0 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -16,7 +16,7 @@ // under the License. use crate::error::{_plan_datafusion_err, _plan_err}; -use crate::logical_type::TypeRelation; +use crate::logical_type::LogicalPhysicalType; use crate::{Result, ScalarValue}; use std::collections::HashMap; @@ -31,7 +31,7 @@ pub enum ParamValues { impl ParamValues { /// Verify parameter list length and type - pub fn verify(&self, expect: &[TypeRelation]) -> Result<()> { + pub fn verify(&self, expect: &[LogicalPhysicalType]) -> Result<()> { match self { ParamValues::List(list) => { // Verify if the number of params matches the number of values diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2c1b6f62bbd8..583663169ed9 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -59,8 +59,8 @@ use datafusion_expr::{ use datafusion_functions_aggregate::expr_fn::{avg, count, median, stddev, sum}; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; +use datafusion_common::logical_type::TypeRelation; /// Contains options that control how data is /// written out from a DataFrame @@ -1656,7 +1656,7 @@ impl TableProvider for DataFrameTableProvider { Ok(vec![TableProviderFilterPushDown::Exact; filters.len()]) } - fn schema(&self) -> LogicalSchemaRef { + fn schema(&self) -> LogicalPhysicalSchemaRef { self.plan.schema().inner().clone() } diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index 7610fb6e4fb3..570e8afb2ea7 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_physical_plan::work_table::WorkTableExec; use crate::{ @@ -68,8 +68,8 @@ impl TableProvider for CteWorkTable { None } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.table_schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.table_schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index bd85edb31fa9..4b3cb695413d 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use crate::datasource::TableProvider; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{internal_err, Constraints}; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; @@ -52,7 +52,7 @@ impl TableSource for DefaultTableSource { } /// Get a reference to the schema for this table - fn schema(&self) -> LogicalSchemaRef { + fn schema(&self) -> LogicalPhysicalSchemaRef { self.table_provider.schema() } diff --git a/datafusion/core/src/datasource/empty.rs b/datafusion/core/src/datasource/empty.rs index 62a800834e73..e2c4ff1bbbe4 100644 --- a/datafusion/core/src/datasource/empty.rs +++ b/datafusion/core/src/datasource/empty.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use arrow::datatypes::*; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::project_schema; use crate::datasource::{TableProvider, TableType}; @@ -60,8 +60,8 @@ impl TableProvider for EmptyTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 5f8e0ae461d4..2d041e4a1c18 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -38,7 +38,7 @@ use futures::stream::FuturesUnordered; use futures::{stream::BoxStream, StreamExt, TryStreamExt}; use log::{debug, trace}; -use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion_common::{Column, DFSchema, DataFusionError}; use datafusion_expr::{Expr, Volatility}; @@ -265,7 +265,7 @@ async fn prune_partitions( let df_schema = DFSchema::from_unqualified_fields( partition_cols .iter() - .map(|(n, d)| LogicalField::new(n, d.clone(), true)) + .map(|(n, d)| LogicalPhysicalField::new(n, d.clone(), true)) .collect(), Default::default(), )?; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 15786224c742..b43751c39161 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -52,7 +52,7 @@ use datafusion_physical_expr::{ }; use async_trait::async_trait; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; @@ -723,8 +723,8 @@ impl TableProvider for ListingTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.table_schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.table_schema.clone().into()) } fn constraints(&self) -> Option<&Constraints> { @@ -790,7 +790,7 @@ impl TableProvider for ListingTable { let filters = if let Some(expr) = conjunction(filters.to_vec()) { // NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns. let table_df_schema = - LogicalSchema::from(self.table_schema.as_ref().clone()).to_dfschema()?; + LogicalPhysicalSchema::from(self.table_schema.as_ref().clone()).to_dfschema()?; let filters = create_physical_expr(&expr, &table_df_schema, state.execution_props())?; Some(filters) diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index d54b21ebef0e..1a78f0924b4f 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -41,7 +41,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use futures::StreamExt; use log::debug; use parking_lot::Mutex; @@ -193,8 +193,8 @@ impl TableProvider for MemTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn constraints(&self) -> Option<&Constraints> { diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 46765269ad4b..89f4ea060bc4 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::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{not_impl_err, Constraints, Statistics}; use datafusion_expr::{CreateExternalTable, LogicalPlan}; pub use datafusion_expr::{TableProviderFilterPushDown, TableType}; @@ -41,7 +41,7 @@ pub trait TableProvider: Sync + Send { fn as_any(&self) -> &dyn Any; /// Get a reference to the schema for this table - fn schema(&self) -> LogicalSchemaRef; + fn schema(&self) -> LogicalPhysicalSchemaRef; /// Get a reference to the constraints of the table. /// Returns: diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 24b49fe7b179..1538ed4b9560 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -42,7 +42,7 @@ use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use futures::StreamExt; /// A [`TableProviderFactory`] for [`StreamTable`] @@ -309,8 +309,8 @@ impl TableProvider for StreamTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.0.source.schema().clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.0.source.schema().clone().into()) } fn constraints(&self) -> Option<&Constraints> { diff --git a/datafusion/core/src/datasource/streaming.rs b/datafusion/core/src/datasource/streaming.rs index 5810cbf226bb..041a9a46097a 100644 --- a/datafusion/core/src/datasource/streaming.rs +++ b/datafusion/core/src/datasource/streaming.rs @@ -27,7 +27,7 @@ use crate::datasource::TableProvider; use crate::execution::context::SessionState; use crate::physical_plan::streaming::{PartitionStream, StreamingTableExec}; use crate::physical_plan::ExecutionPlan; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{plan_err, Result}; use datafusion_expr::{Expr, TableType}; use log::debug; @@ -75,8 +75,8 @@ impl TableProvider for StreamingTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 6dd1c4398262..0d3effcd71a1 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -21,7 +21,7 @@ use std::{any::Any, sync::Arc}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::Column; use datafusion_expr::{LogicalPlanBuilder, TableProviderFilterPushDown}; @@ -83,8 +83,8 @@ impl TableProvider for ViewTable { Some(&self.logical_plan) } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.table_schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.table_schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 4785a7e3e6c8..f6716cd89630 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -50,7 +50,7 @@ use datafusion_common::config::{ConfigExtension, ConfigOptions, TableOptions}; use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; use datafusion_common::file_options::file_type::FileType; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::tree_node::TreeNode; use datafusion_common::{ config_err, not_impl_err, plan_datafusion_err, DFSchema, DataFusionError, @@ -1035,7 +1035,7 @@ impl<'a> ContextProvider for SessionContextProvider<'a> { self.state.window_functions().get(name).cloned() } - fn get_variable_type(&self, variable_names: &[String]) -> Option { + fn get_variable_type(&self, variable_names: &[String]) -> Option { if variable_names.is_empty() { return None; } @@ -1275,7 +1275,7 @@ impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> { self.state.execution_props() } - fn get_data_type(&self, expr: &Expr) -> datafusion_common::Result { + fn get_data_type(&self, expr: &Expr) -> datafusion_common::Result { expr.get_type(self.df_schema) } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 99bb3881f758..6c51a73a53b6 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2271,7 +2271,7 @@ mod tests { use arrow::array::{ArrayRef, DictionaryArray, Int32Array}; use arrow::datatypes::{DataType, Field, Int32Type}; - use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_common::{assert_contains, DFSchemaRef, TableReference}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; @@ -2777,7 +2777,7 @@ mod tests { Self { schema: DFSchemaRef::new( DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", DataType::Int32, false)].into(), + vec![LogicalPhysicalField::new("a", DataType::Int32, false)].into(), HashMap::new(), ) .unwrap(), diff --git a/datafusion/core/src/test/variable.rs b/datafusion/core/src/test/variable.rs index ab0de4a11e24..55915ede3ac5 100644 --- a/datafusion/core/src/test/variable.rs +++ b/datafusion/core/src/test/variable.rs @@ -21,7 +21,7 @@ use crate::error::Result; use crate::scalar::ScalarValue; use crate::variable::VarProvider; use arrow_schema::DataType; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; /// System variable #[derive(Default, Debug)] @@ -41,7 +41,7 @@ impl VarProvider for SystemVar { Ok(ScalarValue::from(s)) } - fn get_type(&self, _: &[String]) -> Option { + fn get_type(&self, _: &[String]) -> Option { Some(DataType::Utf8.into()) } } @@ -68,7 +68,7 @@ impl VarProvider for UserDefinedVar { } } - fn get_type(&self, var_names: &[String]) -> Option { + fn get_type(&self, var_names: &[String]) -> Option { if var_names[0] != "@integer" { Some(DataType::Utf8.into()) } else { diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 87af4c42093f..6e6d52422219 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -49,7 +49,7 @@ use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use futures::Stream; use tempfile::TempDir; // backwards compatibility @@ -203,8 +203,8 @@ impl TableProvider for TestTableProvider { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 03bf63f0320d..e44a35e1a2ca 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -38,7 +38,7 @@ use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig, SessionContext}; use crate::datasource::physical_plan::parquet::ParquetExecBuilder; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; @@ -155,7 +155,7 @@ impl TestParquetFile { }); let df_schema = - LogicalSchema::from(self.schema.as_ref().clone()).to_dfschema_ref()?; + LogicalPhysicalSchema::from(self.schema.as_ref().clone()).to_dfschema_ref()?; // run coercion on the filters to coerce types etc. let props = ExecutionProps::new(); diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index 75cc827b879e..7f36c2634fd5 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -43,7 +43,7 @@ use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::{ExecutionMode, PlanProperties}; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use futures::stream::Stream; mod provider_filter_pushdown; @@ -204,8 +204,8 @@ impl TableProvider for CustomTableProvider { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(TEST_CUSTOM_SCHEMA_REF!().as_ref().clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(TEST_CUSTOM_SCHEMA_REF!().as_ref().clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index d29225cc9f15..0362a9e51e96 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -39,7 +39,7 @@ use datafusion_functions_aggregate::expr_fn::count; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; fn create_batch(value: i32, num_rows: usize) -> Result { let mut builder = Int32Builder::with_capacity(num_rows); @@ -153,8 +153,8 @@ impl TableProvider for CustomProvider { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.zero_batch.schema().clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.zero_batch.schema().clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index aa7e1089d8f9..bd13f62dae22 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -36,7 +36,7 @@ use datafusion_common::{project_schema, stats::Precision}; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; /// This is a testing structure for statistics /// It will act both as a table provider and execution plan @@ -80,8 +80,8 @@ impl TableProvider for StatisticsValidation { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 3cc170d9fe79..9874b01e067d 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -47,7 +47,7 @@ use datafusion::prelude::JoinType; use datafusion::prelude::{CsvReadOptions, ParquetReadOptions}; use datafusion::test_util::{parquet_test_data, populate_csv_partitions}; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{assert_contains, DataFusionError, ScalarValue, UnnestOptions}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; @@ -1928,7 +1928,7 @@ impl VarProvider for HardcodedIntProvider { Ok(ScalarValue::Int64(Some(1234))) } - fn get_type(&self, _: &[String]) -> Option { + fn get_type(&self, _: &[String]) -> Option { Some(DataType::Int64.into()) } } diff --git a/datafusion/core/tests/expr_api/parse_sql_expr.rs b/datafusion/core/tests/expr_api/parse_sql_expr.rs index 7938ae9a8185..f2e021f48018 100644 --- a/datafusion/core/tests/expr_api/parse_sql_expr.rs +++ b/datafusion/core/tests/expr_api/parse_sql_expr.rs @@ -17,8 +17,8 @@ use arrow_schema::DataType; use datafusion::prelude::{CsvReadOptions, SessionContext}; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::{DFSchemaRef, Result, ToDFSchema}; use datafusion_expr::Expr; use datafusion_sql::unparser::Unparser; @@ -29,10 +29,10 @@ use datafusion_sql::unparser::Unparser; /// b: Int32 /// s: Float32 fn schema() -> DFSchemaRef { - LogicalSchema::new(vec![ - LogicalField::new("a", DataType::Int32, true), - LogicalField::new("b", DataType::Int32, false), - LogicalField::new("c", DataType::Float32, false), + LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("a", DataType::Int32, true), + LogicalPhysicalField::new("b", DataType::Int32, false), + LogicalPhysicalField::new("c", DataType::Float32, false), ]) .to_dfschema_ref() .unwrap() diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 60f342c25cb8..83ce741039d8 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -23,10 +23,10 @@ use arrow_buffer::IntervalDayTime; use chrono::{DateTime, TimeZone, Utc}; use datafusion::{error::Result, execution::context::ExecutionProps, prelude::*}; use datafusion_common::cast::as_int32_array; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::ScalarValue; use datafusion_common::{DFSchemaRef, ToDFSchema}; use datafusion_expr::expr::ScalarFunction; @@ -72,7 +72,7 @@ impl SimplifyInfo for MyInfo { &self.execution_props } - fn get_data_type(&self, expr: &Expr) -> Result { + fn get_data_type(&self, expr: &Expr) -> Result { expr.get_type(self.schema.as_ref()) } } @@ -92,10 +92,10 @@ impl From for MyInfo { /// b: Int32 /// s: Utf8 fn schema() -> DFSchemaRef { - LogicalSchema::new(vec![ - LogicalField::new("a", DataType::Int32, true), - LogicalField::new("b", DataType::Int32, false), - LogicalField::new("s", DataType::Utf8, false), + LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("a", DataType::Int32, true), + LogicalPhysicalField::new("b", DataType::Int32, false), + LogicalPhysicalField::new("s", DataType::Utf8, false), ]) .to_dfschema_ref() .unwrap() @@ -487,15 +487,15 @@ fn multiple_now() -> Result<()> { // ------------------------------ fn expr_test_schema() -> DFSchemaRef { - LogicalSchema::new(vec![ - LogicalField::new("c1", DataType::Utf8, true), - LogicalField::new("c2", DataType::Boolean, true), - LogicalField::new("c3", DataType::Int64, true), - LogicalField::new("c4", DataType::UInt32, true), - LogicalField::new("c1_non_null", DataType::Utf8, false), - LogicalField::new("c2_non_null", DataType::Boolean, false), - LogicalField::new("c3_non_null", DataType::Int64, false), - LogicalField::new("c4_non_null", DataType::UInt32, false), + LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("c1", DataType::Utf8, true), + LogicalPhysicalField::new("c2", DataType::Boolean, true), + LogicalPhysicalField::new("c3", DataType::Int64, true), + LogicalPhysicalField::new("c4", DataType::UInt32, true), + LogicalPhysicalField::new("c1_non_null", DataType::Utf8, false), + LogicalPhysicalField::new("c2_non_null", DataType::Boolean, false), + LogicalPhysicalField::new("c3_non_null", DataType::Int64, false), + LogicalPhysicalField::new("c4_non_null", DataType::UInt32, false), ]) .to_dfschema_ref() .unwrap() diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 1c7bf0391ade..01eb235c7403 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -43,7 +43,7 @@ use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use datafusion_common::{assert_contains, Result}; use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_execution::TaskContext; use test_utils::AccessLogGenerator; @@ -733,8 +733,8 @@ impl TableProvider for SortedTableProvider { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/tests/optimizer_integration.rs b/datafusion/core/tests/optimizer_integration.rs index 72c03ef2daf7..c955864eb05a 100644 --- a/datafusion/core/tests/optimizer_integration.rs +++ b/datafusion/core/tests/optimizer_integration.rs @@ -25,8 +25,8 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use arrow_schema::{Fields, SchemaBuilder}; use datafusion_common::config::ConfigOptions; -use datafusion_common::logical_type::schema::LogicalSchemaRef; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{plan_err, DFSchema, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::{Interval, NullableInterval}; @@ -206,7 +206,7 @@ impl ContextProvider for MyContextProvider { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -240,8 +240,8 @@ impl TableSource for MyTableSource { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } } @@ -262,7 +262,7 @@ fn test_nested_schema_nullability() { let dfschema = DFSchema::from_field_specific_qualified_schema( vec![Some("table_name".into()), None], - &LogicalSchemaRef::new(schema.into()), + &LogicalPhysicalSchemaRef::new(schema.into()), ) .unwrap(); diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 31fb84d26624..a2509d6c311f 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -32,7 +32,7 @@ use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; @@ -67,7 +67,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { extensions: None, }; - let df_schema = LogicalSchema::from(schema.as_ref().clone()) + let df_schema = LogicalPhysicalSchema::from(schema.as_ref().clone()) .to_dfschema() .unwrap(); let execution_props = ExecutionProps::new(); diff --git a/datafusion/core/tests/sql/create_drop.rs b/datafusion/core/tests/sql/create_drop.rs index 6934fba3d634..f609288f3539 100644 --- a/datafusion/core/tests/sql/create_drop.rs +++ b/datafusion/core/tests/sql/create_drop.rs @@ -19,7 +19,7 @@ use super::*; use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::test_util::TestTableFactory; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; #[tokio::test] async fn create_custom_table() -> Result<()> { diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 7005552b479b..45cbf4a8e4a8 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -25,7 +25,7 @@ use datafusion::execution::context::{FunctionFactory, RegisterFunction, SessionS use datafusion::prelude::*; use datafusion::{execution::registry::FunctionRegistry, test_util}; use datafusion_common::cast::{as_float64_array, as_int32_array}; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, assert_contains, exec_err, internal_err, diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index 457068d59b2f..10e76f6cd872 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -29,7 +29,7 @@ use datafusion::execution::TaskContext; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::SessionContext; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; use std::fs::File; @@ -118,8 +118,8 @@ impl TableProvider for SimpleCsvTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } fn table_type(&self) -> TableType { diff --git a/datafusion/expr/src/conditional_expressions.rs b/datafusion/expr/src/conditional_expressions.rs index 15cc9967b64d..02aac54b23f1 100644 --- a/datafusion/expr/src/conditional_expressions.rs +++ b/datafusion/expr/src/conditional_expressions.rs @@ -19,7 +19,7 @@ use crate::expr::Case; use crate::{expr_schema::ExprSchemable, Expr}; use arrow::datatypes::DataType; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{plan_err, DFSchema, Result}; use std::collections::HashSet; @@ -71,7 +71,7 @@ impl CaseBuilder { then_expr.push(e.as_ref().to_owned()); } - let then_types: Vec = then_expr + let then_types: Vec = then_expr .iter() .map(|e| match e { Expr::Literal(_) => e.get_type(&DFSchema::empty()), @@ -82,7 +82,7 @@ impl CaseBuilder { if then_types.contains(&DataType::Null.into()) { // cannot verify types until execution type } else { - let unique_types: HashSet<&TypeRelation> = then_types.iter().collect(); + let unique_types: HashSet<&LogicalPhysicalType> = then_types.iter().collect(); if unique_types.len() != 1 { return plan_err!( "CASE expression 'then' values had multiple data types: {unique_types:?}" diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index ef078663dfdd..46878fd2c489 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -34,8 +34,8 @@ use crate::{ use crate::{window_frame, Volatility}; use arrow::datatypes::DataType; -use datafusion_common::logical_type::field::LogicalFieldRef; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::field::LogicalPhysicalFieldRef; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -225,7 +225,7 @@ pub enum Expr { /// A named reference to a qualified filed in a schema. Column(Column), /// A named reference to a variable in a registry. - ScalarVariable(TypeRelation, Vec), + ScalarVariable(LogicalPhysicalType, Vec), /// A constant value. Literal(ScalarValue), /// A binary expression such as "age > 21" @@ -319,7 +319,7 @@ pub enum Expr { Placeholder(Placeholder), /// A place holder which hold a reference to a qualified field /// in the outer query, used for correlated sub queries. - OuterReferenceColumn(TypeRelation, Column), + OuterReferenceColumn(LogicalPhysicalType, Column), /// Unnest expression Unnest(Unnest), } @@ -341,8 +341,8 @@ impl From for Expr { /// useful for creating [`Expr`] from a [`DFSchema`]. /// /// See example on [`Expr`] -impl<'a> From<(Option<&'a TableReference>, &'a LogicalFieldRef)> for Expr { - fn from(value: (Option<&'a TableReference>, &'a LogicalFieldRef)) -> Self { +impl<'a> From<(Option<&'a TableReference>, &'a LogicalPhysicalFieldRef)> for Expr { + fn from(value: (Option<&'a TableReference>, &'a LogicalPhysicalFieldRef)) -> Self { Expr::from(Column::from(value)) } } @@ -566,12 +566,12 @@ pub struct Cast { /// The expression being cast pub expr: Box, /// The `LogicalType` the expression will yield - pub data_type: TypeRelation, + pub data_type: LogicalPhysicalType, } impl Cast { /// Create a new Cast expression - pub fn new(expr: Box, data_type: impl Into) -> Self { + pub fn new(expr: Box, data_type: impl Into) -> Self { Self { expr, data_type: data_type.into(), @@ -585,12 +585,12 @@ pub struct TryCast { /// The expression being cast pub expr: Box, /// The `LogicalType` the expression will yield - pub data_type: TypeRelation, + pub data_type: LogicalPhysicalType, } impl TryCast { /// Create a new TryCast expression - pub fn new(expr: Box, data_type: impl Into) -> Self { + pub fn new(expr: Box, data_type: impl Into) -> Self { Self { expr, data_type: data_type.into(), @@ -939,12 +939,12 @@ pub struct Placeholder { /// The identifier of the parameter, including the leading `$` (e.g, `"$1"` or `"$foo"`) pub id: String, /// The type the parameter will be filled in with - pub data_type: Option, + pub data_type: Option, } impl Placeholder { /// Create a new Placeholder expression - pub fn new(id: String, data_type: Option) -> Self { + pub fn new(id: String, data_type: Option) -> Self { Self { id, data_type } } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index dbd3fd78a2f7..02167cab30c6 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -35,7 +35,7 @@ use arrow::compute::kernels::cast_utils::{ parse_interval_day_time, parse_interval_month_day_nano, parse_interval_year_month, }; use arrow::datatypes::{DataType, Field}; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{Column, Result, ScalarValue}; use std::any::Any; use std::fmt::Debug; @@ -63,7 +63,7 @@ pub fn col(ident: impl Into) -> Expr { /// Create an out reference column which hold a reference that has been resolved to a field /// outside of the current plan. -pub fn out_ref_col(dt: impl Into, ident: impl Into) -> Expr { +pub fn out_ref_col(dt: impl Into, ident: impl Into) -> Expr { Expr::OuterReferenceColumn(dt.into(), ident.into()) } @@ -309,12 +309,12 @@ pub fn rollup(exprs: Vec) -> Expr { } /// Create a cast expression -pub fn cast(expr: Expr, data_type: impl Into) -> Expr { +pub fn cast(expr: Expr, data_type: impl Into) -> Expr { Expr::Cast(Cast::new(Box::new(expr), data_type.into())) } /// Create a try cast expression -pub fn try_cast(expr: Expr, data_type: impl Into) -> Expr { +pub fn try_cast(expr: Expr, data_type: impl Into) -> Expr { Expr::TryCast(TryCast::new(Box::new(expr), data_type.into())) } diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 60e94f285e09..8827ac955999 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -288,8 +288,8 @@ mod test { use crate::expr::Sort; use crate::{col, lit, Cast}; use arrow::datatypes::DataType; - use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::logical_type::field::LogicalPhysicalField; + use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::ScalarValue; #[derive(Default)] @@ -410,9 +410,9 @@ mod test { ) -> DFSchema { let fields = fields .iter() - .map(|f| Arc::new(LogicalField::new(f.to_string(), DataType::Int8, false))) + .map(|f| Arc::new(LogicalPhysicalField::new(f.to_string(), DataType::Int8, false))) .collect::>(); - let schema = Arc::new(LogicalSchema::new(fields)); + let schema = Arc::new(LogicalPhysicalSchema::new(fields)); DFSchema::from_field_specific_qualified_schema(qualifiers, &schema.into()) .unwrap() } diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 26361b0c6ed7..7be6ce38e20f 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -27,9 +27,9 @@ use crate::type_coercion::functions::{ use crate::{utils, LogicalPlan, Projection, Subquery, WindowFunctionDefinition}; use arrow::compute::can_cast_types; use arrow::datatypes::DataType; -use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::{ internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, ExprSchema, Result, TableReference, @@ -40,7 +40,7 @@ use std::sync::Arc; /// trait to allow expr to typable with respect to a schema pub trait ExprSchemable { /// given a schema, return the type of the expr - fn get_type(&self, schema: &dyn ExprSchema) -> Result; + fn get_type(&self, schema: &dyn ExprSchema) -> Result; /// given a schema, return the nullability of the expr fn nullable(&self, input_schema: &dyn ExprSchema) -> Result; @@ -52,12 +52,12 @@ pub trait ExprSchemable { fn to_field( &self, input_schema: &dyn ExprSchema, - ) -> Result<(Option, Arc)>; + ) -> Result<(Option, Arc)>; /// cast to a type with respect to a schema fn cast_to( self, - cast_to_type: &TypeRelation, + cast_to_type: &LogicalPhysicalType, schema: &dyn ExprSchema, ) -> Result; @@ -65,7 +65,7 @@ pub trait ExprSchemable { fn data_type_and_nullable( &self, schema: &dyn ExprSchema, - ) -> Result<(TypeRelation, bool)>; + ) -> Result<(LogicalPhysicalType, bool)>; } impl ExprSchemable for Expr { @@ -107,7 +107,7 @@ impl ExprSchemable for Expr { /// expression refers to a column that does not exist in the /// schema, or when the expression is incorrectly typed /// (e.g. `[utf8] + [bool]`). - fn get_type(&self, schema: &dyn ExprSchema) -> Result { + fn get_type(&self, schema: &dyn ExprSchema) -> Result { match self { Expr::Alias(Alias { expr, name, .. }) => match &**expr { Expr::Placeholder(Placeholder { data_type, .. }) => match &data_type { @@ -431,7 +431,7 @@ impl ExprSchemable for Expr { fn data_type_and_nullable( &self, schema: &dyn ExprSchema, - ) -> Result<(TypeRelation, bool)> { + ) -> Result<(LogicalPhysicalType, bool)> { match self { Expr::Alias(Alias { expr, name, .. }) => match &**expr { Expr::Placeholder(Placeholder { data_type, .. }) => match &data_type { @@ -488,13 +488,13 @@ impl ExprSchemable for Expr { fn to_field( &self, input_schema: &dyn ExprSchema, - ) -> Result<(Option, Arc)> { + ) -> Result<(Option, Arc)> { match self { Expr::Column(c) => { let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; Ok(( c.relation.clone(), - LogicalField::new(&c.name, data_type, nullable) + LogicalPhysicalField::new(&c.name, data_type, nullable) .with_metadata(self.metadata(input_schema)?) .into(), )) @@ -503,7 +503,7 @@ impl ExprSchemable for Expr { let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; Ok(( relation.clone(), - LogicalField::new(name, data_type, nullable) + LogicalPhysicalField::new(name, data_type, nullable) .with_metadata(self.metadata(input_schema)?) .into(), )) @@ -512,7 +512,7 @@ impl ExprSchemable for Expr { let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; Ok(( None, - LogicalField::new(self.display_name()?, data_type, nullable) + LogicalPhysicalField::new(self.display_name()?, data_type, nullable) .with_metadata(self.metadata(input_schema)?) .into(), )) @@ -528,7 +528,7 @@ impl ExprSchemable for Expr { /// expression to the target [arrow::datatypes::DataType]. fn cast_to( self, - cast_to_type: &TypeRelation, + cast_to_type: &LogicalPhysicalType, schema: &dyn ExprSchema, ) -> Result { let this_type = self.get_type(schema)?; @@ -557,7 +557,7 @@ impl ExprSchemable for Expr { /// cast subquery in InSubquery/ScalarSubquery to a given type. pub fn cast_subquery( subquery: Subquery, - cast_to_type: &TypeRelation, + cast_to_type: &LogicalPhysicalType, ) -> Result { if subquery.subquery.schema().field(0).data_type() == cast_to_type { return Ok(subquery); @@ -721,7 +721,7 @@ mod tests { ); let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("foo", DataType::Int32, true) + vec![LogicalPhysicalField::new("foo", DataType::Int32, true) .with_metadata(meta.clone())] .into(), HashMap::new(), @@ -735,7 +735,7 @@ mod tests { #[derive(Debug)] struct MockExprSchema { nullable: bool, - data_type: TypeRelation, + data_type: LogicalPhysicalType, error_on_nullable: bool, metadata: HashMap, } @@ -755,7 +755,7 @@ mod tests { self } - fn with_data_type(mut self, data_type: TypeRelation) -> Self { + fn with_data_type(mut self, data_type: LogicalPhysicalType) -> Self { self.data_type = data_type; self } @@ -780,7 +780,7 @@ mod tests { } } - fn data_type(&self, _col: &Column) -> Result<&TypeRelation> { + fn data_type(&self, _col: &Column) -> Result<&LogicalPhysicalType> { Ok(&self.data_type) } @@ -788,7 +788,7 @@ mod tests { Ok(&self.metadata) } - fn data_type_and_nullable(&self, col: &Column) -> Result<(&TypeRelation, bool)> { + fn data_type_and_nullable(&self, col: &Column) -> Result<(&LogicalPhysicalType, bool)> { Ok((self.data_type(col)?, self.nullable(col)?)) } } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index e7a8d494e637..f28efd43affc 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -51,11 +51,11 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::fields::LogicalPhysicalFields; +use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::{ get_target_functional_dependencies, internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, @@ -187,9 +187,9 @@ impl LogicalPlanBuilder { } let empty_schema = DFSchema::empty(); - let mut field_types: Vec = Vec::with_capacity(n_cols); + let mut field_types: Vec = Vec::with_capacity(n_cols); for j in 0..n_cols { - let mut common_type: Option = None; + let mut common_type: Option = None; for (i, row) in values.iter().enumerate() { let value = &row[j]; let data_type = value.get_type(&empty_schema)?; @@ -227,7 +227,7 @@ impl LogicalPlanBuilder { .map(|(j, data_type)| { // naming is following convention https://www.postgresql.org/docs/current/queries-values.html let name = &format!("column{}", j + 1); - LogicalField::new(name, data_type.clone(), true) + LogicalPhysicalField::new(name, data_type.clone(), true) }) .collect::>(); let dfschema = DFSchema::from_unqualified_fields(fields.into(), HashMap::new())?; @@ -296,7 +296,7 @@ impl LogicalPlanBuilder { pub fn insert_into( input: LogicalPlan, table_name: impl Into, - table_schema: &LogicalSchema, + table_schema: &LogicalPhysicalSchema, overwrite: bool, ) -> Result { let table_schema = table_schema.clone().to_dfschema_ref()?; @@ -390,7 +390,7 @@ impl LogicalPlanBuilder { } /// Make a builder for a prepare logical plan from the builder's plan - pub fn prepare(self, name: String, data_types: Vec) -> Result { + pub fn prepare(self, name: String, data_types: Vec) -> Result { Ok(Self::from(LogicalPlan::Prepare(Prepare { name, data_types, @@ -1188,7 +1188,7 @@ impl From> for LogicalPlanBuilder { } } -pub fn change_redundant_column(fields: &LogicalFields) -> Vec { +pub fn change_redundant_column(fields: &LogicalPhysicalFields) -> Vec { let mut name_map = HashMap::new(); fields .into_iter() @@ -1197,7 +1197,7 @@ pub fn change_redundant_column(fields: &LogicalFields) -> Vec { *counter += 1; if *counter > 1 { let new_name = format!("{}:{}", field.name(), *counter - 1); - LogicalField::new( + LogicalPhysicalField::new( new_name, field.data_type().clone(), field.is_nullable(), @@ -1216,8 +1216,8 @@ pub fn build_join_schema( join_type: &JoinType, ) -> Result { fn nullify_fields<'a>( - fields: impl Iterator, &'a Arc)>, - ) -> Vec<(Option, Arc)> { + fields: impl Iterator, &'a Arc)>, + ) -> Vec<(Option, Arc)> { fields .map(|(q, f)| { // TODO: find a good way to do that @@ -1230,7 +1230,7 @@ pub fn build_join_schema( let right_fields = right.iter(); let left_fields = left.iter(); - let qualified_fields: Vec<(Option, Arc)> = + let qualified_fields: Vec<(Option, Arc)> = match join_type { JoinType::Inner => { // left then right @@ -1409,7 +1409,7 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result LogicalSchemaRef { - LogicalSchemaRef::new(self.table_schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.table_schema.clone().into()) } fn supports_filters_pushdown( @@ -1622,13 +1622,13 @@ pub fn unnest(input: LogicalPlan, columns: Vec) -> Result { // - Struct(field1, field2) returns ["a.field1","a.field2"] pub fn get_unnested_columns( col_name: &String, - data_type: &TypeRelation, -) -> Result)>> { + data_type: &LogicalPhysicalType, +) -> Result)>> { let mut qualified_columns = Vec::with_capacity(1); match data_type.logical() { LogicalType::List(field) => { - let new_field = Arc::new(LogicalField::new( + let new_field = Arc::new(LogicalPhysicalField::new( col_name.clone(), field.data_type().clone(), // Unnesting may produce NULLs even if the list is not null. @@ -1701,7 +1701,7 @@ pub fn unnest_with_options( .extend(std::iter::repeat(index).take(flatten_columns.len())); Ok(flatten_columns .iter() - .map(|col: &(Column, Arc)| { + .map(|col: &(Column, Arc)| { (col.0.relation.to_owned(), col.1.to_owned()) }) .collect()) @@ -1743,7 +1743,7 @@ mod tests { use crate::logical_plan::StringifiedPlan; use crate::{col, expr, expr_fn::exists, in_subquery, lit, scalar_subquery}; use arrow::datatypes::{DataType, Field, Fields}; - use datafusion_common::logical_type::ExtensionType; + use datafusion_common::logical_type::TypeRelation; use datafusion_common::SchemaError; @@ -2244,23 +2244,23 @@ mod tests { #[test] fn test_change_redundant_column() -> Result<()> { - let t1_field_1 = LogicalField::new("a", DataType::Int32, false); - let t2_field_1 = LogicalField::new("a", DataType::Int32, false); - let t2_field_3 = LogicalField::new("a", DataType::Int32, false); - let t1_field_2 = LogicalField::new("b", DataType::Int32, false); - let t2_field_2 = LogicalField::new("b", DataType::Int32, false); + let t1_field_1 = LogicalPhysicalField::new("a", DataType::Int32, false); + let t2_field_1 = LogicalPhysicalField::new("a", DataType::Int32, false); + let t2_field_3 = LogicalPhysicalField::new("a", DataType::Int32, false); + let t1_field_2 = LogicalPhysicalField::new("b", DataType::Int32, false); + let t2_field_2 = LogicalPhysicalField::new("b", DataType::Int32, false); let field_vec = vec![t1_field_1, t2_field_1, t1_field_2, t2_field_2, t2_field_3]; - let remove_redundant = change_redundant_column(&LogicalFields::from(field_vec)); + let remove_redundant = change_redundant_column(&LogicalPhysicalFields::from(field_vec)); assert_eq!( remove_redundant, vec![ - LogicalField::new("a", DataType::Int32, false), - LogicalField::new("a:1", DataType::Int32, false), - LogicalField::new("b", DataType::Int32, false), - LogicalField::new("b:1", DataType::Int32, false), - LogicalField::new("a:2", DataType::Int32, false), + LogicalPhysicalField::new("a", DataType::Int32, false), + LogicalPhysicalField::new("a:1", DataType::Int32, false), + LogicalPhysicalField::new("b", DataType::Int32, false), + LogicalPhysicalField::new("b:1", DataType::Int32, false), + LogicalPhysicalField::new("a:2", DataType::Int32, false), ] ); Ok(()) diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 71ecba456ce3..a5627b7c1f27 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -24,7 +24,7 @@ use std::{ use crate::{Expr, LogicalPlan, Volatility}; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{Constraints, DFSchemaRef, SchemaReference, TableReference}; use sqlparser::ast::Ident; @@ -322,7 +322,7 @@ pub struct CreateFunction { pub temporary: bool, pub name: String, pub args: Option>, - pub return_type: Option, + pub return_type: Option, pub params: CreateFunctionBody, /// Dummy schema pub schema: DFSchemaRef, @@ -332,7 +332,7 @@ pub struct OperateFunctionArg { // TODO: figure out how to support mode // pub mode: Option, pub name: Option, - pub data_type: TypeRelation, + pub data_type: LogicalPhysicalType, pub default_expr: Option, } #[derive(Clone, PartialEq, Eq, Hash, Debug)] diff --git a/datafusion/expr/src/logical_plan/dml.rs b/datafusion/expr/src/logical_plan/dml.rs index 4a8e6022f71a..87d41c2846ef 100644 --- a/datafusion/expr/src/logical_plan/dml.rs +++ b/datafusion/expr/src/logical_plan/dml.rs @@ -23,8 +23,8 @@ use std::sync::Arc; use crate::LogicalPlan; use arrow::datatypes::DataType; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::LogicalSchema; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::{DFSchemaRef, TableReference}; /// Operator that copies the contents of a database to file(s) @@ -131,7 +131,7 @@ impl Display for WriteOp { fn make_count_schema() -> DFSchemaRef { Arc::new( - LogicalSchema::new(vec![LogicalField::new("count", DataType::UInt64, false)]) + LogicalPhysicalSchema::new(vec![LogicalPhysicalField::new("count", DataType::UInt64, false)]) .try_into() .unwrap(), ) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index dac6c7e79f8e..bd101d41eaa6 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -55,9 +55,9 @@ use datafusion_common::{ use crate::display::PgJsonVisitor; use crate::logical_plan::tree_node::unwrap_arc; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; pub use datafusion_common::{JoinConstraint, JoinType}; /// A `LogicalPlan` is a node in a tree of relational operators (such as @@ -355,8 +355,8 @@ impl LogicalPlan { } /// Returns the (fixed) output schema for explain plans - pub fn explain_schema() -> LogicalSchemaRef { - LogicalSchemaRef::new( + pub fn explain_schema() -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new( Schema::new(vec![ Field::new("plan_type", DataType::Utf8, false), Field::new("plan", DataType::Utf8, false), @@ -366,7 +366,7 @@ impl LogicalPlan { } /// Returns the (fixed) output schema for `DESCRIBE` plans - pub fn describe_schema() -> LogicalSchema { + pub fn describe_schema() -> LogicalPhysicalSchema { Schema::new(vec![ Field::new("column_name", DataType::Utf8, false), Field::new("data_type", DataType::Utf8, false), @@ -1395,8 +1395,8 @@ impl LogicalPlan { /// Walk the logical plan, find any `Placeholder` tokens, and return a map of their IDs and DataTypes pub fn get_parameter_types( &self, - ) -> Result>, DataFusionError> { - let mut param_types: HashMap> = HashMap::new(); + ) -> Result>, DataFusionError> { + let mut param_types: HashMap> = HashMap::new(); self.apply_with_subqueries(|plan| { plan.apply_expressions(|expr| { @@ -2226,7 +2226,7 @@ pub struct Window { impl Window { /// Create a new window operator. pub fn try_new(window_expr: Vec, input: Arc) -> Result { - let fields: Vec<(Option, Arc)> = input + let fields: Vec<(Option, Arc)> = input .schema() .iter() .map(|(q, f)| (q.cloned(), Arc::clone(f))) @@ -2440,7 +2440,7 @@ pub struct Prepare { /// The name of the statement pub name: String, /// Data types of the parameters ([`Expr::Placeholder`]) - pub data_types: Vec, + pub data_types: Vec, /// The logical plan of the statements pub input: Arc, } diff --git a/datafusion/expr/src/planner.rs b/datafusion/expr/src/planner.rs index 12e16c231229..a115a8984f86 100644 --- a/datafusion/expr/src/planner.rs +++ b/datafusion/expr/src/planner.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use crate::{AggregateUDF, Expr, GetFieldAccess, ScalarUDF, TableSource, WindowUDF}; use arrow::datatypes::SchemaRef; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{ config::ConfigOptions, file_options::file_type::FileType, not_impl_err, DFSchema, Result, TableReference, @@ -67,7 +67,7 @@ pub trait ContextProvider { /// Getter for a UDWF fn get_window_meta(&self, name: &str) -> Option>; /// Getter for system/user-defined variable type - fn get_variable_type(&self, variable_names: &[String]) -> Option; + fn get_variable_type(&self, variable_names: &[String]) -> Option; /// Get configuration options fn options(&self) -> &ConfigOptions; diff --git a/datafusion/expr/src/simplify.rs b/datafusion/expr/src/simplify.rs index 440aca75f07a..909147f3221b 100644 --- a/datafusion/expr/src/simplify.rs +++ b/datafusion/expr/src/simplify.rs @@ -19,7 +19,7 @@ use crate::{execution_props::ExecutionProps, Expr, ExprSchemable}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::{DFSchemaRef, DataFusionError, Result}; /// Provides the information necessary to apply algebraic simplification to an @@ -39,7 +39,7 @@ pub trait SimplifyInfo { fn execution_props(&self) -> &ExecutionProps; /// Returns data type of this expr needed for determining optimized int type of a value - fn get_data_type(&self, expr: &Expr) -> Result; + fn get_data_type(&self, expr: &Expr) -> Result; } /// Provides simplification information based on DFSchema and @@ -96,7 +96,7 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { } /// Returns data type of this expr needed for determining optimized int type of a value - fn get_data_type(&self, expr: &Expr) -> Result { + fn get_data_type(&self, expr: &Expr) -> Result { let schema = self.schema.as_ref().ok_or_else(|| { DataFusionError::Internal( "attempt to get data type without schema".to_string(), diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index e671c84a3fca..1cb8f2d3aaf2 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -19,7 +19,7 @@ use crate::{Expr, LogicalPlan}; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use datafusion_common::{Constraints, Result}; use std::any::Any; @@ -86,7 +86,7 @@ pub trait TableSource: Sync + Send { fn as_any(&self) -> &dyn Any; /// Get a reference to the schema for this table - fn schema(&self) -> LogicalSchemaRef; + fn schema(&self) -> LogicalPhysicalSchemaRef; /// Get primary key indices, if one exists. fn constraints(&self) -> Option<&Constraints> { diff --git a/datafusion/expr/src/type_coercion/mod.rs b/datafusion/expr/src/type_coercion/mod.rs index 3e610c77e3c8..b3243d2b2b21 100644 --- a/datafusion/expr/src/type_coercion/mod.rs +++ b/datafusion/expr/src/type_coercion/mod.rs @@ -37,11 +37,11 @@ pub mod functions; pub mod other; use datafusion_common::logical_type::{ - signature::LogicalType, ExtensionType, TypeRelation, + signature::LogicalType, TypeRelation, LogicalPhysicalType, }; /// Determine whether the given data type `dt` represents signed numeric values. -pub fn is_signed_numeric(dt: &TypeRelation) -> bool { +pub fn is_signed_numeric(dt: &LogicalPhysicalType) -> bool { use LogicalType::*; matches!( dt.logical(), @@ -57,22 +57,22 @@ pub fn is_signed_numeric(dt: &TypeRelation) -> bool { } /// Determine whether the given data type `dt` is `Null`. -pub fn is_null(dt: &TypeRelation) -> bool { +pub fn is_null(dt: &LogicalPhysicalType) -> bool { *dt.logical() == LogicalType::Null } /// Determine whether the given data type `dt` is a `Timestamp`. -pub fn is_timestamp(dt: &TypeRelation) -> bool { +pub fn is_timestamp(dt: &LogicalPhysicalType) -> bool { matches!(dt.logical(), LogicalType::Timestamp(_, _)) } /// Determine whether the given data type 'dt' is a `Interval`. -pub fn is_interval(dt: &TypeRelation) -> bool { +pub fn is_interval(dt: &LogicalPhysicalType) -> bool { matches!(dt.logical(), LogicalType::Interval(_)) } /// Determine whether the given data type `dt` is a `Date` or `Timestamp`. -pub fn is_datetime(dt: &TypeRelation) -> bool { +pub fn is_datetime(dt: &LogicalPhysicalType) -> bool { matches!( dt.logical(), LogicalType::Date | LogicalType::Timestamp(_, _) @@ -80,12 +80,12 @@ pub fn is_datetime(dt: &TypeRelation) -> bool { } /// Determine whether the given data type `dt` is a `Utf8` or `LargeUtf8`. -pub fn is_utf8_or_large_utf8(dt: &TypeRelation) -> bool { +pub fn is_utf8_or_large_utf8(dt: &LogicalPhysicalType) -> bool { matches!(dt.logical(), LogicalType::Utf8) } /// Determine whether the given data type `dt` is a `Decimal`. -pub fn is_decimal(dt: &TypeRelation) -> bool { +pub fn is_decimal(dt: &LogicalPhysicalType) -> bool { matches!( dt.logical(), LogicalType::Decimal128(_, _) | LogicalType::Decimal256(_, _) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 8ac1e7282293..503112deb36e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -39,9 +39,9 @@ use datafusion_common::{ ScalarValue, TableReference, }; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::schema::LogicalSchema; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; /// The value to which `COUNT(*)` is expanded to in @@ -433,7 +433,7 @@ pub fn expand_qualified_wildcard( return plan_err!("Invalid qualifier {qualifier}"); } - let qualified_schema = Arc::new(LogicalSchema::new(fields_with_qualified)); + let qualified_schema = Arc::new(LogicalPhysicalSchema::new(fields_with_qualified)); let qualified_dfschema = DFSchema::try_from_qualified_schema(qualifier.clone(), &qualified_schema)? .with_functional_dependencies(projected_func_dependencies)?; @@ -731,7 +731,7 @@ pub fn from_plan( pub fn exprlist_to_fields<'a>( exprs: impl IntoIterator, plan: &LogicalPlan, -) -> Result, Arc)>> { +) -> Result, Arc)>> { // look for exact match in plan's output schema let input_schema = &plan.schema(); exprs @@ -834,7 +834,7 @@ pub(crate) fn find_column_indexes_referenced_by_expr( /// can this data type be used in hash join equal conditions?? /// data types here come from function 'equal_rows', if more data types are supported /// in equal_rows(hash join), add those data types here to generate join logical plan. -pub fn can_hash(data_type: &TypeRelation) -> bool { +pub fn can_hash(data_type: &LogicalPhysicalType) -> bool { use LogicalType::*; match data_type.logical() { Null => true, diff --git a/datafusion/expr/src/var_provider.rs b/datafusion/expr/src/var_provider.rs index 5d2b198d6342..62150d637b1e 100644 --- a/datafusion/expr/src/var_provider.rs +++ b/datafusion/expr/src/var_provider.rs @@ -17,7 +17,7 @@ //! Variable provider -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{Result, ScalarValue}; /// Variable type, system/user defined @@ -35,7 +35,7 @@ pub trait VarProvider: std::fmt::Debug { fn get_value(&self, var_names: Vec) -> Result; /// Return the type of the given variable - fn get_type(&self, var_names: &[String]) -> Option; + fn get_type(&self, var_names: &[String]) -> Option; } pub fn is_system_variables(variable_names: &[String]) -> bool { diff --git a/datafusion/functions-array/src/planner.rs b/datafusion/functions-array/src/planner.rs index bf0c1d6786e4..29ca30aa5bf1 100644 --- a/datafusion/functions-array/src/planner.rs +++ b/datafusion/functions-array/src/planner.rs @@ -17,7 +17,7 @@ //! SQL planning extensions like [`ArrayFunctionPlanner`] and [`FieldAccessPlanner`] -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{utils::list_ndims, DFSchema, Result}; use datafusion_expr::{ planner::{ExprPlanner, PlannerResult, RawBinaryExpr, RawFieldAccessExpr}, diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index 44d32b97a7ea..4a28b47d7054 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -21,7 +21,7 @@ use arrow::array::{ use arrow::datatypes::DataType; use datafusion_common::cast::{as_map_array, as_struct_array}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ exec_err, plan_datafusion_err, plan_err, ExprSchema, Result, ScalarValue, }; diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index 9e98023eeafe..39718cbbb73c 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -17,7 +17,7 @@ use arrow::array::StructArray; use arrow::datatypes::{DataType, Field, Fields}; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; use datafusion_expr::{ColumnarValue, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index d7f0c6a7e54f..b4fc8fd46ff5 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -24,7 +24,7 @@ use super::power::PowerFunc; use arrow::array::{ArrayRef, Float32Array, Float64Array}; use arrow::datatypes::DataType; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ exec_err, internal_err, plan_datafusion_err, plan_err, DataFusionError, Result, ScalarValue, diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index ddc546552342..fb4ce2607fd2 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -19,7 +19,7 @@ use arrow::datatypes::{ArrowNativeTypeOp, DataType}; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ arrow_datafusion_err, exec_datafusion_err, exec_err, plan_datafusion_err, DataFusionError, Result, ScalarValue, diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index 9e814de73101..40893fa948ef 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -106,8 +106,8 @@ mod tests { use crate::test::assert_analyzed_plan_eq; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; + use datafusion_common::logical_type::field::LogicalPhysicalField; + use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, TableSource}; pub struct RawTableSource {} @@ -117,10 +117,10 @@ mod tests { self } - fn schema(&self) -> LogicalSchemaRef { - Arc::new(LogicalSchema::new(vec![ - LogicalField::new("a", DataType::Int64, false), - LogicalField::new("b", DataType::Int64, false), + fn schema(&self) -> LogicalPhysicalSchemaRef { + Arc::new(LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("a", DataType::Int64, false), + LogicalPhysicalField::new("b", DataType::Int64, false), ])) } @@ -161,7 +161,7 @@ mod tests { Ok(datafusion_expr::TableProviderFilterPushDown::Exact) } - fn schema(&self) -> LogicalSchemaRef { + fn schema(&self) -> LogicalPhysicalSchemaRef { Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]).into()) } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index c79451275c71..82166331dfab 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -23,7 +23,7 @@ use arrow::datatypes::{DataType, IntervalUnit}; use datafusion_common::config::ConfigOptions; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, @@ -484,7 +484,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { /// Casts the given `value` to `target_type`. Note that this function /// only considers `Null` or `Utf8` values. -fn coerce_scalar(target_type: &TypeRelation, value: &ScalarValue) -> Result { +fn coerce_scalar(target_type: &LogicalPhysicalType, value: &ScalarValue) -> Result { match value { // Coerce Utf8 values: ScalarValue::Utf8(Some(val)) => { @@ -511,7 +511,7 @@ fn coerce_scalar(target_type: &TypeRelation, value: &ScalarValue) -> Result Result { coerce_scalar(target_type, &value).or_else(|err| { @@ -530,7 +530,7 @@ fn coerce_scalar_range_aware( /// This function returns the widest type in the family of `given_type`. /// If the given type is already the widest type, it returns `None`. /// For example, if `given_type` is `Int8`, it returns `Int64`. -fn get_widest_type_in_family(given_type: &TypeRelation) -> Option { +fn get_widest_type_in_family(given_type: &LogicalPhysicalType) -> Option { use LogicalType::*; match given_type.logical() { UInt8 | UInt16 | UInt32 => Some(DataType::UInt64.into()), @@ -542,7 +542,7 @@ fn get_widest_type_in_family(given_type: &TypeRelation) -> Option /// Coerces the given (window frame) `bound` to `target_type`. fn coerce_frame_bound( - target_type: &TypeRelation, + target_type: &LogicalPhysicalType, bound: WindowFrameBound, ) -> Result { match bound { @@ -811,8 +811,8 @@ mod test { use arrow::datatypes::DataType::Utf8; use arrow::datatypes::{DataType, Field, TimeUnit}; - use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::TypeRelation; + use datafusion_common::logical_type::field::LogicalPhysicalField; + use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{DFSchema, DFSchemaRef, Result, ScalarValue}; use datafusion_expr::expr::{self, InSubquery, Like, ScalarFunction}; @@ -838,12 +838,12 @@ mod test { })) } - fn empty_with_type(data_type: impl Into) -> Arc { + fn empty_with_type(data_type: impl Into) -> Arc { Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::new( DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", data_type, true)].into(), + vec![LogicalPhysicalField::new("a", data_type, true)].into(), std::collections::HashMap::new(), ) .unwrap(), @@ -1081,7 +1081,7 @@ mod test { let empty = Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", DataType::Decimal128(12, 4), true)].into(), + vec![LogicalPhysicalField::new("a", DataType::Decimal128(12, 4), true)].into(), std::collections::HashMap::new(), )?), })); @@ -1279,7 +1279,7 @@ mod test { fn test_type_coercion_rewrite() -> Result<()> { // gt let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", DataType::Int64, true)].into(), + vec![LogicalPhysicalField::new("a", DataType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; @@ -1290,7 +1290,7 @@ mod test { // eq let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", DataType::Int64, true)].into(), + vec![LogicalPhysicalField::new("a", DataType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; @@ -1301,7 +1301,7 @@ mod test { // lt let schema = Arc::new(DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", DataType::Int64, true)].into(), + vec![LogicalPhysicalField::new("a", DataType::Int64, true)].into(), std::collections::HashMap::new(), )?); let mut rewriter = TypeCoercionRewriter { schema: &schema }; @@ -1331,7 +1331,7 @@ mod test { fn cast_if_not_same_type( expr: Box, - data_type: &TypeRelation, + data_type: &LogicalPhysicalType, schema: &DFSchemaRef, ) -> Box { if &expr.get_type(schema).unwrap() != data_type { @@ -1343,8 +1343,8 @@ mod test { fn cast_helper( case: Case, - case_when_type: TypeRelation, - then_else_type: TypeRelation, + case_when_type: LogicalPhysicalType, + then_else_type: LogicalPhysicalType, schema: &DFSchemaRef, ) -> Case { let expr = case @@ -1375,23 +1375,23 @@ mod test { fn test_case_expression_coercion() -> Result<()> { let schema = Arc::new(DFSchema::from_unqualified_fields( vec![ - LogicalField::new("boolean", DataType::Boolean, true), - LogicalField::new("integer", DataType::Int32, true), - LogicalField::new("float", DataType::Float32, true), - LogicalField::new( + LogicalPhysicalField::new("boolean", DataType::Boolean, true), + LogicalPhysicalField::new("integer", DataType::Int32, true), + LogicalPhysicalField::new("float", DataType::Float32, true), + LogicalPhysicalField::new( "timestamp", DataType::Timestamp(TimeUnit::Nanosecond, None), true, ), - LogicalField::new("date", DataType::Date32, true), - LogicalField::new( + LogicalPhysicalField::new("date", DataType::Date32, true), + LogicalPhysicalField::new( "interval", DataType::Interval(arrow::datatypes::IntervalUnit::MonthDayNano), true, ), - LogicalField::new("binary", DataType::Binary, true), - LogicalField::new("string", DataType::Utf8, true), - LogicalField::new("decimal", DataType::Decimal128(10, 10), true), + LogicalPhysicalField::new("binary", DataType::Binary, true), + LogicalPhysicalField::new("string", DataType::Utf8, true), + LogicalPhysicalField::new("decimal", DataType::Decimal128(10, 10), true), ] .into(), std::collections::HashMap::new(), diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 406f56f0bc37..68be437f77e8 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1132,7 +1132,7 @@ mod test { use std::iter; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_expr::expr::AggregateFunction; use datafusion_expr::logical_plan::{table_scan, JoinType}; use datafusion_expr::{ @@ -1744,9 +1744,9 @@ mod test { let grouping = grouping_set(vec![vec![col("a"), col("b")], vec![col("c")]]); let schema = DFSchema::from_unqualified_fields( vec![ - LogicalField::new("a", DataType::Int32, false), - LogicalField::new("b", DataType::Int32, false), - LogicalField::new("c", DataType::Int32, false), + LogicalPhysicalField::new("a", DataType::Int32, false), + LogicalPhysicalField::new("b", DataType::Int32, false), + LogicalPhysicalField::new("c", DataType::Int32, false), ] .into(), HashMap::default(), @@ -1763,8 +1763,8 @@ mod test { let grouping = grouping_set(vec![vec![col("a"), col("b")], vec![col("a")]]); let schema = DFSchema::from_unqualified_fields( vec![ - LogicalField::new("a", DataType::Int32, false), - LogicalField::new("b", DataType::Int32, false), + LogicalPhysicalField::new("a", DataType::Int32, false), + LogicalPhysicalField::new("b", DataType::Int32, false), ] .into(), HashMap::default(), @@ -1831,7 +1831,7 @@ mod test { fn test_extract_expressions_from_col() -> Result<()> { let mut result = Vec::with_capacity(1); let schema = DFSchema::from_unqualified_fields( - vec![LogicalField::new("a", DataType::Int32, false)].into(), + vec![LogicalPhysicalField::new("a", DataType::Int32, false)].into(), HashMap::default(), )?; extract_expressions(&col("a"), &schema, &mut result)?; diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index cba594325baa..c0a6e7d9b581 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -65,7 +65,7 @@ mod tests { use super::*; use crate::test::*; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::ToDFSchema; use datafusion_expr::{ expr_rewriter::coerce_plan_expr_for_schema, logical_plan::table_scan, @@ -109,7 +109,7 @@ mod tests { fn eliminate_one_union() -> Result<()> { let table_plan = coerce_plan_expr_for_schema( &table_scan(Some("table"), &schema(), None)?.build()?, - &LogicalSchema::from(schema()).to_dfschema()?, + &LogicalPhysicalSchema::from(schema()).to_dfschema()?, )?; let schema = table_plan.schema().clone(); let single_union_plan = LogicalPlan::Union(Union { diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 9446c3c959e0..c2810718fd9a 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -807,7 +807,7 @@ mod tests { }; use crate::{OptimizerContext, OptimizerRule}; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_common::{ Column, DFSchema, DFSchemaRef, JoinType, Result, TableReference, }; @@ -1545,15 +1545,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(LogicalField::new("a", DataType::UInt32, false)) + Arc::new(LogicalPhysicalField::new("a", DataType::UInt32, false)) ), ( Some("test".into()), - Arc::new(LogicalField::new("b", DataType::UInt32, false)) + Arc::new(LogicalPhysicalField::new("b", DataType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(LogicalField::new("c1", DataType::UInt32, true)) + Arc::new(LogicalPhysicalField::new("c1", DataType::UInt32, true)) ), ], HashMap::new() @@ -1597,15 +1597,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(LogicalField::new("a", DataType::UInt32, false)) + Arc::new(LogicalPhysicalField::new("a", DataType::UInt32, false)) ), ( Some("test".into()), - Arc::new(LogicalField::new("b", DataType::UInt32, false)) + Arc::new(LogicalPhysicalField::new("b", DataType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(LogicalField::new("c1", DataType::UInt32, true)) + Arc::new(LogicalPhysicalField::new("c1", DataType::UInt32, true)) ), ], HashMap::new() @@ -1647,15 +1647,15 @@ mod tests { vec![ ( Some("test".into()), - Arc::new(LogicalField::new("a", DataType::UInt32, false)) + Arc::new(LogicalPhysicalField::new("a", DataType::UInt32, false)) ), ( Some("test".into()), - Arc::new(LogicalField::new("b", DataType::UInt32, false)) + Arc::new(LogicalPhysicalField::new("b", DataType::UInt32, false)) ), ( Some("test2".into()), - Arc::new(LogicalField::new("a", DataType::UInt32, true)) + Arc::new(LogicalPhysicalField::new("a", DataType::UInt32, true)) ), ], HashMap::new() diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 930906fd8405..f5ec4d428e27 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -250,7 +250,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Fields, Schema}; - use datafusion_common::logical_type::fields::LogicalFields; + use datafusion_common::logical_type::fields::LogicalPhysicalFields; use datafusion_common::{Column, DFSchema, JoinType, ScalarValue}; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ @@ -577,7 +577,7 @@ mod tests { fn test_empty_with_non_empty() -> Result<()> { let table_scan = test_table_scan()?; - let fields = LogicalFields::from(Fields::from(test_table_scan_fields())); + let fields = LogicalPhysicalFields::from(Fields::from(test_table_scan_fields())); let empty = LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 01a458757657..cfcd0ea33ef3 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -1199,8 +1199,8 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use async_trait::async_trait; - use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::schema::{LogicalSchema, LogicalSchemaRef}; + use datafusion_common::logical_type::field::LogicalPhysicalField; + use datafusion_common::logical_type::schema::{LogicalPhysicalSchema, LogicalPhysicalSchemaRef}; use datafusion_common::ScalarValue; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::table_scan; @@ -2406,10 +2406,10 @@ mod tests { #[async_trait] impl TableSource for PushDownProvider { - fn schema(&self) -> LogicalSchemaRef { - Arc::new(LogicalSchema::new(vec![ - LogicalField::new("a", DataType::Int32, true), - LogicalField::new("b", DataType::Int32, true), + fn schema(&self) -> LogicalPhysicalSchemaRef { + Arc::new(LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("a", DataType::Int32, true), + LogicalPhysicalField::new("b", DataType::Int32, true), ])) } @@ -2437,7 +2437,7 @@ mod tests { let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), filters: vec![], - projected_schema: Arc::new(DFSchema::try_from(LogicalSchema::from( + projected_schema: Arc::new(DFSchema::try_from(LogicalPhysicalSchema::from( (*test_provider.schema()).clone(), ))?), projection: None, @@ -2509,7 +2509,7 @@ mod tests { let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), filters: vec![col("a").eq(lit(10i64)), col("b").gt(lit(11i64))], - projected_schema: Arc::new(DFSchema::try_from(LogicalSchema::from( + projected_schema: Arc::new(DFSchema::try_from(LogicalPhysicalSchema::from( (*test_provider.schema()).clone(), ))?), projection: Some(vec![0]), @@ -2538,7 +2538,7 @@ mod tests { let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), filters: vec![], - projected_schema: Arc::new(DFSchema::try_from(LogicalSchema::from( + projected_schema: Arc::new(DFSchema::try_from(LogicalPhysicalSchema::from( (*test_provider.schema()).clone(), ))?), projection: Some(vec![0]), diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index ec4278a30330..2ec6960ac498 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -29,7 +29,7 @@ use arrow::{ use datafusion_common::{ cast::{as_large_list_array, as_list_array}, - logical_type::ExtensionType, + logical_type::TypeRelation, tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, }; use datafusion_common::{internal_err, DFSchema, DataFusionError, Result, ScalarValue}; @@ -1776,8 +1776,8 @@ fn inlist_except(mut l1: InList, l2: InList) -> Result { mod tests { use crate::simplify_expressions::SimplifyContext; use crate::test::test_table_scan_with_name; - use datafusion_common::logical_type::field::LogicalField; - use datafusion_common::logical_type::schema::LogicalSchema; + use datafusion_common::logical_type::field::LogicalPhysicalField; + use datafusion_common::logical_type::schema::LogicalPhysicalSchema; use datafusion_common::{ assert_contains, logical_type::signature::LogicalType, DFSchemaRef, ToDFSchema, }; @@ -1830,9 +1830,9 @@ mod tests { } fn test_schema() -> DFSchemaRef { - LogicalSchema::new(vec![ - LogicalField::new("i", DataType::Int64, false), - LogicalField::new("b", DataType::Boolean, true), + LogicalPhysicalSchema::new(vec![ + LogicalPhysicalField::new("i", DataType::Int64, false), + LogicalPhysicalField::new("b", DataType::Boolean, true), ]) .to_dfschema_ref() .unwrap() @@ -3015,14 +3015,14 @@ mod tests { Arc::new( DFSchema::from_unqualified_fields( vec![ - LogicalField::new("c1", DataType::Utf8, true), - LogicalField::new("c2", DataType::Boolean, true), - LogicalField::new("c3", DataType::Int64, true), - LogicalField::new("c4", DataType::UInt32, true), - LogicalField::new("c1_non_null", DataType::Utf8, false), - LogicalField::new("c2_non_null", DataType::Boolean, false), - LogicalField::new("c3_non_null", DataType::Int64, false), - LogicalField::new("c4_non_null", DataType::UInt32, false), + LogicalPhysicalField::new("c1", DataType::Utf8, true), + LogicalPhysicalField::new("c2", DataType::Boolean, true), + LogicalPhysicalField::new("c3", DataType::Int64, true), + LogicalPhysicalField::new("c4", DataType::UInt32, true), + LogicalPhysicalField::new("c1_non_null", DataType::Utf8, false), + LogicalPhysicalField::new("c2_non_null", DataType::Boolean, false), + LogicalPhysicalField::new("c3_non_null", DataType::Int64, false), + LogicalPhysicalField::new("c4_non_null", DataType::UInt32, false), ] .into(), HashMap::new(), diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 3d4a33a29458..f33ef5767afb 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -30,7 +30,7 @@ use arrow::datatypes::{ }; use arrow::temporal_conversions::{MICROSECONDS, MILLISECONDS, NANOSECONDS}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{internal_err, DFSchema, DFSchemaRef, Result, ScalarValue}; use datafusion_expr::expr::{BinaryExpr, Cast, InList, TryCast}; @@ -277,12 +277,12 @@ fn is_comparison_op(op: &Operator) -> bool { } /// Returns true if [UnwrapCastExprRewriter] supports this data type -fn is_supported_type(data_type: &TypeRelation) -> bool { +fn is_supported_type(data_type: &LogicalPhysicalType) -> bool { is_supported_numeric_type(data_type) || is_supported_string_type(data_type) } /// Returns true if [[UnwrapCastExprRewriter]] suppors this numeric type -fn is_supported_numeric_type(data_type: &TypeRelation) -> bool { +fn is_supported_numeric_type(data_type: &LogicalPhysicalType) -> bool { use LogicalType::*; matches!( data_type.logical(), @@ -300,14 +300,14 @@ fn is_supported_numeric_type(data_type: &TypeRelation) -> bool { } /// Returns true if [UnwrapCastExprRewriter] supports casting this value as a string -fn is_supported_string_type(data_type: &TypeRelation) -> bool { +fn is_supported_string_type(data_type: &LogicalPhysicalType) -> bool { matches!(data_type.logical(), LogicalType::Utf8) } /// Convert a literal value from one data type to another fn try_cast_literal_to_type( lit_value: &ScalarValue, - target_type: &TypeRelation, + target_type: &LogicalPhysicalType, ) -> Option { let lit_data_type = lit_value.data_type().into(); if !is_supported_type(&lit_data_type) || !is_supported_type(target_type) { @@ -324,7 +324,7 @@ fn try_cast_literal_to_type( /// Convert a numeric value from one numeric data type to another fn try_cast_numeric_literal( lit_value: &ScalarValue, - target_type: &TypeRelation, + target_type: &LogicalPhysicalType, ) -> Option { let lit_data_type = lit_value.data_type().into(); if !is_supported_numeric_type(&lit_data_type) @@ -464,7 +464,7 @@ fn try_cast_numeric_literal( fn try_cast_string_literal( lit_value: &ScalarValue, - target_type: &TypeRelation, + target_type: &LogicalPhysicalType, ) -> Option { let string_value = match lit_value { ScalarValue::Utf8(s) | ScalarValue::LargeUtf8(s) => s.clone(), @@ -479,8 +479,8 @@ fn try_cast_string_literal( /// Cast a timestamp value from one unit to another fn cast_between_timestamp( - from: impl Into, - to: impl Into, + from: impl Into, + to: impl Into, value: i128, ) -> Option { let value = value as i64; @@ -517,7 +517,7 @@ mod tests { compute::{cast_with_options, CastOptions}, datatypes::DataType, }; - use datafusion_common::logical_type::field::LogicalField; + use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_common::tree_node::TransformedResult; use datafusion_expr::{cast, col, in_list, try_cast}; @@ -776,16 +776,16 @@ mod tests { Arc::new( DFSchema::from_unqualified_fields( vec![ - LogicalField::new("c1", DataType::Int32, false), - LogicalField::new("c2", DataType::Int64, false), - LogicalField::new("c3", DataType::Decimal128(18, 2), false), - LogicalField::new("c4", DataType::Decimal128(38, 37), false), - LogicalField::new("c5", DataType::Float32, false), - LogicalField::new("c6", DataType::UInt32, false), - LogicalField::new("ts_nano_none", timestamp_nano_none_type(), false), - LogicalField::new("ts_nano_utf", timestamp_nano_utc_type(), false), - LogicalField::new("str1", DataType::Utf8, false), - LogicalField::new("largestr", DataType::LargeUtf8, false), + LogicalPhysicalField::new("c1", DataType::Int32, false), + LogicalPhysicalField::new("c2", DataType::Int64, false), + LogicalPhysicalField::new("c3", DataType::Decimal128(18, 2), false), + LogicalPhysicalField::new("c4", DataType::Decimal128(38, 37), false), + LogicalPhysicalField::new("c5", DataType::Float32, false), + LogicalPhysicalField::new("c6", DataType::UInt32, false), + LogicalPhysicalField::new("ts_nano_none", timestamp_nano_none_type(), false), + LogicalPhysicalField::new("ts_nano_utf", timestamp_nano_utc_type(), false), + LogicalPhysicalField::new("str1", DataType::Utf8, false), + LogicalPhysicalField::new("largestr", DataType::LargeUtf8, false), ] .into(), HashMap::new(), @@ -823,12 +823,12 @@ mod tests { lit(ScalarValue::Decimal128(None, precision, scale)) } - fn timestamp_nano_none_type() -> TypeRelation { + fn timestamp_nano_none_type() -> LogicalPhysicalType { DataType::Timestamp(TimeUnit::Nanosecond, None).into() } // this is the type that now() returns - fn timestamp_nano_utc_type() -> TypeRelation { + fn timestamp_nano_utc_type() -> LogicalPhysicalType { let utc = Some("+0:00".into()); DataType::Timestamp(TimeUnit::Nanosecond, utc).into() } @@ -1014,10 +1014,10 @@ mod tests { assert_eq!(lit_tz_none, lit_tz_utc); // e.g. LogicalType::Timestamp(_, None) - let dt_tz_none: TypeRelation = lit_tz_none.data_type().into(); + let dt_tz_none: LogicalPhysicalType = lit_tz_none.data_type().into(); // e.g. LogicalType::Timestamp(_, Some(utc)) - let dt_tz_utc: TypeRelation = lit_tz_utc.data_type().into(); + let dt_tz_utc: LogicalPhysicalType = lit_tz_utc.data_type().into(); // None <--> None expect_cast( @@ -1100,7 +1100,7 @@ mod tests { /// casting is consistent with the Arrow kernels fn expect_cast( literal: ScalarValue, - target_type: impl Into, + target_type: impl Into, expected_result: ExpectedCast, ) { let target_type = target_type.into(); diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index bf3fb12a439c..ae2b15165149 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -22,8 +22,8 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; use datafusion_common::config::ConfigOptions; -use datafusion_common::logical_type::schema::LogicalSchemaRef; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{plan_err, Result}; use datafusion_expr::test::function_stub::sum_udaf; use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF}; @@ -416,7 +416,7 @@ impl ContextProvider for MyContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } @@ -450,7 +450,7 @@ impl TableSource for MyTableSource { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.schema.clone().into()) } } diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index a371e249c03c..8da8dc931157 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -21,7 +21,7 @@ use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::Schema; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{exec_err, Result}; use datafusion_expr::expr::Alias; use datafusion_expr::sort_properties::ExprProperties; diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 723f3e38dc6c..713325a31306 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -24,8 +24,8 @@ use crate::{ }; use arrow::datatypes::Schema; -use datafusion_common::logical_type::schema::LogicalSchema; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ exec_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, ToDFSchema, }; @@ -371,7 +371,7 @@ where /// Convert a logical expression to a physical expression (without any simplification, etc) pub fn logical2physical(expr: &Expr, schema: &Schema) -> Arc { - let df_schema = LogicalSchema::from(schema.clone()).to_dfschema().unwrap(); + let df_schema = LogicalPhysicalSchema::from(schema.clone()).to_dfschema().unwrap(); let execution_props = ExecutionProps::new(); create_physical_expr(expr, &df_schema, &execution_props).unwrap() } diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index f5e1ac373e65..9006f4bd1357 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -30,8 +30,8 @@ use arrow::datatypes::{ }; use arrow::ipc::{reader::read_record_batch, root_as_message}; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{ arrow_datafusion_err, config::{ @@ -160,11 +160,11 @@ impl TryFrom<&protobuf::DfSchema> for DFSchema { df_schema: &protobuf::DfSchema, ) -> datafusion_common::Result { let df_fields = df_schema.columns.clone(); - let qualifiers_and_fields: Vec<(Option, Arc)> = + let qualifiers_and_fields: Vec<(Option, Arc)> = df_fields .iter() .map(|df_field| { - let field: LogicalField = + let field: LogicalPhysicalField = df_field.field.as_ref().required("field")?; Ok(( df_field @@ -194,7 +194,7 @@ impl TryFrom for DFSchemaRef { } } -impl TryFrom<&protobuf::ArrowType> for TypeRelation { +impl TryFrom<&protobuf::ArrowType> for LogicalPhysicalType { type Error = Error; fn try_from( @@ -346,7 +346,7 @@ impl TryFrom<&protobuf::Field> for Field { } } -impl TryFrom<&protobuf::Field> for LogicalField { +impl TryFrom<&protobuf::Field> for LogicalPhysicalField { type Error = Error; fn try_from(field: &protobuf::Field) -> Result { Field::try_from(field).map(|t| t.into()) diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index e06f604aef34..eb0ad074bbfe 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -28,7 +28,7 @@ use arrow::datatypes::{ SchemaRef, TimeUnit, UnionMode, }; use arrow::ipc::writer::{DictionaryTracker, IpcDataGenerator}; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::{ config::{ ColumnOptions, CsvOptions, JsonOptions, ParquetOptions, TableParquetOptions, @@ -113,10 +113,10 @@ impl TryFrom<&DataType> for protobuf::ArrowType { } } -impl TryFrom<&TypeRelation> for protobuf::ArrowType { +impl TryFrom<&LogicalPhysicalType> for protobuf::ArrowType { type Error = Error; - fn try_from(val: &TypeRelation) -> Result { + fn try_from(val: &LogicalPhysicalType) -> Result { let arrow_type_enum: ArrowTypeEnum = val.physical().try_into()?; Ok(Self { arrow_type_enum: Some(arrow_type_enum), diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index d9f9c468be9c..4fdd9e6c4424 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use datafusion::execution::registry::FunctionRegistry; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{ internal_err, plan_datafusion_err, DataFusionError, Result, ScalarValue, TableReference, UnnestOptions, @@ -552,7 +552,7 @@ pub fn parse_expr( "expr", codec, )?); - let data_type: TypeRelation = + let data_type: LogicalPhysicalType = cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::Cast(Cast::new(expr, data_type))) } @@ -563,7 +563,7 @@ pub fn parse_expr( "expr", codec, )?); - let data_type: TypeRelation = + let data_type: LogicalPhysicalType = cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::TryCast(TryCast::new(expr, data_type))) } diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 2ed27c61fbe6..3fe2baca2ea9 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -64,7 +64,7 @@ use datafusion_expr::{ }; use self::to_proto::serialize_expr; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_proto_common::ArrowType; use prost::bytes::BufMut; use prost::Message; @@ -837,7 +837,7 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Prepare(prepare) => { let input: LogicalPlan = into_logical_plan!(prepare.input, ctx, extension_codec)?; - let data_types: Vec = prepare + let data_types: Vec = prepare .data_types .iter() .map(|t| DataType::try_from(t).map(|t| t.into())) diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index e2d9d93dfb5e..ec9e014029a5 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -32,7 +32,7 @@ use datafusion::datasource::file_format::arrow::ArrowFormatFactory; use datafusion::datasource::file_format::csv::CsvFormatFactory; use datafusion::datasource::file_format::format_as_file_type; use datafusion::datasource::file_format::parquet::ParquetFormatFactory; -use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_proto::logical_plan::file_formats::{ ArrowLogicalExtensionCodec, CsvLogicalExtensionCodec, ParquetLogicalExtensionCodec, }; @@ -1612,12 +1612,12 @@ fn roundtrip_dfschema() { vec![ ( None, - Arc::new(LogicalField::new("a", DataType::Int64, false)), + Arc::new(LogicalPhysicalField::new("a", DataType::Int64, false)), ), ( Some("t".into()), Arc::new( - LogicalField::new("b", DataType::Decimal128(15, 2), true) + LogicalPhysicalField::new("b", DataType::Decimal128(15, 2), true) .with_metadata(HashMap::from([( String::from("k1"), String::from("v1"), diff --git a/datafusion/sql/examples/sql.rs b/datafusion/sql/examples/sql.rs index f2f83447b56f..c5888682d596 100644 --- a/datafusion/sql/examples/sql.rs +++ b/datafusion/sql/examples/sql.rs @@ -17,7 +17,7 @@ use arrow_schema::{DataType, Field, Schema}; use datafusion_common::config::ConfigOptions; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{plan_err, Result}; use datafusion_expr::WindowUDF; use datafusion_expr::{ @@ -133,7 +133,7 @@ impl ContextProvider for MyContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index f91433d25cce..034dee9c5747 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -17,7 +17,7 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::{ internal_datafusion_err, not_impl_err, plan_datafusion_err, plan_err, DFSchema, Dependency, Result, diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index 39d0da8cd206..f391b9ec02ea 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -16,7 +16,7 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::logical_type::field::LogicalField; +use datafusion_common::logical_type::field::LogicalPhysicalField; use datafusion_common::{ internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, ScalarValue, TableReference, @@ -280,7 +280,7 @@ fn search_dfschema<'ids, 'schema>( ids: &'ids [String], schema: &'schema DFSchema, ) -> Option<( - &'schema LogicalField, + &'schema LogicalPhysicalField, Option<&'schema TableReference>, &'ids [String], )> { diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 440ef679370b..d56414db36cf 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -25,7 +25,7 @@ use sqlparser::ast::{ Value, }; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, Result, ScalarValue, @@ -336,7 +336,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { { Expr::Cast(Cast::new( Box::new(expr), - TypeRelation::from(DataType::Timestamp( + LogicalPhysicalType::from(DataType::Timestamp( TimeUnit::Second, tz.clone(), )), @@ -617,7 +617,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context, )?), match *time_zone { - SQLExpr::Value(Value::SingleQuotedString(s)) => TypeRelation::from( + SQLExpr::Value(Value::SingleQuotedString(s)) => LogicalPhysicalType::from( DataType::Timestamp(TimeUnit::Nanosecond, Some(s.into())), ), _ => { @@ -1005,7 +1005,7 @@ mod tests { None } - fn get_variable_type(&self, _variable_names: &[String]) -> Option { + fn get_variable_type(&self, _variable_names: &[String]) -> Option { None } diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index 2bf50cec24e1..4c3db1a15d06 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -18,7 +18,7 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; use arrow::datatypes::DECIMAL128_MAX_PRECISION; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{ internal_err, not_impl_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -34,7 +34,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn parse_value( &self, value: Value, - param_data_types: &[TypeRelation], + param_data_types: &[LogicalPhysicalType], ) -> Result { match value { Value::Number(n, _) => self.parse_sql_number(&n, false), @@ -97,7 +97,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// number 1, 2, ... etc. For example, `$1` is the first placeholder; $2 is the second one and so on. fn create_placeholder_expr( param: String, - param_data_types: &[TypeRelation], + param_data_types: &[LogicalPhysicalType], ) -> Result { // Parse the placeholder as a number because it is the only support from sqlparser and postgres let index = param[1..].parse::(); diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 7a92a1bb7034..b903be8aee68 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -30,10 +30,10 @@ use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo}; use sqlparser::ast::{ColumnDef as SQLColumnDef, ColumnOption}; use sqlparser::ast::{DataType as SQLDataType, Ident, ObjectName, TableAlias}; -use datafusion_common::logical_type::field::LogicalField; -use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::schema::LogicalSchema; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::field::LogicalPhysicalField; +use datafusion_common::logical_type::fields::LogicalPhysicalFields; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::TableReference; use datafusion_common::{ not_impl_err, plan_err, unqualified_field_not_found, DFSchema, DataFusionError, @@ -107,7 +107,7 @@ impl IdentNormalizer { pub struct PlannerContext { /// Data types for numbered parameters ($1, $2, etc), if supplied /// in `PREPARE` statement - prepare_param_data_types: Arc>, + prepare_param_data_types: Arc>, /// Map of CTE name to logical plan of the WITH clause. /// Use `Arc` to allow cheap cloning ctes: HashMap>, @@ -134,7 +134,7 @@ impl PlannerContext { /// Update the PlannerContext with provided prepare_param_data_types pub fn with_prepare_param_data_types( mut self, - prepare_param_data_types: Vec, + prepare_param_data_types: Vec, ) -> Self { self.prepare_param_data_types = prepare_param_data_types.into(); self @@ -156,7 +156,7 @@ impl PlannerContext { } /// Return the types of parameters (`$1`, `$2`, etc) if known - pub fn prepare_param_data_types(&self) -> &[TypeRelation] { + pub fn prepare_param_data_types(&self) -> &[LogicalPhysicalType] { &self.prepare_param_data_types } @@ -218,7 +218,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - pub fn build_schema(&self, columns: Vec) -> Result { + pub fn build_schema(&self, columns: Vec) -> Result { let mut fields = Vec::with_capacity(columns.len()); for column in columns { @@ -227,14 +227,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .options .iter() .any(|x| x.option == ColumnOption::NotNull); - fields.push(LogicalField::new( + fields.push(LogicalPhysicalField::new( self.normalizer.normalize(column.name), data_type, !not_nullable, )); } - Ok(LogicalSchema::new(fields)) + Ok(LogicalPhysicalSchema::new(fields)) } /// Returns a vector of (column_name, default_expr) pairs @@ -341,13 +341,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn convert_data_type( &self, sql_type: &SQLDataType, - ) -> Result { + ) -> Result { match sql_type { SQLDataType::Array(ArrayElemTypeDef::AngleBracket(inner_sql_type)) | SQLDataType::Array(ArrayElemTypeDef::SquareBracket(inner_sql_type, _)) => { // Arrays may be multi-dimensional. let inner_data_type = self.convert_data_type(inner_sql_type)?; - Ok(TypeRelation::new_list(inner_data_type, true).into()) + Ok(LogicalPhysicalType::new_list(inner_data_type, true).into()) } SQLDataType::Array(ArrayElemTypeDef::None) => { not_impl_err!("Arrays with unspecified type is not supported") @@ -356,7 +356,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - fn convert_simple_data_type(&self, sql_type: &SQLDataType) -> Result { + fn convert_simple_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { SQLDataType::Boolean | SQLDataType::Bool => Ok(DataType::Boolean.into()), SQLDataType::TinyInt(_) => Ok(DataType::Int8.into()), @@ -431,14 +431,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Some(ident) => ident.clone(), None => Ident::new(format!("c{idx}")) }; - Ok(Arc::new(LogicalField::new( + Ok(Arc::new(LogicalPhysicalField::new( self.normalizer.normalize(field_name), data_type, true, ))) }) .collect::>>()?; - Ok(TypeRelation::new_struct(LogicalFields::from(fields))) + Ok(LogicalPhysicalType::new_struct(LogicalPhysicalFields::from(fields))) } // Explicitly list all other types so that if sqlparser // adds/changes the `SQLDataType` the compiler will tell us on upgrade diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 93b302f03e50..6c63a3be4830 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -29,9 +29,9 @@ use std::path::Path; use std::str::FromStr; use std::sync::Arc; -use datafusion_common::logical_type::fields::LogicalFields; -use datafusion_common::logical_type::schema::LogicalSchema; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::fields::LogicalPhysicalFields; +use datafusion_common::logical_type::schema::LogicalPhysicalSchema; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ exec_err, not_impl_err, plan_datafusion_err, plan_err, schema_err, @@ -455,7 +455,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { statement, } => { // Convert parser data types to DataFusion data types - let data_types: Vec = data_types + let data_types: Vec = data_types .into_iter() .map(|t| self.convert_data_type(&t)) .collect::>()?; @@ -1225,7 +1225,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_ref = self.object_name_to_table_reference(table_name.clone())?; let table_source = self.context_provider.get_table_source(table_ref.clone())?; - let schema: LogicalSchema = (*table_source.schema()).clone().into(); + let schema: LogicalPhysicalSchema = (*table_source.schema()).clone().into(); let schema = DFSchema::try_from(schema)?; let scan = LogicalPlanBuilder::scan( object_name_to_string(&table_name), @@ -1383,7 +1383,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_name = self.object_name_to_table_reference(table_name)?; let table_source = self.context_provider.get_table_source(table_name.clone())?; - let arrow_schema: LogicalSchema = (*table_source.schema()).clone().into(); + let arrow_schema: LogicalPhysicalSchema = (*table_source.schema()).clone().into(); let table_schema = DFSchema::try_from(arrow_schema)?; // Get insert fields and target table's value indices @@ -1421,7 +1421,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(table_schema.field(column_index).clone()) }) .collect::>>()?; - (LogicalFields::from(fields), value_indices) + (LogicalPhysicalFields::from(fields), value_indices) }; // infer types for Values clause... other types should be resolvable the regular way diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 811bc085ec0a..09a80df44712 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -34,7 +34,7 @@ use sqlparser::ast::{ UnaryOperator, }; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, Column, Result, ScalarValue, @@ -961,7 +961,7 @@ impl Unparser<'_> { fn arrow_dtype_to_ast_dtype( &self, - data_type: &TypeRelation, + data_type: &LogicalPhysicalType, ) -> Result { match data_type.logical() { LogicalType::Null => { diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 578ed0a929ca..2d7cc56753ad 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -23,7 +23,7 @@ use arrow_schema::{ DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, }; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::{ExtensionType, TypeRelation}; +use datafusion_common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -230,7 +230,7 @@ pub fn window_expr_common_partition_keys(window_exprs: &[Expr]) -> Result<&[Expr pub(crate) fn make_decimal_type( precision: Option, scale: Option, -) -> Result { +) -> Result { // postgres like behavior let (precision, scale) = match (precision, scale) { (Some(p), Some(s)) => (p as u8, s as i8), diff --git a/datafusion/sql/tests/common/mod.rs b/datafusion/sql/tests/common/mod.rs index d49b5f348b80..3ef172187718 100644 --- a/datafusion/sql/tests/common/mod.rs +++ b/datafusion/sql/tests/common/mod.rs @@ -24,8 +24,8 @@ use std::{sync::Arc, vec}; use arrow_schema::*; use datafusion_common::config::ConfigOptions; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::logical_type::schema::LogicalSchemaRef; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_common::{plan_err, GetExt, Result, TableReference}; use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource, WindowUDF}; use datafusion_sql::planner::ContextProvider; @@ -204,7 +204,7 @@ impl ContextProvider for MockContextProvider { self.udafs.get(name).cloned() } - fn get_variable_type(&self, _: &[String]) -> Option { + fn get_variable_type(&self, _: &[String]) -> Option { unimplemented!() } @@ -259,7 +259,7 @@ impl TableSource for EmptyTable { self } - fn schema(&self) -> LogicalSchemaRef { - LogicalSchemaRef::new(self.table_schema.clone().into()) + fn schema(&self) -> LogicalPhysicalSchemaRef { + LogicalPhysicalSchemaRef::new(self.table_schema.clone().into()) } } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index b8c69180f464..2c1a10370cb7 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -37,7 +37,7 @@ use datafusion_sql::{ planner::{ParserOptions, SqlToRel}, }; -use datafusion_common::logical_type::TypeRelation; +use datafusion_common::logical_type::LogicalPhysicalType; use datafusion_functions_aggregate::{ approx_median::approx_median_udaf, count::count_udaf, }; @@ -3648,8 +3648,8 @@ fn test_prepare_statement_should_infer_types() { let plan = logical_plan(sql).unwrap(); let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), - ("$2".to_string(), Some(TypeRelation::from(DataType::Int64))), + ("$1".to_string(), Some(LogicalPhysicalType::from(DataType::Int32))), + ("$2".to_string(), Some(LogicalPhysicalType::from(DataType::Int64))), ]); assert_eq!(actual_types, expected_types); } @@ -3662,7 +3662,7 @@ fn test_non_prepare_statement_should_infer_types() { let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ // constant 1 is inferred to be int64 - ("$1".to_string(), Some(TypeRelation::from(DataType::Int64))), + ("$1".to_string(), Some(LogicalPhysicalType::from(DataType::Int64))), ]); assert_eq!(actual_types, expected_types); } @@ -3838,7 +3838,7 @@ Projection: person.id, orders.order_id let actual_types = plan.get_parameter_types().unwrap(); let expected_types = - HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); + HashMap::from([("$1".to_string(), Some(LogicalPhysicalType::from(DataType::Int32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3871,7 +3871,7 @@ Projection: person.id, person.age let actual_types = plan.get_parameter_types().unwrap(); let expected_types = - HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::Int32)))]); + HashMap::from([("$1".to_string(), Some(LogicalPhysicalType::from(DataType::Int32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3903,8 +3903,8 @@ Projection: person.id, person.age let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), - ("$2".to_string(), Some(TypeRelation::from(DataType::Int32))), + ("$1".to_string(), Some(LogicalPhysicalType::from(DataType::Int32))), + ("$2".to_string(), Some(LogicalPhysicalType::from(DataType::Int32))), ]); assert_eq!(actual_types, expected_types); @@ -3943,7 +3943,7 @@ Projection: person.id, person.age let actual_types = plan.get_parameter_types().unwrap(); let expected_types = - HashMap::from([("$1".to_string(), Some(TypeRelation::from(DataType::UInt32)))]); + HashMap::from([("$1".to_string(), Some(LogicalPhysicalType::from(DataType::UInt32)))]); assert_eq!(actual_types, expected_types); // replace params with values @@ -3981,8 +3981,8 @@ Dml: op=[Update] table=[person] let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(TypeRelation::from(DataType::Int32))), - ("$2".to_string(), Some(TypeRelation::from(DataType::UInt32))), + ("$1".to_string(), Some(LogicalPhysicalType::from(DataType::Int32))), + ("$2".to_string(), Some(LogicalPhysicalType::from(DataType::UInt32))), ]); assert_eq!(actual_types, expected_types); @@ -4016,9 +4016,9 @@ fn test_prepare_statement_insert_infer() { let actual_types = plan.get_parameter_types().unwrap(); let expected_types = HashMap::from([ - ("$1".to_string(), Some(TypeRelation::from(DataType::UInt32))), - ("$2".to_string(), Some(TypeRelation::from(DataType::Utf8))), - ("$3".to_string(), Some(TypeRelation::from(DataType::Utf8))), + ("$1".to_string(), Some(LogicalPhysicalType::from(DataType::UInt32))), + ("$2".to_string(), Some(LogicalPhysicalType::from(DataType::Utf8))), + ("$3".to_string(), Some(LogicalPhysicalType::from(DataType::Utf8))), ]); assert_eq!(actual_types, expected_types); diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 9e54b5e9bde3..4791669b10b2 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -18,9 +18,9 @@ use arrow::util::display::ArrayFormatter; use arrow::{array, array::ArrayRef, datatypes::DataType, record_batch::RecordBatch}; use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; -use datafusion_common::logical_type::fields::LogicalFields; +use datafusion_common::logical_type::fields::LogicalPhysicalFields; use datafusion_common::logical_type::signature::LogicalType; -use datafusion_common::logical_type::ExtensionType; +use datafusion_common::logical_type::TypeRelation; use datafusion_common::DataFusionError; use std::path::PathBuf; use std::sync::OnceLock; @@ -245,7 +245,7 @@ pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { } /// Converts columns to a result as expected by sqllogicteset. -pub(crate) fn convert_schema_to_types(columns: &LogicalFields) -> Vec { +pub(crate) fn convert_schema_to_types(columns: &LogicalPhysicalFields) -> Vec { columns .iter() .map(|f| f.data_type()) diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 281f82da5544..1b5f19b29885 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -40,7 +40,7 @@ use datafusion_common::cast::as_float64_array; use datafusion_common::DataFusionError; use async_trait::async_trait; -use datafusion_common::logical_type::schema::LogicalSchemaRef; +use datafusion_common::logical_type::schema::LogicalPhysicalSchemaRef; use log::info; use tempfile::TempDir; @@ -216,7 +216,7 @@ pub async fn register_temp_table(ctx: &SessionContext) { self.0 } - fn schema(&self) -> LogicalSchemaRef { + fn schema(&self) -> LogicalPhysicalSchemaRef { unimplemented!() } diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 1bc62d474d37..0113166bb62e 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -21,9 +21,9 @@ use datafusion::arrow::array::GenericListArray; use datafusion::arrow::datatypes::{ DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, }; -use datafusion::common::logical_type::field::LogicalField; -use datafusion::common::logical_type::schema::LogicalSchema; -use datafusion::common::logical_type::{ExtensionType, TypeRelation}; +use datafusion::common::logical_type::field::LogicalPhysicalField; +use datafusion::common::logical_type::schema::LogicalPhysicalSchema; +use datafusion::common::logical_type::{TypeRelation, LogicalPhysicalType}; use datafusion::common::plan_err; use datafusion::common::{ not_impl_datafusion_err, not_impl_err, plan_datafusion_err, substrait_datafusion_err, @@ -361,7 +361,7 @@ fn make_renamed_schema( let mut name_idx = 0; - let (qualifiers, fields): (_, Vec) = schema + let (qualifiers, fields): (_, Vec) = schema .iter() .map(|(q, f)| { let name = next_struct_field_name(0, dfs_names, &mut name_idx)?; @@ -390,7 +390,7 @@ fn make_renamed_schema( DFSchema::from_field_specific_qualified_schema( qualifiers, - &Arc::new(LogicalSchema::new(fields)), + &Arc::new(LogicalPhysicalSchema::new(fields)), ) } @@ -1347,7 +1347,7 @@ pub async fn from_substrait_rex( } } -pub(crate) fn from_substrait_type_without_names(dt: &Type) -> Result { +pub(crate) fn from_substrait_type_without_names(dt: &Type) -> Result { from_substrait_type(dt, &[], &mut 0) } @@ -1355,7 +1355,7 @@ fn from_substrait_type( dt: &Type, dfs_names: &[String], name_idx: &mut usize, -) -> Result { +) -> Result { match &dt.kind { Some(s_kind) => match s_kind { r#type::Kind::Bool(_) => Ok(DataType::Boolean.into()), @@ -1930,7 +1930,7 @@ fn from_substrait_null( d.scale as i8, )), r#type::Kind::List(l) => { - let field = LogicalField::new_list_field( + let field = LogicalPhysicalField::new_list_field( from_substrait_type( l.r#type.clone().unwrap().as_ref(), dfs_names, diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 8e89198ca081..4a9fa680f23f 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -45,7 +45,7 @@ use crate::variation_const::{ TIMESTAMP_SECOND_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; -use datafusion::common::logical_type::ExtensionType; +use datafusion::common::logical_type::TypeRelation; use datafusion::common::{ exec_err, internal_err, not_impl_err, plan_err, substrait_datafusion_err, };