From 5be9029e0148d4742072399bf3ac0a928c389d20 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Wed, 17 Jul 2024 14:13:26 +0200 Subject: [PATCH] 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, };