@@ -21,7 +21,6 @@ use crate::arrow::{arrow_schema_to_schema, get_arrow_datum};
21
21
use crate :: expr:: visitors:: bound_predicate_visitor:: { visit, BoundPredicateVisitor } ;
22
22
use crate :: expr:: { BoundPredicate , BoundReference } ;
23
23
use crate :: io:: { FileIO , FileMetadata , FileRead } ;
24
- use crate :: scan:: FileScanTask ;
25
24
use crate :: scan:: { ArrowRecordBatchStream , FileScanTaskStream } ;
26
25
use crate :: spec:: { Datum , SchemaRef } ;
27
26
use crate :: Result ;
@@ -30,13 +29,12 @@ use arrow_arith::boolean::{and, is_not_null, is_null, not, or};
30
29
use arrow_array:: { ArrayRef , BooleanArray , RecordBatch } ;
31
30
use arrow_ord:: cmp:: { eq, gt, gt_eq, lt, lt_eq, neq} ;
32
31
use arrow_schema:: { ArrowError , DataType , SchemaRef as ArrowSchemaRef } ;
32
+ use async_stream:: try_stream;
33
33
use bytes:: Bytes ;
34
34
use fnv:: FnvHashSet ;
35
- use futures:: channel:: mpsc:: { channel, Sender } ;
36
35
use futures:: future:: BoxFuture ;
37
36
use futures:: stream:: StreamExt ;
38
37
use futures:: { try_join, TryFutureExt } ;
39
- use futures:: { SinkExt , TryStreamExt } ;
40
38
use parquet:: arrow:: arrow_reader:: { ArrowPredicateFn , RowFilter } ;
41
39
use parquet:: arrow:: async_reader:: { AsyncFileReader , MetadataLoader } ;
42
40
use parquet:: arrow:: { ParquetRecordBatchStreamBuilder , ProjectionMask , PARQUET_FIELD_ID_META_KEY } ;
@@ -46,15 +44,11 @@ use std::collections::{HashMap, HashSet};
46
44
use std:: ops:: Range ;
47
45
use std:: str:: FromStr ;
48
46
use std:: sync:: Arc ;
49
- use tokio:: spawn;
50
-
51
- const CHANNEL_BUFFER_SIZE : usize = 10 ;
52
- const CONCURRENCY_LIMIT_TASKS : usize = 10 ;
53
47
54
48
/// Builder to create ArrowReader
55
49
pub struct ArrowReaderBuilder {
56
50
batch_size : Option < usize > ,
57
- field_ids : Arc < Vec < usize > > ,
51
+ field_ids : Vec < usize > ,
58
52
file_io : FileIO ,
59
53
schema : SchemaRef ,
60
54
predicate : Option < BoundPredicate > ,
@@ -65,7 +59,7 @@ impl ArrowReaderBuilder {
65
59
pub fn new ( file_io : FileIO , schema : SchemaRef ) -> Self {
66
60
ArrowReaderBuilder {
67
61
batch_size : None ,
68
- field_ids : Arc :: new ( vec ! [ ] ) ,
62
+ field_ids : vec ! [ ] ,
69
63
file_io,
70
64
schema,
71
65
predicate : None ,
@@ -81,10 +75,7 @@ impl ArrowReaderBuilder {
81
75
82
76
/// Sets the desired column projection with a list of field ids.
83
77
pub fn with_field_ids ( mut self , field_ids : impl IntoIterator < Item = usize > ) -> Self {
84
- let field_ids = field_ids. into_iter ( ) . collect ( ) ;
85
- let field_ids_arc = Arc :: new ( field_ids) ;
86
- self . field_ids = field_ids_arc;
87
-
78
+ self . field_ids = field_ids. into_iter ( ) . collect ( ) ;
88
79
self
89
80
}
90
81
@@ -109,7 +100,7 @@ impl ArrowReaderBuilder {
109
100
/// Reads data from Parquet files
110
101
pub struct ArrowReader {
111
102
batch_size : Option < usize > ,
112
- field_ids : Arc < Vec < usize > > ,
103
+ field_ids : Vec < usize > ,
113
104
#[ allow( dead_code) ]
114
105
schema : SchemaRef ,
115
106
file_io : FileIO ,
@@ -119,9 +110,7 @@ pub struct ArrowReader {
119
110
impl ArrowReader {
120
111
/// Take a stream of FileScanTasks and reads all the files.
121
112
/// Returns a stream of Arrow RecordBatches containing the data from the files
122
- pub fn read ( self , tasks : FileScanTaskStream ) -> Result < ArrowRecordBatchStream > {
123
- let ( sender, receiver) = channel ( CHANNEL_BUFFER_SIZE ) ;
124
-
113
+ pub fn read ( self , mut tasks : FileScanTaskStream ) -> Result < ArrowRecordBatchStream > {
125
114
// Collect Parquet column indices from field ids
126
115
let mut collector = CollectFieldIdVisitor {
127
116
field_ids : HashSet :: default ( ) ,
@@ -130,116 +119,44 @@ impl ArrowReader {
130
119
visit ( & mut collector, predicates) ?;
131
120
}
132
121
133
- let tasks = tasks. map ( move |task| self . build_file_scan_task_context ( task, sender. clone ( ) ) ) ;
134
-
135
- spawn ( async move {
136
- tasks
137
- . try_for_each_concurrent ( CONCURRENCY_LIMIT_TASKS , Self :: process_file_scan_task)
138
- . await
139
- } ) ;
140
-
141
- Ok ( receiver. boxed ( ) )
142
- }
143
-
144
- fn build_file_scan_task_context (
145
- & self ,
146
- task : Result < FileScanTask > ,
147
- sender : Sender < Result < RecordBatch > > ,
148
- ) -> Result < FileScanTaskContext > {
149
- Ok ( FileScanTaskContext :: new (
150
- task?,
151
- self . file_io . clone ( ) ,
152
- sender,
153
- self . batch_size ,
154
- self . field_ids . clone ( ) ,
155
- self . schema . clone ( ) ,
156
- self . predicate . clone ( ) ,
157
- ) )
158
- }
159
-
160
- async fn process_file_scan_task ( mut context : FileScanTaskContext ) -> Result < ( ) > {
161
- let file_scan_task = context. take_task ( ) ;
162
-
163
- // Collect Parquet column indices from field ids
164
- let mut collector = CollectFieldIdVisitor {
165
- field_ids : HashSet :: default ( ) ,
166
- } ;
167
- if let Some ( predicate) = & context. predicate {
168
- visit ( & mut collector, predicate) ?;
169
- }
170
-
171
- let parquet_file = context
172
- . file_io
173
- . new_input ( file_scan_task. data ( ) . data_file ( ) . file_path ( ) ) ?;
174
- let ( parquet_metadata, parquet_reader) =
175
- try_join ! ( parquet_file. metadata( ) , parquet_file. reader( ) ) ?;
176
- let arrow_file_reader = ArrowFileReader :: new ( parquet_metadata, parquet_reader) ;
177
-
178
- let mut batch_stream_builder =
179
- ParquetRecordBatchStreamBuilder :: new ( arrow_file_reader) . await ?;
122
+ Ok ( try_stream ! {
123
+ while let Some ( Ok ( task) ) = tasks. next( ) . await {
180
124
181
- let parquet_schema = batch_stream_builder. parquet_schema ( ) ;
182
- let arrow_schema = batch_stream_builder. schema ( ) ;
125
+ let parquet_file = self
126
+ . file_io
127
+ . new_input( task. data( ) . data_file( ) . file_path( ) ) ?;
128
+ let ( parquet_metadata, parquet_reader) =
129
+ try_join!( parquet_file. metadata( ) , parquet_file. reader( ) ) ?;
130
+ let arrow_file_reader = ArrowFileReader :: new( parquet_metadata, parquet_reader) ;
183
131
184
- let projection_mask = context . get_arrow_projection_mask ( parquet_schema , arrow_schema ) ? ;
185
- batch_stream_builder = batch_stream_builder . with_projection ( projection_mask ) ;
132
+ let mut batch_stream_builder =
133
+ ParquetRecordBatchStreamBuilder :: new ( arrow_file_reader ) . await ? ;
186
134
187
- let parquet_schema = batch_stream_builder. parquet_schema ( ) ;
188
- let row_filter = context. get_row_filter ( parquet_schema, & collector) ?;
189
-
190
- if let Some ( row_filter) = row_filter {
191
- batch_stream_builder = batch_stream_builder. with_row_filter ( row_filter) ;
192
- }
135
+ let parquet_schema = batch_stream_builder. parquet_schema( ) ;
136
+ let arrow_schema = batch_stream_builder. schema( ) ;
193
137
194
- if let Some ( batch_size) = context. batch_size {
195
- batch_stream_builder = batch_stream_builder. with_batch_size ( batch_size) ;
196
- }
138
+ let projection_mask = self . get_arrow_projection_mask( parquet_schema, arrow_schema) ?;
139
+ batch_stream_builder = batch_stream_builder. with_projection( projection_mask) ;
197
140
198
- let mut batch_stream = batch_stream_builder. build ( ) ?;
141
+ let parquet_schema = batch_stream_builder. parquet_schema( ) ;
142
+ let row_filter = self . get_row_filter( parquet_schema, & collector) ?;
199
143
200
- while let Some ( batch ) = batch_stream . next ( ) . await {
201
- context . sender . send ( Ok ( batch? ) ) . await ? ;
202
- }
144
+ if let Some ( row_filter ) = row_filter {
145
+ batch_stream_builder = batch_stream_builder . with_row_filter ( row_filter ) ;
146
+ }
203
147
204
- Ok ( ( ) )
205
- }
206
- }
148
+ if let Some ( batch_size ) = self . batch_size {
149
+ batch_stream_builder = batch_stream_builder . with_batch_size ( batch_size ) ;
150
+ }
207
151
208
- struct FileScanTaskContext {
209
- file_scan_task : Option < FileScanTask > ,
210
- file_io : FileIO ,
211
- sender : Sender < Result < RecordBatch > > ,
212
- batch_size : Option < usize > ,
213
- field_ids : Arc < Vec < usize > > ,
214
- schema : SchemaRef ,
215
- predicate : Option < BoundPredicate > ,
216
- }
152
+ let mut batch_stream = batch_stream_builder. build( ) ?;
217
153
218
- impl FileScanTaskContext {
219
- fn new (
220
- file_scan_task : FileScanTask ,
221
- file_io : FileIO ,
222
- sender : Sender < Result < RecordBatch > > ,
223
- batch_size : Option < usize > ,
224
- field_ids : Arc < Vec < usize > > ,
225
- schema : SchemaRef ,
226
- predicate : Option < BoundPredicate > ,
227
- ) -> Self {
228
- FileScanTaskContext {
229
- file_scan_task : Some ( file_scan_task) ,
230
- file_io,
231
- sender,
232
- batch_size,
233
- field_ids,
234
- schema,
235
- predicate,
154
+ while let Some ( batch) = batch_stream. next( ) . await {
155
+ yield batch?;
156
+ }
157
+ }
236
158
}
237
- }
238
-
239
- fn take_task ( & mut self ) -> FileScanTask {
240
- let mut result = None ;
241
- std:: mem:: swap ( & mut self . file_scan_task , & mut result) ;
242
- result. unwrap ( )
159
+ . boxed ( ) )
243
160
}
244
161
245
162
fn get_arrow_projection_mask (
@@ -297,8 +214,8 @@ impl FileScanTaskContext {
297
214
}
298
215
299
216
let mut indices = vec ! [ ] ;
300
- for field_id in self . field_ids . as_ref ( ) {
301
- if let Some ( col_idx) = column_map. get ( & ( * field_id as i32 ) ) {
217
+ for & field_id in & self . field_ids {
218
+ if let Some ( col_idx) = column_map. get ( & ( field_id as i32 ) ) {
302
219
indices. push ( * col_idx) ;
303
220
} else {
304
221
return Err ( Error :: new (
0 commit comments