1919
2020use std:: sync:: Arc ;
2121
22+ use arrow:: array:: new_empty_array;
2223use arrow:: {
2324 array:: {
2425 Array , ArrayBuilder , ArrayRef , Date64Array , Date64Builder , StringBuilder ,
@@ -176,7 +177,7 @@ pub async fn pruned_partition_list<'a>(
176177 table_path : & ' a ListingTableUrl ,
177178 filters : & ' a [ Expr ] ,
178179 file_extension : & ' a str ,
179- table_partition_cols : & ' a [ String ] ,
180+ table_partition_cols : & ' a [ ( String , DataType ) ] ,
180181) -> Result < BoxStream < ' a , Result < PartitionedFile > > > {
181182 let list = table_path. list_all_files ( store, file_extension) ;
182183
@@ -187,7 +188,15 @@ pub async fn pruned_partition_list<'a>(
187188
188189 let applicable_filters: Vec < _ > = filters
189190 . iter ( )
190- . filter ( |f| expr_applicable_for_cols ( table_partition_cols, f) )
191+ . filter ( |f| {
192+ expr_applicable_for_cols (
193+ & table_partition_cols
194+ . iter ( )
195+ . map ( |x| x. 0 . clone ( ) )
196+ . collect :: < Vec < _ > > ( ) ,
197+ f,
198+ )
199+ } )
191200 . collect ( ) ;
192201
193202 if applicable_filters. is_empty ( ) {
@@ -200,11 +209,26 @@ pub async fn pruned_partition_list<'a>(
200209 let parsed_path = parse_partitions_for_path (
201210 table_path,
202211 & object_meta. location ,
203- table_partition_cols,
212+ & table_partition_cols
213+ . iter ( )
214+ . map ( |x| x. 0 . clone ( ) )
215+ . collect :: < Vec < _ > > ( ) ,
204216 )
205217 . map ( |p| {
206218 p. iter ( )
207- . map ( |& pn| ScalarValue :: Utf8 ( Some ( pn. to_owned ( ) ) ) )
219+ . zip ( table_partition_cols)
220+ . map ( |( & part_value, part_column) | {
221+ ScalarValue :: try_from_string (
222+ part_value. to_string ( ) ,
223+ & part_column. 1 ,
224+ )
225+ . unwrap_or_else ( |_| {
226+ panic ! (
227+ "Failed to cast str {} to type {}" ,
228+ part_value, part_column. 1
229+ )
230+ } )
231+ } )
208232 . collect ( )
209233 } ) ;
210234
@@ -221,6 +245,7 @@ pub async fn pruned_partition_list<'a>(
221245 let metas: Vec < _ > = list. try_collect ( ) . await ?;
222246 let batch = paths_to_batch ( table_partition_cols, table_path, & metas) ?;
223247 let mem_table = MemTable :: try_new ( batch. schema ( ) , vec ! [ vec![ batch] ] ) ?;
248+ debug ! ( "get mem_table: {:?}" , mem_table) ;
224249
225250 // Filter the partitions using a local datafusion context
226251 // TODO having the external context would allow us to resolve `Volatility::Stable`
@@ -245,28 +270,35 @@ pub async fn pruned_partition_list<'a>(
245270///
246271/// Note: For the last modified date, this looses precisions higher than millisecond.
247272fn paths_to_batch (
248- table_partition_cols : & [ String ] ,
273+ table_partition_cols : & [ ( String , DataType ) ] ,
249274 table_path : & ListingTableUrl ,
250275 metas : & [ ObjectMeta ] ,
251276) -> Result < RecordBatch > {
252277 let mut key_builder = StringBuilder :: with_capacity ( metas. len ( ) , 1024 ) ;
253278 let mut length_builder = UInt64Builder :: with_capacity ( metas. len ( ) ) ;
254279 let mut modified_builder = Date64Builder :: with_capacity ( metas. len ( ) ) ;
255- let mut partition_builders = table_partition_cols
280+ let mut partition_scalar_values = table_partition_cols
256281 . iter ( )
257- . map ( |_| StringBuilder :: with_capacity ( metas . len ( ) , 1024 ) )
282+ . map ( |_| Vec :: new ( ) )
258283 . collect :: < Vec < _ > > ( ) ;
259284 for file_meta in metas {
260285 if let Some ( partition_values) = parse_partitions_for_path (
261286 table_path,
262287 & file_meta. location ,
263- table_partition_cols,
288+ & table_partition_cols
289+ . iter ( )
290+ . map ( |x| x. 0 . clone ( ) )
291+ . collect :: < Vec < _ > > ( ) ,
264292 ) {
265293 key_builder. append_value ( file_meta. location . as_ref ( ) ) ;
266294 length_builder. append_value ( file_meta. size as u64 ) ;
267295 modified_builder. append_value ( file_meta. last_modified . timestamp_millis ( ) ) ;
268296 for ( i, part_val) in partition_values. iter ( ) . enumerate ( ) {
269- partition_builders[ i] . append_value ( part_val) ;
297+ let scalar_val = ScalarValue :: try_from_string (
298+ part_val. to_string ( ) ,
299+ & table_partition_cols[ i] . 1 ,
300+ ) ?;
301+ partition_scalar_values[ i] . push ( scalar_val) ;
270302 }
271303 } else {
272304 debug ! ( "No partitioning for path {}" , file_meta. location) ;
@@ -279,8 +311,13 @@ fn paths_to_batch(
279311 ArrayBuilder :: finish( & mut length_builder) ,
280312 ArrayBuilder :: finish( & mut modified_builder) ,
281313 ] ;
282- for mut partition_builder in partition_builders {
283- col_arrays. push ( ArrayBuilder :: finish ( & mut partition_builder) ) ;
314+ for ( i, part_scalar_val) in partition_scalar_values. into_iter ( ) . enumerate ( ) {
315+ if part_scalar_val. is_empty ( ) {
316+ col_arrays. push ( new_empty_array ( & table_partition_cols[ i] . 1 ) ) ;
317+ } else {
318+ let partition_val_array = ScalarValue :: iter_to_array ( part_scalar_val) ?;
319+ col_arrays. push ( partition_val_array) ;
320+ }
284321 }
285322
286323 // put the schema together
@@ -289,8 +326,8 @@ fn paths_to_batch(
289326 Field :: new( FILE_SIZE_COLUMN_NAME , DataType :: UInt64 , false ) ,
290327 Field :: new( FILE_MODIFIED_COLUMN_NAME , DataType :: Date64 , true ) ,
291328 ] ;
292- for pn in table_partition_cols {
293- fields. push ( Field :: new ( pn , DataType :: Utf8 , false ) ) ;
329+ for part_col in table_partition_cols {
330+ fields. push ( Field :: new ( & part_col . 0 , part_col . 1 . to_owned ( ) , false ) ) ;
294331 }
295332
296333 let batch = RecordBatch :: try_new ( Arc :: new ( Schema :: new ( fields) ) , col_arrays) ?;
@@ -366,9 +403,10 @@ fn parse_partitions_for_path<'a>(
366403
367404#[ cfg( test) ]
368405mod tests {
406+ use futures:: StreamExt ;
407+
369408 use crate :: logical_expr:: { case, col, lit} ;
370409 use crate :: test:: object_store:: make_test_store;
371- use futures:: StreamExt ;
372410
373411 use super :: * ;
374412
@@ -424,7 +462,7 @@ mod tests {
424462 & ListingTableUrl :: parse ( "file:///tablepath/" ) . unwrap ( ) ,
425463 & [ filter] ,
426464 ".parquet" ,
427- & [ String :: from ( "mypartition" ) ] ,
465+ & [ ( String :: from ( "mypartition" ) , DataType :: Utf8 ) ] ,
428466 )
429467 . await
430468 . expect ( "partition pruning failed" )
@@ -447,7 +485,7 @@ mod tests {
447485 & ListingTableUrl :: parse ( "file:///tablepath/" ) . unwrap ( ) ,
448486 & [ filter] ,
449487 ".parquet" ,
450- & [ String :: from ( "mypartition" ) ] ,
488+ & [ ( String :: from ( "mypartition" ) , DataType :: Utf8 ) ] ,
451489 )
452490 . await
453491 . expect ( "partition pruning failed" )
@@ -494,7 +532,10 @@ mod tests {
494532 & ListingTableUrl :: parse ( "file:///tablepath/" ) . unwrap ( ) ,
495533 & [ filter1, filter2, filter3] ,
496534 ".parquet" ,
497- & [ String :: from ( "part1" ) , String :: from ( "part2" ) ] ,
535+ & [
536+ ( String :: from ( "part1" ) , DataType :: Utf8 ) ,
537+ ( String :: from ( "part2" ) , DataType :: Utf8 ) ,
538+ ] ,
498539 )
499540 . await
500541 . expect ( "partition pruning failed" )
@@ -645,7 +686,7 @@ mod tests {
645686 ] ;
646687
647688 let batches = paths_to_batch (
648- & [ String :: from ( "part1" ) ] ,
689+ & [ ( String :: from ( "part1" ) , DataType :: Utf8 ) ] ,
649690 & ListingTableUrl :: parse ( "file:///mybucket/tablepath" ) . unwrap ( ) ,
650691 & files,
651692 )
0 commit comments