@@ -453,7 +453,7 @@ impl SyncDbConnection<r2d2::PooledConnection<DuckdbConnectionManager>, DuckDBPar
453453 & self ,
454454 sql : & str ,
455455 params : & [ DuckDBParameter ] ,
456- _projected_schema : Option < SchemaRef > ,
456+ projected_schema : Option < SchemaRef > ,
457457 ) -> Result < SendableRecordBatchStream > {
458458 let ( batch_tx, mut batch_rx) = tokio:: sync:: mpsc:: channel :: < RecordBatch > ( 4 ) ;
459459
@@ -501,9 +501,20 @@ impl SyncDbConnection<r2d2::PooledConnection<DuckdbConnectionManager>, DuckDBPar
501501 Ok :: < _ , Box < dyn std:: error:: Error + Send + Sync > > ( ( ) )
502502 } ) ;
503503
504+ let stream_schema = projected_schema. clone ( ) . unwrap_or ( schema) ;
505+
504506 let output_stream = stream ! {
505507 while let Some ( batch) = batch_rx. recv( ) . await {
506- yield Ok ( batch) ;
508+ if let Some ( ref target_schema) = projected_schema {
509+ match datafusion_federation:: schema_cast:: record_convert:: try_cast_to( batch, Arc :: clone( target_schema) ) {
510+ Ok ( casted) => yield Ok ( casted) ,
511+ Err ( e) => yield Err ( DataFusionError :: Execution ( format!(
512+ "Failed to cast DuckDB result batch to projected schema: {e}"
513+ ) ) ) ,
514+ }
515+ } else {
516+ yield Ok ( batch) ;
517+ }
507518 }
508519
509520 match join_handle. await {
@@ -522,7 +533,7 @@ impl SyncDbConnection<r2d2::PooledConnection<DuckdbConnectionManager>, DuckDBPar
522533 } ;
523534
524535 Ok ( Box :: pin ( RecordBatchStreamAdapter :: new (
525- schema ,
536+ stream_schema ,
526537 output_stream,
527538 ) ) )
528539 } ;
@@ -945,4 +956,70 @@ mod tests {
945956
946957 Ok ( ( ) )
947958 }
959+
960+ #[ test]
961+ fn test_query_arrow_casts_to_projected_schema ( ) {
962+ use arrow:: datatypes:: { Schema , TimeUnit } ;
963+ use futures:: StreamExt ;
964+
965+ use crate :: sql:: db_connection_pool:: duckdbpool:: DuckDbConnectionPool ;
966+ use crate :: sql:: db_connection_pool:: DbConnectionPool ;
967+
968+ let rt = tokio:: runtime:: Runtime :: new ( ) . expect ( "runtime" ) ;
969+ rt. block_on ( async {
970+ let pool = DuckDbConnectionPool :: new_memory ( ) . expect ( "pool created" ) ;
971+
972+ // Create table with TIMESTAMPTZ (DuckDB stores as Microsecond)
973+ let conn = pool. connect ( ) . await . expect ( "connection" ) ;
974+ let conn = conn. as_sync ( ) . expect ( "sync connection" ) ;
975+ conn. execute (
976+ "CREATE TABLE test_ts (id INTEGER, created_at TIMESTAMPTZ)" ,
977+ & [ ] ,
978+ )
979+ . expect ( "table created" ) ;
980+ conn. execute (
981+ "INSERT INTO test_ts VALUES (1, '2023-01-01T00:00:00Z')" ,
982+ & [ ] ,
983+ )
984+ . expect ( "data inserted" ) ;
985+
986+ // Request Nanosecond via projected_schema
987+ let projected_schema = Arc :: new ( Schema :: new ( vec ! [
988+ Field :: new( "id" , DataType :: Int32 , true ) ,
989+ Field :: new(
990+ "created_at" ,
991+ DataType :: Timestamp ( TimeUnit :: Nanosecond , Some ( "UTC" . into( ) ) ) ,
992+ true ,
993+ ) ,
994+ ] ) ) ;
995+
996+ let stream = conn
997+ . query_arrow (
998+ "SELECT id, created_at FROM test_ts" ,
999+ & [ ] ,
1000+ Some ( projected_schema. clone ( ) ) ,
1001+ )
1002+ . expect ( "query_arrow should succeed" ) ;
1003+
1004+ // Verify stream schema matches projected_schema
1005+ assert_eq ! ( stream. schema( ) , projected_schema) ;
1006+
1007+ let mut batches = vec ! [ ] ;
1008+ let mut stream = stream;
1009+ while let Some ( batch) = stream. next ( ) . await {
1010+ batches. push ( batch. expect ( "batch should be Ok" ) ) ;
1011+ }
1012+
1013+ assert_eq ! ( batches. len( ) , 1 ) ;
1014+ let batch = & batches[ 0 ] ;
1015+ assert_eq ! ( batch. schema( ) , projected_schema) ;
1016+
1017+ // Verify the timestamp column is Nanosecond
1018+ let ts_col = batch. column ( 1 ) ;
1019+ assert_eq ! (
1020+ ts_col. data_type( ) ,
1021+ & DataType :: Timestamp ( TimeUnit :: Nanosecond , Some ( "UTC" . into( ) ) )
1022+ ) ;
1023+ } ) ;
1024+ }
9481025}
0 commit comments