12
12
// See the License for the specific language governing permissions and
13
13
// limitations under the License.
14
14
15
+ use std:: collections:: HashMap ;
15
16
use std:: fmt:: Debug ;
16
17
use std:: fmt:: Formatter ;
18
+ use std:: io:: Cursor ;
19
+ use std:: io:: Read ;
20
+ use std:: io:: Seek ;
21
+ use std:: mem;
17
22
use std:: sync:: Arc ;
18
23
24
+ use common_arrow:: arrow:: array:: Array ;
25
+ use common_arrow:: arrow:: chunk:: Chunk ;
26
+ use common_arrow:: arrow:: datatypes:: Field ;
27
+ use common_arrow:: arrow:: io:: parquet:: read;
28
+ use common_arrow:: arrow:: io:: parquet:: read:: read_columns;
29
+ use common_arrow:: arrow:: io:: parquet:: read:: to_deserializer;
30
+ use common_arrow:: arrow:: io:: parquet:: read:: RowGroupDeserializer ;
31
+ use common_arrow:: parquet:: metadata:: ColumnChunkMetaData ;
32
+ use common_arrow:: parquet:: metadata:: FileMetaData ;
19
33
use common_arrow:: parquet:: metadata:: RowGroupMetaData ;
34
+ use common_arrow:: parquet:: read:: read_metadata;
20
35
use common_base:: base:: tokio:: sync:: mpsc:: Receiver ;
21
36
use common_datablocks:: DataBlock ;
37
+ use common_datavalues:: remove_nullable;
38
+ use common_datavalues:: DataField ;
39
+ use common_datavalues:: DataSchemaRef ;
40
+ use common_exception:: ErrorCode ;
22
41
use common_exception:: Result ;
23
42
use common_pipeline_core:: Pipeline ;
24
43
use opendal:: Object ;
44
+ use similar_asserts:: traits:: MakeDiff ;
25
45
46
+ use crate :: processors:: sources:: input_formats:: delimiter:: RecordDelimiter ;
26
47
use crate :: processors:: sources:: input_formats:: input_context:: InputContext ;
27
48
use crate :: processors:: sources:: input_formats:: input_format:: FileInfo ;
28
49
use crate :: processors:: sources:: input_formats:: input_format:: InputData ;
@@ -33,32 +54,45 @@ use crate::processors::sources::input_formats::input_pipeline::InputFormatPipe;
33
54
use crate :: processors:: sources:: input_formats:: input_pipeline:: StreamingReadBatch ;
34
55
use crate :: processors:: sources:: input_formats:: InputFormat ;
35
56
36
- struct InputFormatParquet ;
57
+ pub struct InputFormatParquet ;
37
58
38
59
#[ async_trait:: async_trait]
39
60
impl InputFormat for InputFormatParquet {
61
+ fn default_record_delimiter ( & self ) -> RecordDelimiter {
62
+ RecordDelimiter :: Crlf
63
+ }
64
+
65
+ fn default_field_delimiter ( & self ) -> u8 {
66
+ b'_'
67
+ }
68
+
40
69
async fn read_file_meta (
41
70
& self ,
42
- obj : & Object ,
43
- size : usize ,
71
+ _obj : & Object ,
72
+ _size : usize ,
44
73
) -> Result < Option < Arc < dyn InputData > > > {
45
- todo ! ( )
74
+ // todo(youngsofun): execute_copy_aligned
75
+ Ok ( None )
46
76
}
47
77
48
78
async fn read_split_meta (
49
79
& self ,
50
- obj : & Object ,
51
- split_info : & SplitInfo ,
80
+ _obj : & Object ,
81
+ _split_info : & SplitInfo ,
52
82
) -> Result < Option < Box < dyn InputData > > > {
53
- todo ! ( )
83
+ Ok ( None )
54
84
}
55
85
56
- fn split_files ( & self , file_infos : Vec < FileInfo > , split_size : usize ) -> Vec < SplitInfo > {
57
- todo ! ( )
86
+ fn split_files ( & self , file_infos : Vec < FileInfo > , _split_size : usize ) -> Vec < SplitInfo > {
87
+ file_infos
88
+ . into_iter ( )
89
+ . map ( SplitInfo :: from_file_info)
90
+ . collect ( )
58
91
}
59
92
60
93
fn exec_copy ( & self , ctx : Arc < InputContext > , pipeline : & mut Pipeline ) -> Result < ( ) > {
61
- todo ! ( )
94
+ // todo(youngsofun): execute_copy_aligned
95
+ ParquetFormatPipe :: execute_copy_with_aligner ( ctx, pipeline)
62
96
}
63
97
64
98
fn exec_stream (
@@ -67,7 +101,7 @@ impl InputFormat for InputFormatParquet {
67
101
pipeline : & mut Pipeline ,
68
102
input : Receiver < StreamingReadBatch > ,
69
103
) -> Result < ( ) > {
70
- todo ! ( )
104
+ ParquetFormatPipe :: execute_stream ( ctx , pipeline , input )
71
105
}
72
106
}
73
107
@@ -81,11 +115,58 @@ impl InputFormatPipe for ParquetFormatPipe {
81
115
type BlockBuilder = ParquetBlockBuilder ;
82
116
}
83
117
84
- pub struct SplitMeta {
85
- row_groups : Vec < RowGroupMetaData > ,
118
+ pub struct RowGroupInMemory {
119
+ pub meta : RowGroupMetaData ,
120
+ pub fields : Arc < Vec < Field > > ,
121
+ pub field_meta_indexes : Vec < Vec < usize > > ,
122
+ pub field_arrays : Vec < Vec < Vec < u8 > > > ,
86
123
}
87
124
88
- pub struct RowGroupInMemory { }
125
+ impl RowGroupInMemory {
126
+ fn read < R : Read + Seek > (
127
+ reader : & mut R ,
128
+ meta : RowGroupMetaData ,
129
+ fields : Arc < Vec < Field > > ,
130
+ ) -> Result < Self > {
131
+ let field_names = fields. iter ( ) . map ( |x| x. name . as_str ( ) ) . collect :: < Vec < _ > > ( ) ;
132
+ let field_meta_indexes = split_column_metas_by_field ( meta. columns ( ) , & field_names) ;
133
+ let mut filed_arrays = vec ! [ ] ;
134
+ for field_name in field_names {
135
+ let meta_data = read_columns ( reader, meta. columns ( ) , field_name) ?;
136
+ let data = meta_data. into_iter ( ) . map ( |t| t. 1 ) . collect :: < Vec < _ > > ( ) ;
137
+ filed_arrays. push ( data)
138
+ }
139
+ Ok ( Self {
140
+ meta,
141
+ field_meta_indexes,
142
+ field_arrays : filed_arrays,
143
+ fields,
144
+ } )
145
+ }
146
+
147
+ fn get_arrow_chunk ( & mut self ) -> Result < Chunk < Box < dyn Array > > > {
148
+ let mut column_chunks = vec ! [ ] ;
149
+ let field_arrays = mem:: take ( & mut self . field_arrays ) ;
150
+ for ( f, datas) in field_arrays. into_iter ( ) . enumerate ( ) {
151
+ let meta_iters = self . field_meta_indexes [ f]
152
+ . iter ( )
153
+ . map ( |c| & self . meta . columns ( ) [ * c] ) ;
154
+ let meta_data = meta_iters. zip ( datas. into_iter ( ) ) . collect :: < Vec < _ > > ( ) ;
155
+ let array_iters = to_deserializer (
156
+ meta_data,
157
+ self . fields [ f] . clone ( ) ,
158
+ self . meta . num_rows ( ) as usize ,
159
+ None ,
160
+ ) ?;
161
+ column_chunks. push ( array_iters) ;
162
+ }
163
+ match RowGroupDeserializer :: new ( column_chunks, self . meta . num_rows ( ) , None ) . next ( ) {
164
+ None => Err ( ErrorCode :: ParquetError ( "fail to get a chunk" ) ) ,
165
+ Some ( Ok ( chunk) ) => Ok ( chunk) ,
166
+ Some ( Err ( e) ) => Err ( ErrorCode :: ParquetError ( e. to_string ( ) ) ) ,
167
+ }
168
+ }
169
+ }
89
170
90
171
impl Debug for RowGroupInMemory {
91
172
fn fmt ( & self , f : & mut Formatter < ' _ > ) -> std:: fmt:: Result {
@@ -96,6 +177,7 @@ impl Debug for RowGroupInMemory {
96
177
#[ derive( Debug ) ]
97
178
pub enum ReadBatch {
98
179
Buffer ( Vec < u8 > ) ,
180
+ #[ allow( unused) ]
99
181
RowGroup ( RowGroupInMemory ) ,
100
182
}
101
183
@@ -116,23 +198,117 @@ impl BlockBuilderTrait for ParquetBlockBuilder {
116
198
ParquetBlockBuilder { ctx }
117
199
}
118
200
119
- fn deserialize ( & mut self , batch : Option < RowGroupInMemory > ) -> Result < Vec < DataBlock > > {
120
- todo ! ( )
201
+ fn deserialize ( & mut self , mut batch : Option < RowGroupInMemory > ) -> Result < Vec < DataBlock > > {
202
+ if let Some ( rg) = batch. as_mut ( ) {
203
+ let chunk = rg. get_arrow_chunk ( ) ?;
204
+ let block = DataBlock :: from_chunk ( & self . ctx . schema , & chunk) ?;
205
+ Ok ( vec ! [ block] )
206
+ } else {
207
+ Ok ( vec ! [ ] )
208
+ }
121
209
}
122
210
}
123
211
124
212
pub struct AligningState {
213
+ ctx : Arc < InputContext > ,
214
+ split_info : SplitInfo ,
125
215
buffers : Vec < Vec < u8 > > ,
126
216
}
127
217
128
218
impl AligningStateTrait for AligningState {
129
219
type Pipe = ParquetFormatPipe ;
130
220
131
221
fn try_create ( ctx : & Arc < InputContext > , split_info : & SplitInfo ) -> Result < Self > {
132
- todo ! ( )
222
+ Ok ( AligningState {
223
+ ctx : ctx. clone ( ) ,
224
+ split_info : split_info. clone ( ) ,
225
+ buffers : vec ! [ ] ,
226
+ } )
133
227
}
134
228
135
229
fn align ( & mut self , read_batch : Option < ReadBatch > ) -> Result < Vec < RowGroupInMemory > > {
136
- todo ! ( )
230
+ if let Some ( rb) = read_batch {
231
+ if let ReadBatch :: Buffer ( b) = rb {
232
+ self . buffers . push ( b)
233
+ } ;
234
+ Ok ( vec ! [ ] )
235
+ } else {
236
+ let file_in_memory = self . buffers . concat ( ) ;
237
+ let size = file_in_memory. len ( ) ;
238
+ tracing:: debug!(
239
+ "aligning parquet file {} of {} bytes" ,
240
+ self . split_info. file_info. path,
241
+ size,
242
+ ) ;
243
+ let mut cursor = Cursor :: new ( file_in_memory) ;
244
+ let file_meta =
245
+ read_metadata ( & mut cursor) . map_err ( |e| ErrorCode :: ParquetError ( e. to_string ( ) ) ) ?;
246
+ let read_fields = Arc :: new ( get_fields ( & file_meta, & self . ctx . schema ) ?) ;
247
+
248
+ let mut row_batches = Vec :: with_capacity ( file_meta. row_groups . len ( ) ) ;
249
+ for row_group in file_meta. row_groups . into_iter ( ) {
250
+ row_batches. push ( RowGroupInMemory :: read (
251
+ & mut cursor,
252
+ row_group,
253
+ read_fields. clone ( ) ,
254
+ ) ?)
255
+ }
256
+ tracing:: info!(
257
+ "align parquet file {} of {} bytes to {} row groups" ,
258
+ self . split_info. file_info. path,
259
+ size,
260
+ row_batches. len( )
261
+ ) ;
262
+ Ok ( row_batches)
263
+ }
264
+ }
265
+ }
266
+
267
+ fn get_fields ( file_meta : & FileMetaData , schema : & DataSchemaRef ) -> Result < Vec < Field > > {
268
+ let infer_schema = read:: infer_schema ( file_meta) ?;
269
+ let mut read_fields = Vec :: with_capacity ( schema. num_fields ( ) ) ;
270
+ for f in schema. fields ( ) . iter ( ) {
271
+ if let Some ( m) = infer_schema
272
+ . fields
273
+ . iter ( )
274
+ . filter ( |c| c. name . eq_ignore_ascii_case ( f. name ( ) ) )
275
+ . last ( )
276
+ {
277
+ let tf = DataField :: from ( m) ;
278
+ if remove_nullable ( tf. data_type ( ) ) != remove_nullable ( f. data_type ( ) ) {
279
+ let pair = ( f, m) ;
280
+ let diff = pair. make_diff ( "expected_field" , "infer_field" ) ;
281
+ return Err ( ErrorCode :: ParquetError ( format ! (
282
+ "parquet schema mismatch, differ: {}" ,
283
+ diff
284
+ ) ) ) ;
285
+ }
286
+
287
+ read_fields. push ( m. clone ( ) ) ;
288
+ } else {
289
+ return Err ( ErrorCode :: ParquetError ( format ! (
290
+ "schema field size mismatch, expected to find column: {}" ,
291
+ f. name( )
292
+ ) ) ) ;
293
+ }
137
294
}
295
+ Ok ( read_fields)
296
+ }
297
+
298
+ pub fn split_column_metas_by_field (
299
+ columns : & [ ColumnChunkMetaData ] ,
300
+ field_names : & [ & str ] ,
301
+ ) -> Vec < Vec < usize > > {
302
+ let mut r = field_names. iter ( ) . map ( |_| vec ! [ ] ) . collect :: < Vec < _ > > ( ) ;
303
+ let d = field_names
304
+ . iter ( )
305
+ . enumerate ( )
306
+ . map ( |( i, name) | ( name, i) )
307
+ . collect :: < HashMap < _ , _ > > ( ) ;
308
+ columns. iter ( ) . enumerate ( ) . for_each ( |( col_i, x) | {
309
+ if let Some ( field_i) = d. get ( & x. descriptor ( ) . path_in_schema [ 0 ] . as_str ( ) ) {
310
+ r[ * field_i] . push ( col_i) ;
311
+ }
312
+ } ) ;
313
+ r
138
314
}
0 commit comments