diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 43263196bb84..2aaf8ec0aa06 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -42,17 +42,18 @@ use crate::variation_const::{ DATE_32_TYPE_VARIATION_REF, DATE_64_TYPE_VARIATION_REF, DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME, LARGE_CONTAINER_TYPE_VARIATION_REF, - UNSIGNED_INTEGER_TYPE_VARIATION_REF, VIEW_CONTAINER_TYPE_VARIATION_REF, + LARGE_CONTAINER_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, }; #[allow(deprecated)] use crate::variation_const::{ - INTERVAL_DAY_TIME_TYPE_REF, INTERVAL_MONTH_DAY_NANO_TYPE_REF, - INTERVAL_YEAR_MONTH_TYPE_REF, TIMESTAMP_MICRO_TYPE_VARIATION_REF, - TIMESTAMP_MILLI_TYPE_VARIATION_REF, TIMESTAMP_NANO_TYPE_VARIATION_REF, - TIMESTAMP_SECOND_TYPE_VARIATION_REF, + INTERVAL_DAY_TIME_TYPE_REF, INTERVAL_MONTH_DAY_NANO_TYPE_NAME, + INTERVAL_MONTH_DAY_NANO_TYPE_REF, INTERVAL_YEAR_MONTH_TYPE_REF, + TIMESTAMP_MICRO_TYPE_VARIATION_REF, TIMESTAMP_MILLI_TYPE_VARIATION_REF, + TIMESTAMP_NANO_TYPE_VARIATION_REF, TIMESTAMP_SECOND_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{new_empty_array, AsArray}; +use datafusion::arrow::temporal_conversions::NANOSECONDS; use datafusion::common::scalar::ScalarStructBuilder; use datafusion::dataframe::DataFrame; use datafusion::logical_expr::expr::InList; @@ -71,10 +72,10 @@ use datafusion::{ use std::collections::HashSet; use std::sync::Arc; use substrait::proto::exchange_rel::ExchangeKind; -use substrait::proto::expression::literal::interval_day_to_second::PrecisionMode; use substrait::proto::expression::literal::user_defined::Val; use substrait::proto::expression::literal::{ - IntervalDayToSecond, IntervalYearToMonth, UserDefined, + interval_day_to_second, IntervalCompound, IntervalDayToSecond, IntervalYearToMonth, + UserDefined, }; use substrait::proto::expression::subquery::SubqueryType; use substrait::proto::expression::{FieldReference, Literal, ScalarFunction}; @@ -1845,9 +1846,14 @@ fn from_substrait_type( Ok(DataType::Interval(IntervalUnit::YearMonth)) } r#type::Kind::IntervalDay(_) => Ok(DataType::Interval(IntervalUnit::DayTime)), + r#type::Kind::IntervalCompound(_) => { + Ok(DataType::Interval(IntervalUnit::MonthDayNano)) + } r#type::Kind::UserDefined(u) => { if let Some(name) = extensions.types.get(&u.type_reference) { + #[allow(deprecated)] match name.as_ref() { + // Kept for backwards compatibility, producers should use IntervalCompound instead INTERVAL_MONTH_DAY_NANO_TYPE_NAME => Ok(DataType::Interval(IntervalUnit::MonthDayNano)), _ => not_impl_err!( "Unsupported Substrait user defined type with ref {} and variation {}", @@ -1856,18 +1862,17 @@ fn from_substrait_type( ), } } else { - // Kept for backwards compatibility, new plans should include the extension instead #[allow(deprecated)] match u.type_reference { - // Kept for backwards compatibility, use IntervalYear instead + // Kept for backwards compatibility, producers should use IntervalYear instead INTERVAL_YEAR_MONTH_TYPE_REF => { Ok(DataType::Interval(IntervalUnit::YearMonth)) } - // Kept for backwards compatibility, use IntervalDay instead + // Kept for backwards compatibility, producers should use IntervalDay instead INTERVAL_DAY_TIME_TYPE_REF => { Ok(DataType::Interval(IntervalUnit::DayTime)) } - // Not supported yet by Substrait + // Kept for backwards compatibility, producers should use IntervalCompound instead INTERVAL_MONTH_DAY_NANO_TYPE_REF => { Ok(DataType::Interval(IntervalUnit::MonthDayNano)) } @@ -2285,6 +2290,7 @@ fn from_substrait_literal( subseconds, precision_mode, })) => { + use interval_day_to_second::PrecisionMode; // DF only supports millisecond precision, so for any more granular type we lose precision let milliseconds = match precision_mode { Some(PrecisionMode::Microseconds(ms)) => ms / 1000, @@ -2309,6 +2315,35 @@ fn from_substrait_literal( Some(LiteralType::IntervalYearToMonth(IntervalYearToMonth { years, months })) => { ScalarValue::new_interval_ym(*years, *months) } + Some(LiteralType::IntervalCompound(IntervalCompound { + interval_year_to_month, + interval_day_to_second, + })) => match (interval_year_to_month, interval_day_to_second) { + ( + Some(IntervalYearToMonth { years, months }), + Some(IntervalDayToSecond { + days, + seconds, + subseconds, + precision_mode: + Some(interval_day_to_second::PrecisionMode::Precision(p)), + }), + ) => { + if *p < 0 || *p > 9 { + return plan_err!( + "Unsupported Substrait interval day to second precision: {}", + p + ); + } + let nanos = *subseconds * i64::pow(10, (9 - p) as u32); + ScalarValue::new_interval_mdn( + *years * 12 + months, + *days, + *seconds as i64 * NANOSECONDS + nanos, + ) + } + _ => return plan_err!("Substrait compound interval missing components"), + }, Some(LiteralType::FixedChar(c)) => ScalarValue::Utf8(Some(c.clone())), Some(LiteralType::UserDefined(user_defined)) => { // Helper function to prevent duplicating this code - can be inlined once the non-extension path is removed @@ -2339,6 +2374,8 @@ fn from_substrait_literal( if let Some(name) = extensions.types.get(&user_defined.type_reference) { match name.as_ref() { + // Kept for backwards compatibility - producers should use IntervalCompound instead + #[allow(deprecated)] INTERVAL_MONTH_DAY_NANO_TYPE_NAME => { interval_month_day_nano(user_defined)? } @@ -2351,10 +2388,9 @@ fn from_substrait_literal( } } } else { - // Kept for backwards compatibility - new plans should include extension instead #[allow(deprecated)] match user_defined.type_reference { - // Kept for backwards compatibility, use IntervalYearToMonth instead + // Kept for backwards compatibility, producers should useIntervalYearToMonth instead INTERVAL_YEAR_MONTH_TYPE_REF => { let Some(Val::Value(raw_val)) = user_defined.val.as_ref() else { return substrait_err!("Interval year month value is empty"); @@ -2369,7 +2405,7 @@ fn from_substrait_literal( value_slice, ))) } - // Kept for backwards compatibility, use IntervalDayToSecond instead + // Kept for backwards compatibility, producers should useIntervalDayToSecond instead INTERVAL_DAY_TIME_TYPE_REF => { let Some(Val::Value(raw_val)) = user_defined.val.as_ref() else { return substrait_err!("Interval day time value is empty"); @@ -2389,6 +2425,7 @@ fn from_substrait_literal( milliseconds, })) } + // Kept for backwards compatibility, producers should useIntervalCompound instead INTERVAL_MONTH_DAY_NANO_TYPE_REF => { interval_month_day_nano(user_defined)? } @@ -2738,3 +2775,52 @@ impl BuiltinExprBuilder { })) } } + +#[cfg(test)] +mod test { + use crate::extensions::Extensions; + use crate::logical_plan::consumer::from_substrait_literal_without_names; + use arrow_buffer::IntervalMonthDayNano; + use datafusion::error::Result; + use datafusion::scalar::ScalarValue; + use substrait::proto::expression::literal::{ + interval_day_to_second, IntervalCompound, IntervalDayToSecond, + IntervalYearToMonth, LiteralType, + }; + use substrait::proto::expression::Literal; + + #[test] + fn interval_compound_different_precision() -> Result<()> { + // DF producer (and thus roundtrip) always uses precision = 9, + // this test exists to test with some other value. + let substrait = Literal { + nullable: false, + type_variation_reference: 0, + literal_type: Some(LiteralType::IntervalCompound(IntervalCompound { + interval_year_to_month: Some(IntervalYearToMonth { + years: 1, + months: 2, + }), + interval_day_to_second: Some(IntervalDayToSecond { + days: 3, + seconds: 4, + subseconds: 5, + precision_mode: Some( + interval_day_to_second::PrecisionMode::Precision(6), + ), + }), + })), + }; + + assert_eq!( + from_substrait_literal_without_names(&substrait, &Extensions::default())?, + ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano { + months: 14, + days: 3, + nanoseconds: 4_000_005_000 + })) + ); + + Ok(()) + } +} diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 17ed41f016bd..408885f70687 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -21,7 +21,6 @@ use datafusion::optimizer::AnalyzerRule; use std::sync::Arc; use substrait::proto::expression_reference::ExprType; -use arrow_buffer::ToByteSlice; use datafusion::arrow::datatypes::{Field, IntervalUnit}; use datafusion::logical_expr::{ Distinct, FetchType, Like, Partitioning, SkipType, WindowFrameUnits, @@ -39,10 +38,11 @@ use crate::variation_const::{ DATE_32_TYPE_VARIATION_REF, DATE_64_TYPE_VARIATION_REF, DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME, LARGE_CONTAINER_TYPE_VARIATION_REF, - UNSIGNED_INTEGER_TYPE_VARIATION_REF, VIEW_CONTAINER_TYPE_VARIATION_REF, + LARGE_CONTAINER_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; +use datafusion::arrow::temporal_conversions::NANOSECONDS; use datafusion::common::{ exec_err, internal_err, not_impl_err, plan_err, substrait_datafusion_err, substrait_err, DFSchemaRef, ToDFSchema, @@ -58,8 +58,8 @@ use substrait::proto::exchange_rel::{ExchangeKind, RoundRobin, ScatterFields}; use substrait::proto::expression::literal::interval_day_to_second::PrecisionMode; use substrait::proto::expression::literal::map::KeyValue; use substrait::proto::expression::literal::{ - user_defined, IntervalDayToSecond, IntervalYearToMonth, List, Map, - PrecisionTimestamp, Struct, UserDefined, + IntervalCompound, IntervalDayToSecond, IntervalYearToMonth, List, Map, + PrecisionTimestamp, Struct, }; use substrait::proto::expression::subquery::InPredicate; use substrait::proto::expression::window_function::BoundsType; @@ -114,7 +114,7 @@ pub fn to_substrait_plan(plan: &LogicalPlan, ctx: &SessionContext) -> Result>>()?; - let substrait_schema = to_substrait_named_struct(schema, &mut extensions)?; + let substrait_schema = to_substrait_named_struct(schema)?; Ok(Box::new(ExtendedExpression { advanced_extensions: None, @@ -203,7 +203,7 @@ pub fn to_substrait_rel( }); let table_schema = scan.source.schema().to_dfschema_ref()?; - let base_schema = to_substrait_named_struct(&table_schema, extensions)?; + let base_schema = to_substrait_named_struct(&table_schema)?; Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { @@ -229,7 +229,7 @@ pub fn to_substrait_rel( Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { common: None, - base_schema: Some(to_substrait_named_struct(&e.schema, extensions)?), + base_schema: Some(to_substrait_named_struct(&e.schema)?), filter: None, best_effort_filter: None, projection: None, @@ -268,7 +268,7 @@ pub fn to_substrait_rel( Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { common: None, - base_schema: Some(to_substrait_named_struct(&v.schema, extensions)?), + base_schema: Some(to_substrait_named_struct(&v.schema)?), filter: None, best_effort_filter: None, projection: None, @@ -664,10 +664,7 @@ fn flatten_names(field: &Field, skip_self: bool, names: &mut Vec) -> Res Ok(()) } -fn to_substrait_named_struct( - schema: &DFSchemaRef, - extensions: &mut Extensions, -) -> Result { +fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { let mut names = Vec::with_capacity(schema.fields().len()); for field in schema.fields() { flatten_names(field, false, &mut names)?; @@ -677,7 +674,7 @@ fn to_substrait_named_struct( types: schema .fields() .iter() - .map(|f| to_substrait_type(f.data_type(), f.is_nullable(), extensions)) + .map(|f| to_substrait_type(f.data_type(), f.is_nullable())) .collect::>()?, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability: r#type::Nullability::Unspecified as i32, @@ -1150,7 +1147,7 @@ pub fn to_substrait_rex( Ok(Expression { rex_type: Some(RexType::Cast(Box::new( substrait::proto::expression::Cast { - r#type: Some(to_substrait_type(data_type, true, extensions)?), + r#type: Some(to_substrait_type(data_type, true)?), input: Some(Box::new(to_substrait_rex( ctx, expr, @@ -1356,11 +1353,7 @@ pub fn to_substrait_rex( } } -fn to_substrait_type( - dt: &DataType, - nullable: bool, - extensions: &mut Extensions, -) -> Result { +fn to_substrait_type(dt: &DataType, nullable: bool) -> Result { let nullability = if nullable { r#type::Nullability::Nullable as i32 } else { @@ -1489,16 +1482,14 @@ fn to_substrait_type( })), }), IntervalUnit::MonthDayNano => { - // Substrait doesn't currently support this type, so we represent it as a UDT Ok(substrait::proto::Type { - kind: Some(r#type::Kind::UserDefined(r#type::UserDefined { - type_reference: extensions.register_type( - INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string(), - ), - type_variation_reference: DEFAULT_TYPE_VARIATION_REF, - nullability, - type_parameters: vec![], - })), + kind: Some(r#type::Kind::IntervalCompound( + r#type::IntervalCompound { + type_variation_reference: DEFAULT_TYPE_VARIATION_REF, + nullability, + precision: 9, // nanos + }, + )), }) } } @@ -1547,8 +1538,7 @@ fn to_substrait_type( })), }), DataType::List(inner) => { - let inner_type = - to_substrait_type(inner.data_type(), inner.is_nullable(), extensions)?; + let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { r#type: Some(Box::new(inner_type)), @@ -1558,8 +1548,7 @@ fn to_substrait_type( }) } DataType::LargeList(inner) => { - let inner_type = - to_substrait_type(inner.data_type(), inner.is_nullable(), extensions)?; + let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { r#type: Some(Box::new(inner_type)), @@ -1573,12 +1562,10 @@ fn to_substrait_type( let key_type = to_substrait_type( key_and_value[0].data_type(), key_and_value[0].is_nullable(), - extensions, )?; let value_type = to_substrait_type( key_and_value[1].data_type(), key_and_value[1].is_nullable(), - extensions, )?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::Map(Box::new(r#type::Map { @@ -1594,9 +1581,7 @@ fn to_substrait_type( DataType::Struct(fields) => { let field_types = fields .iter() - .map(|field| { - to_substrait_type(field.data_type(), field.is_nullable(), extensions) - }) + .map(|field| to_substrait_type(field.data_type(), field.is_nullable())) .collect::>>()?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::Struct(r#type::Struct { @@ -1783,7 +1768,6 @@ fn to_substrait_literal( literal_type: Some(LiteralType::Null(to_substrait_type( &value.data_type(), true, - extensions, )?)), }); } @@ -1892,23 +1876,21 @@ fn to_substrait_literal( }), DEFAULT_TYPE_VARIATION_REF, ), - ScalarValue::IntervalMonthDayNano(Some(i)) => { - // IntervalMonthDayNano is internally represented as a 128-bit integer, containing - // months (32bit), days (32bit), and nanoseconds (64bit) - let bytes = i.to_byte_slice(); - ( - LiteralType::UserDefined(UserDefined { - type_reference: extensions - .register_type(INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string()), - type_parameters: vec![], - val: Some(user_defined::Val::Value(ProtoAny { - type_url: INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string(), - value: bytes.to_vec().into(), - })), + ScalarValue::IntervalMonthDayNano(Some(i)) => ( + LiteralType::IntervalCompound(IntervalCompound { + interval_year_to_month: Some(IntervalYearToMonth { + years: i.months / 12, + months: i.months % 12, }), - DEFAULT_TYPE_VARIATION_REF, - ) - } + interval_day_to_second: Some(IntervalDayToSecond { + days: i.days, + seconds: (i.nanoseconds / NANOSECONDS) as i32, + subseconds: i.nanoseconds % NANOSECONDS, + precision_mode: Some(PrecisionMode::Precision(9)), // nanoseconds + }), + }), + DEFAULT_TYPE_VARIATION_REF, + ), ScalarValue::IntervalDayTime(Some(i)) => ( LiteralType::IntervalDayToSecond(IntervalDayToSecond { days: i.days, @@ -1964,7 +1946,7 @@ fn to_substrait_literal( ), ScalarValue::Map(m) => { let map = if m.is_empty() || m.value(0).is_empty() { - let mt = to_substrait_type(m.data_type(), m.is_nullable(), extensions)?; + let mt = to_substrait_type(m.data_type(), m.is_nullable())?; let mt = match mt { substrait::proto::Type { kind: Some(r#type::Kind::Map(mt)), @@ -2049,11 +2031,7 @@ fn convert_array_to_literal_list( .collect::>>()?; if values.is_empty() { - let lt = match to_substrait_type( - array.data_type(), - array.is_nullable(), - extensions, - )? { + let lt = match to_substrait_type(array.data_type(), array.is_nullable())? { substrait::proto::Type { kind: Some(r#type::Kind::List(lt)), } => lt.as_ref().to_owned(), @@ -2179,7 +2157,6 @@ mod test { use datafusion::arrow::datatypes::{Field, Fields, Schema}; use datafusion::common::scalar::ScalarStructBuilder; use datafusion::common::DFSchema; - use std::collections::HashMap; #[test] fn round_trip_literals() -> Result<()> { @@ -2310,39 +2287,6 @@ mod test { Ok(()) } - #[test] - fn custom_type_literal_extensions() -> Result<()> { - let mut extensions = Extensions::default(); - // IntervalMonthDayNano is represented as a custom type in Substrait - let scalar = ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano::new( - 17, 25, 1234567890, - ))); - let substrait_literal = to_substrait_literal(&scalar, &mut extensions)?; - let roundtrip_scalar = - from_substrait_literal_without_names(&substrait_literal, &extensions)?; - assert_eq!(scalar, roundtrip_scalar); - - assert_eq!( - extensions, - Extensions { - functions: HashMap::new(), - types: HashMap::from([( - 0, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string() - )]), - type_variations: HashMap::new(), - } - ); - - // Check we fail if we don't propagate extensions - assert!(from_substrait_literal_without_names( - &substrait_literal, - &Extensions::default() - ) - .is_err()); - Ok(()) - } - #[test] fn round_trip_types() -> Result<()> { round_trip_type(DataType::Boolean)?; @@ -2414,50 +2358,17 @@ mod test { fn round_trip_type(dt: DataType) -> Result<()> { println!("Checking round trip of {dt:?}"); - let mut extensions = Extensions::default(); - // As DataFusion doesn't consider nullability as a property of the type, but field, // it doesn't matter if we set nullability to true or false here. - let substrait = to_substrait_type(&dt, true, &mut extensions)?; - let roundtrip_dt = from_substrait_type_without_names(&substrait, &extensions)?; + let substrait = to_substrait_type(&dt, true)?; + let roundtrip_dt = + from_substrait_type_without_names(&substrait, &Extensions::default())?; assert_eq!(dt, roundtrip_dt); Ok(()) } - #[test] - fn custom_type_extensions() -> Result<()> { - let mut extensions = Extensions::default(); - // IntervalMonthDayNano is represented as a custom type in Substrait - let dt = DataType::Interval(IntervalUnit::MonthDayNano); - - let substrait = to_substrait_type(&dt, true, &mut extensions)?; - let roundtrip_dt = from_substrait_type_without_names(&substrait, &extensions)?; - assert_eq!(dt, roundtrip_dt); - - assert_eq!( - extensions, - Extensions { - functions: HashMap::new(), - types: HashMap::from([( - 0, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string() - )]), - type_variations: HashMap::new(), - } - ); - - // Check we fail if we don't propagate extensions - assert!( - from_substrait_type_without_names(&substrait, &Extensions::default()) - .is_err() - ); - - Ok(()) - } - #[test] fn named_struct_names() -> Result<()> { - let mut extensions = Extensions::default(); let schema = DFSchemaRef::new(DFSchema::try_from(Schema::new(vec![ Field::new("int", DataType::Int32, true), Field::new( @@ -2472,7 +2383,7 @@ mod test { Field::new("trailer", DataType::Float64, true), ]))?); - let named_struct = to_substrait_named_struct(&schema, &mut extensions)?; + let named_struct = to_substrait_named_struct(&schema)?; // Struct field names should be flattened DFS style // List field names should be omitted @@ -2481,7 +2392,8 @@ mod test { vec!["int", "struct", "inner", "trailer"] ); - let roundtrip_schema = from_substrait_named_struct(&named_struct, &extensions)?; + let roundtrip_schema = + from_substrait_named_struct(&named_struct, &Extensions::default())?; assert_eq!(schema.as_ref(), &roundtrip_schema); Ok(()) } diff --git a/datafusion/substrait/src/variation_const.rs b/datafusion/substrait/src/variation_const.rs index a3e76389d510..58774db424da 100644 --- a/datafusion/substrait/src/variation_const.rs +++ b/datafusion/substrait/src/variation_const.rs @@ -96,7 +96,7 @@ pub const INTERVAL_DAY_TIME_TYPE_REF: u32 = 2; /// [`ScalarValue::IntervalMonthDayNano`]: datafusion::common::ScalarValue::IntervalMonthDayNano #[deprecated( since = "41.0.0", - note = "Use Substrait `UserDefinedType` with name `INTERVAL_MONTH_DAY_NANO_TYPE_NAME` instead" + note = "Use Substrait `IntervalCompund` type instead" )] pub const INTERVAL_MONTH_DAY_NANO_TYPE_REF: u32 = 3; @@ -104,4 +104,8 @@ pub const INTERVAL_MONTH_DAY_NANO_TYPE_REF: u32 = 3; /// /// [`DataType::Interval`]: datafusion::arrow::datatypes::DataType::Interval /// [`IntervalUnit::MonthDayNano`]: datafusion::arrow::datatypes::IntervalUnit::MonthDayNano +#[deprecated( + since = "43.0.0", + note = "Use Substrait `IntervalCompund` type instead" +)] pub const INTERVAL_MONTH_DAY_NANO_TYPE_NAME: &str = "interval-month-day-nano"; diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 8108b9ad6767..04530dd34d4b 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -39,10 +39,7 @@ use std::hash::Hash; use std::sync::Arc; use datafusion::execution::session_state::SessionStateBuilder; -use substrait::proto::extensions::simple_extension_declaration::{ - ExtensionType, MappingType, -}; -use substrait::proto::extensions::SimpleExtensionDeclaration; +use substrait::proto::extensions::simple_extension_declaration::MappingType; use substrait::proto::rel::RelType; use substrait::proto::{plan_rel, Plan, Rel}; @@ -230,23 +227,6 @@ async fn select_with_reused_functions() -> Result<()> { Ok(()) } -#[tokio::test] -async fn roundtrip_udt_extensions() -> Result<()> { - let ctx = create_context().await?; - let proto = - roundtrip_with_ctx("SELECT INTERVAL '1 YEAR 1 DAY 1 SECOND' FROM data", ctx) - .await?; - let expected_type = SimpleExtensionDeclaration { - mapping_type: Some(MappingType::ExtensionType(ExtensionType { - extension_uri_reference: u32::MAX, - type_anchor: 0, - name: "interval-month-day-nano".to_string(), - })), - }; - assert_eq!(proto.extensions, vec![expected_type]); - Ok(()) -} - #[tokio::test] async fn select_with_filter_date() -> Result<()> { roundtrip("SELECT * FROM data WHERE c > CAST('2020-01-01' AS DATE)").await