@@ -24,9 +24,18 @@ use std::ops::Range;
2424use std:: sync:: Arc ;
2525
2626use arrow:: array:: RecordBatch ;
27+ use arrow:: datatypes:: { Fields , Schema , SchemaRef , TimeUnit } ;
28+ use datafusion_datasource:: file_compression_type:: FileCompressionType ;
29+ use datafusion_datasource:: file_sink_config:: { FileSink , FileSinkConfig } ;
30+ use datafusion_datasource:: write:: { create_writer, get_writer_schema, SharedBuffer } ;
31+
32+ use datafusion_datasource:: file_format:: {
33+ FileFormat , FileFormatFactory , FilePushdownSupport ,
34+ } ;
35+ use datafusion_datasource:: write:: demux:: DemuxedStreamReceiver ;
36+
2737use arrow:: compute:: sum;
2838use arrow:: datatypes:: { DataType , Field , FieldRef } ;
29- use arrow:: datatypes:: { Fields , Schema , SchemaRef } ;
3039use datafusion_common:: config:: { ConfigField , ConfigFileType , TableParquetOptions } ;
3140use datafusion_common:: parsers:: CompressionTypeVariant ;
3241use datafusion_common:: stats:: Precision ;
@@ -38,15 +47,8 @@ use datafusion_common::{HashMap, Statistics};
3847use datafusion_common_runtime:: { JoinSet , SpawnedTask } ;
3948use datafusion_datasource:: display:: FileGroupDisplay ;
4049use datafusion_datasource:: file:: FileSource ;
41- use datafusion_datasource:: file_compression_type:: FileCompressionType ;
42- use datafusion_datasource:: file_format:: {
43- FileFormat , FileFormatFactory , FilePushdownSupport ,
44- } ;
4550use datafusion_datasource:: file_scan_config:: { FileScanConfig , FileScanConfigBuilder } ;
46- use datafusion_datasource:: file_sink_config:: { FileSink , FileSinkConfig } ;
4751use datafusion_datasource:: sink:: { DataSink , DataSinkExec } ;
48- use datafusion_datasource:: write:: demux:: DemuxedStreamReceiver ;
49- use datafusion_datasource:: write:: { create_writer, get_writer_schema, SharedBuffer } ;
5052use datafusion_execution:: memory_pool:: { MemoryConsumer , MemoryPool , MemoryReservation } ;
5153use datafusion_execution:: { SendableRecordBatchStream , TaskContext } ;
5254use datafusion_expr:: dml:: InsertOp ;
@@ -76,11 +78,13 @@ use parquet::arrow::arrow_writer::{
7678} ;
7779use parquet:: arrow:: async_reader:: MetadataFetch ;
7880use parquet:: arrow:: { parquet_to_arrow_schema, ArrowSchemaConverter , AsyncArrowWriter } ;
81+ use parquet:: basic:: Type ;
7982use parquet:: errors:: ParquetError ;
8083use parquet:: file:: metadata:: { ParquetMetaData , ParquetMetaDataReader , RowGroupMetaData } ;
8184use parquet:: file:: properties:: { WriterProperties , WriterPropertiesBuilder } ;
8285use parquet:: file:: writer:: SerializedFileWriter ;
8386use parquet:: format:: FileMetaData ;
87+ use parquet:: schema:: types:: SchemaDescriptor ;
8488use tokio:: io:: { AsyncWrite , AsyncWriteExt } ;
8589use tokio:: sync:: mpsc:: { self , Receiver , Sender } ;
8690
@@ -268,6 +272,15 @@ impl ParquetFormat {
268272 self . options . global . binary_as_string = binary_as_string;
269273 self
270274 }
275+
276+ pub fn coerce_int96 ( & self ) -> Option < String > {
277+ self . options . global . coerce_int96 . clone ( )
278+ }
279+
280+ pub fn with_coerce_int96 ( mut self , time_unit : Option < String > ) -> Self {
281+ self . options . global . coerce_int96 = time_unit;
282+ self
283+ }
271284}
272285
273286/// Clears all metadata (Schema level and field level) on an iterator
@@ -569,6 +582,46 @@ pub fn apply_file_schema_type_coercions(
569582 ) )
570583}
571584
585+ /// Coerces the file schema's Timestamps to the provided TimeUnit if Parquet schema contains INT96.
586+ pub fn coerce_int96_to_resolution (
587+ parquet_schema : & SchemaDescriptor ,
588+ file_schema : & Schema ,
589+ time_unit : & TimeUnit ,
590+ ) -> Option < Schema > {
591+ let mut transform = false ;
592+ let parquet_fields: HashMap < _ , _ > = parquet_schema
593+ . columns ( )
594+ . iter ( )
595+ . map ( |f| {
596+ let dt = f. physical_type ( ) ;
597+ if dt. eq ( & Type :: INT96 ) {
598+ transform = true ;
599+ }
600+ ( f. name ( ) , dt)
601+ } )
602+ . collect ( ) ;
603+
604+ if !transform {
605+ return None ;
606+ }
607+
608+ let transformed_fields: Vec < Arc < Field > > = file_schema
609+ . fields
610+ . iter ( )
611+ . map ( |field| match parquet_fields. get ( field. name ( ) . as_str ( ) ) {
612+ Some ( Type :: INT96 ) => {
613+ field_with_new_type ( field, DataType :: Timestamp ( * time_unit, None ) )
614+ }
615+ _ => Arc :: clone ( field) ,
616+ } )
617+ . collect ( ) ;
618+
619+ Some ( Schema :: new_with_metadata (
620+ transformed_fields,
621+ file_schema. metadata . clone ( ) ,
622+ ) )
623+ }
624+
572625/// Coerces the file schema if the table schema uses a view type.
573626#[ deprecated(
574627 since = "47.0.0" ,
0 commit comments