20
20
21
21
use std:: { any:: Any , sync:: Arc } ;
22
22
23
- use arrow:: datatypes:: SchemaRef ;
23
+ use arrow:: datatypes:: { DataType , Field , Schema , SchemaRef } ;
24
+ use async_trait:: async_trait;
24
25
use futures:: { StreamExt , TryStreamExt } ;
25
26
26
27
use crate :: {
@@ -32,6 +33,7 @@ use crate::{
32
33
33
34
use super :: {
34
35
datasource:: TableProviderFilterPushDown , format:: FileFormat , PartitionedFile ,
36
+ TableProvider ,
35
37
} ;
36
38
37
39
/// Options for creating a `ListingTable`
@@ -75,7 +77,13 @@ impl ListingOptions {
75
77
& self . file_extension, path
76
78
) )
77
79
} ) ?;
78
- self . format . infer_schema ( & first_file) . await
80
+ let file_schema = self . format . infer_schema ( & first_file) . await ?;
81
+ // Add the partition columns to the file schema
82
+ let mut fields = file_schema. fields ( ) . clone ( ) ;
83
+ for part in & self . partitions {
84
+ fields. push ( Field :: new ( part, DataType :: Utf8 , false ) ) ;
85
+ }
86
+ Ok ( Arc :: new ( Schema :: new ( fields) ) )
79
87
}
80
88
}
81
89
@@ -104,9 +112,8 @@ impl ListingTable {
104
112
}
105
113
}
106
114
107
- // TODO add back impl ExecutionPlan
108
- #[ allow( dead_code) ]
109
- impl ListingTable {
115
+ #[ async_trait]
116
+ impl TableProvider for ListingTable {
110
117
fn as_any ( & self ) -> & dyn Any {
111
118
self
112
119
}
@@ -122,7 +129,7 @@ impl ListingTable {
122
129
filters : & [ Expr ] ,
123
130
limit : Option < usize > ,
124
131
) -> Result < Arc < dyn ExecutionPlan > > {
125
- // 1. list files (with partitions)
132
+ // list files (with partitions)
126
133
let file_list = pruned_partition_list (
127
134
& self . path ,
128
135
filters,
@@ -214,47 +221,47 @@ fn split_files(
214
221
215
222
#[ cfg( test) ]
216
223
mod tests {
217
- // use super::*;
218
- // use futures::StreamExt;
224
+ use super :: * ;
225
+ use futures:: StreamExt ;
219
226
220
- // #[tokio::test]
221
- // async fn read_small_batches() -> Result<()> {
222
- // let table = load_table("alltypes_plain.parquet").await?;
223
- // let projection = None;
224
- // let exec = table.scan(&projection, 2, &[], None)?;
225
- // let stream = exec.execute(0).await?;
227
+ #[ tokio:: test]
228
+ async fn read_small_batches ( ) -> Result < ( ) > {
229
+ let table = load_table ( "alltypes_plain.parquet" ) . await ?;
230
+ let projection = None ;
231
+ let exec = table. scan ( & projection, 2 , & [ ] , None ) . await ?;
232
+ let stream = exec. execute ( 0 ) . await ?;
226
233
227
- // let _ = stream
228
- // .map(|batch| {
229
- // let batch = batch.unwrap();
230
- // assert_eq!(11, batch.num_columns());
231
- // assert_eq!(2, batch.num_rows());
232
- // })
233
- // .fold(0, |acc, _| async move { acc + 1i32 })
234
- // .await;
234
+ let _ = stream
235
+ . map ( |batch| {
236
+ let batch = batch. unwrap ( ) ;
237
+ assert_eq ! ( 11 , batch. num_columns( ) ) ;
238
+ assert_eq ! ( 2 , batch. num_rows( ) ) ;
239
+ } )
240
+ . fold ( 0 , |acc, _| async move { acc + 1i32 } )
241
+ . await ;
235
242
236
- // // test metadata
237
- // assert_eq!(exec.statistics().num_rows, Some(8));
238
- // assert_eq!(exec.statistics().total_byte_size, Some(671));
243
+ // test metadata
244
+ assert_eq ! ( exec. statistics( ) . num_rows, Some ( 8 ) ) ;
245
+ assert_eq ! ( exec. statistics( ) . total_byte_size, Some ( 671 ) ) ;
239
246
240
- // Ok(())
241
- // }
247
+ Ok ( ( ) )
248
+ }
242
249
243
- // async fn load_table(name: &str) -> Result<Arc<dyn TableProvider>> {
244
- // let testdata = crate::test_util::parquet_test_data();
245
- // let filename = format!("{}/{}", testdata, name);
246
- // let opt = ListingOptions {
247
- // file_extension: "parquet".to_owned(),
248
- // format: Arc::new(format::parquet::ParquetFormat {
249
- // enable_pruning: true,
250
- // }),
251
- // partitions: vec![],
252
- // max_partitions: 2,
253
- // collect_stat: true,
254
- // };
255
- // // here we resolve the schema locally
256
- // let schema = opt.infer_schema(&filename).await?;
257
- // let table = ListingTable::try_new(&filename, schema, opt)?;
258
- // Ok(Arc::new(table))
259
- // }
250
+ async fn load_table ( name : & str ) -> Result < Arc < dyn TableProvider > > {
251
+ let testdata = crate :: test_util:: parquet_test_data ( ) ;
252
+ let filename = format ! ( "{}/{}" , testdata, name) ;
253
+ let opt = ListingOptions {
254
+ file_extension : "parquet" . to_owned ( ) ,
255
+ format : Arc :: new ( format:: parquet:: ParquetFormat {
256
+ enable_pruning : true ,
257
+ } ) ,
258
+ partitions : vec ! [ ] ,
259
+ max_partitions : 2 ,
260
+ collect_stat : true ,
261
+ } ;
262
+ // here we resolve the schema locally
263
+ let schema = opt. infer_schema ( & filename) . await ?;
264
+ let table = ListingTable :: try_new ( & filename, schema, opt) ?;
265
+ Ok ( Arc :: new ( table) )
266
+ }
260
267
}
0 commit comments