17
17
18
18
//! This module provide `DataFileWriter`.
19
19
20
- use crate :: spec:: { DataContentType , DataFileBuilder } ;
20
+ use crate :: spec:: { DataContentType , DataFile , Struct } ;
21
21
use crate :: writer:: file_writer:: FileWriter ;
22
22
use crate :: writer:: CurrentFileStatus ;
23
23
use crate :: writer:: { file_writer:: FileWriterBuilder , IcebergWriter , IcebergWriterBuilder } ;
@@ -38,14 +38,30 @@ impl<B: FileWriterBuilder> DataFileWriterBuilder<B> {
38
38
}
39
39
}
40
40
41
- #[ allow( async_fn_in_trait) ]
41
+ /// Config for `DataFileWriter`.
42
+ pub struct DataFileWriterConfig {
43
+ partition_value : Struct ,
44
+ }
45
+
46
+ impl DataFileWriterConfig {
47
+ /// Create a new `DataFileWriterConfig` with partition value.
48
+ pub fn new ( partition_value : Option < Struct > ) -> Self {
49
+ Self {
50
+ partition_value : partition_value. unwrap_or ( Struct :: empty ( ) ) ,
51
+ }
52
+ }
53
+ }
54
+
55
+ #[ async_trait:: async_trait]
42
56
impl < B : FileWriterBuilder > IcebergWriterBuilder for DataFileWriterBuilder < B > {
43
57
type R = DataFileWriter < B > ;
58
+ type C = DataFileWriterConfig ;
44
59
45
- async fn build ( self ) -> Result < Self :: R > {
60
+ async fn build ( self , config : Self :: C ) -> Result < Self :: R > {
46
61
Ok ( DataFileWriter {
47
62
inner_writer : self . inner . clone ( ) . build ( ) . await ?,
48
63
builder : self . inner ,
64
+ partition_value : config. partition_value ,
49
65
} )
50
66
}
51
67
}
@@ -54,6 +70,7 @@ impl<B: FileWriterBuilder> IcebergWriterBuilder for DataFileWriterBuilder<B> {
54
70
pub struct DataFileWriter < B : FileWriterBuilder > {
55
71
builder : B ,
56
72
inner_writer : B :: R ,
73
+ partition_value : Struct ,
57
74
}
58
75
59
76
#[ async_trait:: async_trait]
@@ -62,15 +79,16 @@ impl<B: FileWriterBuilder> IcebergWriter for DataFileWriter<B> {
62
79
self . inner_writer . write ( & batch) . await
63
80
}
64
81
65
- async fn flush ( & mut self ) -> Result < Vec < DataFileBuilder > > {
82
+ async fn flush ( & mut self ) -> Result < Vec < DataFile > > {
66
83
let writer = std:: mem:: replace ( & mut self . inner_writer , self . builder . clone ( ) . build ( ) . await ?) ;
67
84
let res = writer
68
85
. close ( )
69
86
. await ?
70
87
. into_iter ( )
71
88
. map ( |mut res| {
72
89
res. content ( DataContentType :: Data ) ;
73
- res
90
+ res. partition ( self . partition_value . clone ( ) ) ;
91
+ res. build ( ) . expect ( "Guranteed to be valid" )
74
92
} )
75
93
. collect_vec ( ) ;
76
94
Ok ( res)
@@ -101,9 +119,9 @@ mod test {
101
119
102
120
use crate :: {
103
121
io:: FileIOBuilder ,
104
- spec:: { DataFileFormat , Struct } ,
122
+ spec:: DataFileFormat ,
105
123
writer:: {
106
- base_writer:: data_file_writer:: DataFileWriterBuilder ,
124
+ base_writer:: data_file_writer:: { DataFileWriterBuilder , DataFileWriterConfig } ,
107
125
file_writer:: {
108
126
location_generator:: { test:: MockLocationGenerator , DefaultFileNameGenerator } ,
109
127
ParquetWriterBuilder ,
@@ -141,7 +159,7 @@ mod test {
141
159
)
142
160
. with_metadata( HashMap :: from( [ (
143
161
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
144
- "-1 " . to_string( ) ,
162
+ "5 " . to_string( ) ,
145
163
) ] ) ) ]
146
164
. into( ) ,
147
165
) ,
@@ -160,7 +178,7 @@ mod test {
160
178
arrow_schema:: Field :: new( "item" , arrow_schema:: DataType :: Int64 , true )
161
179
. with_metadata( HashMap :: from( [ (
162
180
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
163
- "-1 " . to_string( ) ,
181
+ "6 " . to_string( ) ,
164
182
) ] ) ) ,
165
183
) ) ,
166
184
true ,
@@ -182,15 +200,15 @@ mod test {
182
200
)
183
201
. with_metadata( HashMap :: from( [ (
184
202
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
185
- "-1 " . to_string( ) ,
203
+ "7 " . to_string( ) ,
186
204
) ] ) ) ]
187
205
. into( ) ,
188
206
) ,
189
207
true ,
190
208
)
191
209
. with_metadata( HashMap :: from( [ (
192
210
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
193
- "-1 " . to_string( ) ,
211
+ "8 " . to_string( ) ,
194
212
) ] ) ) ]
195
213
. into( ) ,
196
214
) ,
@@ -209,7 +227,7 @@ mod test {
209
227
arrow_schema:: Field :: new( "sub_col" , arrow_schema:: DataType :: Int64 , true )
210
228
. with_metadata( HashMap :: from( [ (
211
229
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
212
- "-1 " . to_string( ) ,
230
+ "5 " . to_string( ) ,
213
231
) ] ) ) ,
214
232
]
215
233
. into ( ) ,
@@ -231,7 +249,7 @@ mod test {
231
249
arrow_array:: ListArray :: new (
232
250
Arc :: new ( list_parts. 0 . as_ref ( ) . clone ( ) . with_metadata ( HashMap :: from ( [ (
233
251
PARQUET_FIELD_ID_META_KEY . to_string ( ) ,
234
- "-1 " . to_string ( ) ,
252
+ "6 " . to_string ( ) ,
235
253
) ] ) ) ) ,
236
254
list_parts. 1 ,
237
255
list_parts. 2 ,
@@ -249,23 +267,23 @@ mod test {
249
267
)
250
268
. with_metadata( HashMap :: from( [ (
251
269
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
252
- "-1 " . to_string( ) ,
270
+ "7 " . to_string( ) ,
253
271
) ] ) ) ]
254
272
. into( ) ,
255
273
) ,
256
274
true ,
257
275
)
258
276
. with_metadata( HashMap :: from( [ (
259
277
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
260
- "-1 " . to_string( ) ,
278
+ "8 " . to_string( ) ,
261
279
) ] ) ) ]
262
280
. into ( ) ,
263
281
vec ! [ Arc :: new( StructArray :: new(
264
282
vec![
265
283
arrow_schema:: Field :: new( "sub_sub_col" , arrow_schema:: DataType :: Int64 , true )
266
284
. with_metadata( HashMap :: from( [ (
267
285
PARQUET_FIELD_ID_META_KEY . to_string( ) ,
268
- "-1 " . to_string( ) ,
286
+ "7 " . to_string( ) ,
269
287
) ] ) ) ,
270
288
]
271
289
. into( ) ,
@@ -285,20 +303,16 @@ mod test {
285
303
location_gen,
286
304
file_name_gen,
287
305
) ;
288
- let mut data_file_writer = DataFileWriterBuilder :: new ( pb) . build ( ) . await ?;
306
+ let mut data_file_writer = DataFileWriterBuilder :: new ( pb)
307
+ . build ( DataFileWriterConfig :: new ( None ) )
308
+ . await ?;
289
309
290
310
for _ in 0 ..3 {
291
311
// write
292
312
data_file_writer. write ( to_write. clone ( ) ) . await ?;
293
313
let res = data_file_writer. flush ( ) . await ?;
294
314
assert_eq ! ( res. len( ) , 1 ) ;
295
- let data_file = res
296
- . into_iter ( )
297
- . next ( )
298
- . unwrap ( )
299
- . partition ( Struct :: empty ( ) )
300
- . build ( )
301
- . unwrap ( ) ;
315
+ let data_file = res. into_iter ( ) . next ( ) . unwrap ( ) ;
302
316
303
317
// check
304
318
check_parquet_data_file ( & file_io, & data_file, & to_write) . await ;
0 commit comments