@@ -19,7 +19,7 @@ use crate::parquet::cast_column::CometCastColumnExpr;
1919use crate :: parquet:: parquet_support:: { spark_parquet_convert, SparkParquetOptions } ;
2020use arrow:: datatypes:: { DataType , Field , Schema , SchemaRef } ;
2121use datafusion:: common:: tree_node:: { Transformed , TransformedResult , TreeNode } ;
22- use datafusion:: common:: Result as DataFusionResult ;
22+ use datafusion:: common:: { DataFusionError , Result as DataFusionResult } ;
2323use datafusion:: physical_expr:: expressions:: Column ;
2424use datafusion:: physical_expr:: PhysicalExpr ;
2525use datafusion:: physical_plan:: ColumnarValue ;
@@ -95,12 +95,61 @@ fn remap_physical_schema_names(
9595 Arc :: new ( Schema :: new ( remapped_fields) )
9696}
9797
98+ /// Check if the logical (table) schema and physical (file) schema have type
99+ /// mismatches. Returns an error message describing the first mismatch found,
100+ /// or None if all types match.
101+ fn detect_schema_mismatch (
102+ logical_schema : & SchemaRef ,
103+ physical_schema : & SchemaRef ,
104+ case_sensitive : bool ,
105+ ) -> Option < String > {
106+ for logical_field in logical_schema. fields ( ) {
107+ let physical_field = if case_sensitive {
108+ physical_schema
109+ . fields ( )
110+ . iter ( )
111+ . find ( |f| f. name ( ) == logical_field. name ( ) )
112+ } else {
113+ physical_schema
114+ . fields ( )
115+ . iter ( )
116+ . find ( |f| f. name ( ) . to_lowercase ( ) == logical_field. name ( ) . to_lowercase ( ) )
117+ } ;
118+ if let Some ( physical_field) = physical_field {
119+ if logical_field. data_type ( ) != physical_field. data_type ( ) {
120+ return Some ( format ! (
121+ "Parquet column cannot be converted. \
122+ Column: [{}], Expected: {}, Found: {} \
123+ (schema evolution is disabled)",
124+ logical_field. name( ) ,
125+ logical_field. data_type( ) ,
126+ physical_field. data_type( )
127+ ) ) ;
128+ }
129+ }
130+ }
131+ None
132+ }
133+
98134impl PhysicalExprAdapterFactory for SparkPhysicalExprAdapterFactory {
99135 fn create (
100136 & self ,
101137 logical_file_schema : SchemaRef ,
102138 physical_file_schema : SchemaRef ,
103139 ) -> Arc < dyn PhysicalExprAdapter > {
140+ // When schema evolution is disabled, check for type mismatches between the
141+ // logical (table) schema and the physical (file) schema. If any column has
142+ // a different type, store the error to be raised during rewrite().
143+ let schema_mismatch_error = if !self . parquet_options . schema_evolution_enabled {
144+ detect_schema_mismatch (
145+ & logical_file_schema,
146+ & physical_file_schema,
147+ self . parquet_options . case_sensitive ,
148+ )
149+ } else {
150+ None
151+ } ;
152+
104153 // When case-insensitive, remap physical schema field names to match logical
105154 // field names. The DefaultPhysicalExprAdapter uses exact name matching, so
106155 // without this remapping, columns like "a" won't match logical "A" and will
@@ -154,6 +203,7 @@ impl PhysicalExprAdapterFactory for SparkPhysicalExprAdapterFactory {
154203 default_values : self . default_values . clone ( ) ,
155204 default_adapter,
156205 logical_to_physical_names,
206+ schema_mismatch_error,
157207 } )
158208 }
159209}
@@ -183,10 +233,18 @@ struct SparkPhysicalExprAdapter {
183233 /// physical names so that downstream reassign_expr_columns can find
184234 /// columns in the actual stream schema.
185235 logical_to_physical_names : Option < HashMap < String , String > > ,
236+ /// When schema evolution is disabled and file/table types differ, this
237+ /// holds the error message to return from rewrite().
238+ schema_mismatch_error : Option < String > ,
186239}
187240
188241impl PhysicalExprAdapter for SparkPhysicalExprAdapter {
189242 fn rewrite ( & self , expr : Arc < dyn PhysicalExpr > ) -> DataFusionResult < Arc < dyn PhysicalExpr > > {
243+ // When schema evolution is disabled and types differ, reject the read
244+ if let Some ( err) = & self . schema_mismatch_error {
245+ return Err ( DataFusionError :: Plan ( err. clone ( ) ) ) ;
246+ }
247+
190248 // First let the default adapter handle column remapping, missing columns,
191249 // and simple scalar type casts. Then replace DataFusion's CastColumnExpr
192250 // with Spark-compatible equivalents.
@@ -496,11 +554,51 @@ mod test {
496554 Ok ( ( ) )
497555 }
498556
557+ #[ tokio:: test]
558+ async fn parquet_schema_mismatch_rejected_when_evolution_disabled ( ) {
559+ let file_schema = Arc :: new ( Schema :: new ( vec ! [
560+ Field :: new( "id" , DataType :: Int32 , false ) ,
561+ Field :: new( "name" , DataType :: Utf8 , false ) ,
562+ ] ) ) ;
563+
564+ let ids = Arc :: new ( Int32Array :: from ( vec ! [ 1 , 2 , 3 ] ) ) as Arc < dyn arrow:: array:: Array > ;
565+ let names = Arc :: new ( StringArray :: from ( vec ! [ "Alice" , "Bob" , "Charlie" ] ) )
566+ as Arc < dyn arrow:: array:: Array > ;
567+ let batch = RecordBatch :: try_new ( Arc :: clone ( & file_schema) , vec ! [ ids, names] ) . unwrap ( ) ;
568+
569+ // Read as Int64 (widening) with schema evolution disabled
570+ let required_schema = Arc :: new ( Schema :: new ( vec ! [
571+ Field :: new( "id" , DataType :: Int64 , false ) ,
572+ Field :: new( "name" , DataType :: Utf8 , false ) ,
573+ ] ) ) ;
574+
575+ let result =
576+ roundtrip_with_schema_evolution ( & batch, required_schema. clone ( ) , false ) . await ;
577+ assert ! ( result. is_err( ) , "Expected error when schema evolution is disabled" ) ;
578+ let err_msg = result. unwrap_err ( ) . to_string ( ) ;
579+ assert ! (
580+ err_msg. contains( "schema evolution is disabled" ) ,
581+ "Error should mention schema evolution: {err_msg}"
582+ ) ;
583+
584+ // Same read with schema evolution enabled should succeed
585+ let result = roundtrip_with_schema_evolution ( & batch, required_schema, true ) . await ;
586+ assert ! ( result. is_ok( ) , "Expected success when schema evolution is enabled" ) ;
587+ }
588+
499589 /// Create a Parquet file containing a single batch and then read the batch back using
500590 /// the specified required_schema. This will cause the PhysicalExprAdapter code to be used.
501591 async fn roundtrip (
502592 batch : & RecordBatch ,
503593 required_schema : SchemaRef ,
594+ ) -> Result < RecordBatch , DataFusionError > {
595+ roundtrip_with_schema_evolution ( batch, required_schema, true ) . await
596+ }
597+
598+ async fn roundtrip_with_schema_evolution (
599+ batch : & RecordBatch ,
600+ required_schema : SchemaRef ,
601+ schema_evolution_enabled : bool ,
504602 ) -> Result < RecordBatch , DataFusionError > {
505603 let filename = get_temp_filename ( ) ;
506604 let filename = filename. as_path ( ) . as_os_str ( ) . to_str ( ) . unwrap ( ) . to_string ( ) ;
@@ -513,6 +611,7 @@ mod test {
513611
514612 let mut spark_parquet_options = SparkParquetOptions :: new ( EvalMode :: Legacy , "UTC" , false ) ;
515613 spark_parquet_options. allow_cast_unsigned_ints = true ;
614+ spark_parquet_options. schema_evolution_enabled = schema_evolution_enabled;
516615
517616 // Create expression adapter factory for Spark-compatible schema adaptation
518617 let expr_adapter_factory: Arc < dyn PhysicalExprAdapterFactory > = Arc :: new (
0 commit comments