1717
1818//! This module provide `DataFileWriter`.
1919
20- use crate :: spec:: { DataContentType , DataFileBuilder } ;
20+ use crate :: spec:: { DataContentType , DataFile , Struct } ;
2121use crate :: writer:: file_writer:: FileWriter ;
2222use crate :: writer:: CurrentFileStatus ;
2323use crate :: writer:: { file_writer:: FileWriterBuilder , IcebergWriter , IcebergWriterBuilder } ;
@@ -38,14 +38,30 @@ impl<B: FileWriterBuilder> DataFileWriterBuilder<B> {
3838 }
3939}
4040
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]
4256impl < B : FileWriterBuilder > IcebergWriterBuilder for DataFileWriterBuilder < B > {
4357 type R = DataFileWriter < B > ;
58+ type C = DataFileWriterConfig ;
4459
45- async fn build ( self ) -> Result < Self :: R > {
60+ async fn build ( self , config : Self :: C ) -> Result < Self :: R > {
4661 Ok ( DataFileWriter {
4762 inner_writer : self . inner . clone ( ) . build ( ) . await ?,
4863 builder : self . inner ,
64+ partition_value : config. partition_value ,
4965 } )
5066 }
5167}
@@ -54,6 +70,7 @@ impl<B: FileWriterBuilder> IcebergWriterBuilder for DataFileWriterBuilder<B> {
5470pub struct DataFileWriter < B : FileWriterBuilder > {
5571 builder : B ,
5672 inner_writer : B :: R ,
73+ partition_value : Struct ,
5774}
5875
5976#[ async_trait:: async_trait]
@@ -62,15 +79,16 @@ impl<B: FileWriterBuilder> IcebergWriter for DataFileWriter<B> {
6279 self . inner_writer . write ( & batch) . await
6380 }
6481
65- async fn flush ( & mut self ) -> Result < Vec < DataFileBuilder > > {
82+ async fn flush ( & mut self ) -> Result < Vec < DataFile > > {
6683 let writer = std:: mem:: replace ( & mut self . inner_writer , self . builder . clone ( ) . build ( ) . await ?) ;
6784 let res = writer
6885 . close ( )
6986 . await ?
7087 . into_iter ( )
7188 . map ( |mut res| {
7289 res. content ( DataContentType :: Data ) ;
73- res
90+ res. partition ( self . partition_value . clone ( ) ) ;
91+ res. build ( ) . expect ( "Guranteed to be valid" )
7492 } )
7593 . collect_vec ( ) ;
7694 Ok ( res)
@@ -101,9 +119,9 @@ mod test {
101119
102120 use crate :: {
103121 io:: FileIOBuilder ,
104- spec:: { DataFileFormat , Struct } ,
122+ spec:: DataFileFormat ,
105123 writer:: {
106- base_writer:: data_file_writer:: DataFileWriterBuilder ,
124+ base_writer:: data_file_writer:: { DataFileWriterBuilder , DataFileWriterConfig } ,
107125 file_writer:: {
108126 location_generator:: { test:: MockLocationGenerator , DefaultFileNameGenerator } ,
109127 ParquetWriterBuilder ,
@@ -141,7 +159,7 @@ mod test {
141159 )
142160 . with_metadata( HashMap :: from( [ (
143161 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
144- "-1 " . to_string( ) ,
162+ "5 " . to_string( ) ,
145163 ) ] ) ) ]
146164 . into( ) ,
147165 ) ,
@@ -160,7 +178,7 @@ mod test {
160178 arrow_schema:: Field :: new( "item" , arrow_schema:: DataType :: Int64 , true )
161179 . with_metadata( HashMap :: from( [ (
162180 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
163- "-1 " . to_string( ) ,
181+ "6 " . to_string( ) ,
164182 ) ] ) ) ,
165183 ) ) ,
166184 true ,
@@ -182,15 +200,15 @@ mod test {
182200 )
183201 . with_metadata( HashMap :: from( [ (
184202 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
185- "-1 " . to_string( ) ,
203+ "7 " . to_string( ) ,
186204 ) ] ) ) ]
187205 . into( ) ,
188206 ) ,
189207 true ,
190208 )
191209 . with_metadata( HashMap :: from( [ (
192210 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
193- "-1 " . to_string( ) ,
211+ "8 " . to_string( ) ,
194212 ) ] ) ) ]
195213 . into( ) ,
196214 ) ,
@@ -209,7 +227,7 @@ mod test {
209227 arrow_schema:: Field :: new( "sub_col" , arrow_schema:: DataType :: Int64 , true )
210228 . with_metadata( HashMap :: from( [ (
211229 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
212- "-1 " . to_string( ) ,
230+ "5 " . to_string( ) ,
213231 ) ] ) ) ,
214232 ]
215233 . into ( ) ,
@@ -231,7 +249,7 @@ mod test {
231249 arrow_array:: ListArray :: new (
232250 Arc :: new ( list_parts. 0 . as_ref ( ) . clone ( ) . with_metadata ( HashMap :: from ( [ (
233251 PARQUET_FIELD_ID_META_KEY . to_string ( ) ,
234- "-1 " . to_string ( ) ,
252+ "6 " . to_string ( ) ,
235253 ) ] ) ) ) ,
236254 list_parts. 1 ,
237255 list_parts. 2 ,
@@ -249,23 +267,23 @@ mod test {
249267 )
250268 . with_metadata( HashMap :: from( [ (
251269 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
252- "-1 " . to_string( ) ,
270+ "7 " . to_string( ) ,
253271 ) ] ) ) ]
254272 . into( ) ,
255273 ) ,
256274 true ,
257275 )
258276 . with_metadata( HashMap :: from( [ (
259277 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
260- "-1 " . to_string( ) ,
278+ "8 " . to_string( ) ,
261279 ) ] ) ) ]
262280 . into ( ) ,
263281 vec ! [ Arc :: new( StructArray :: new(
264282 vec![
265283 arrow_schema:: Field :: new( "sub_sub_col" , arrow_schema:: DataType :: Int64 , true )
266284 . with_metadata( HashMap :: from( [ (
267285 PARQUET_FIELD_ID_META_KEY . to_string( ) ,
268- "-1 " . to_string( ) ,
286+ "7 " . to_string( ) ,
269287 ) ] ) ) ,
270288 ]
271289 . into( ) ,
@@ -285,20 +303,16 @@ mod test {
285303 location_gen,
286304 file_name_gen,
287305 ) ;
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 ?;
289309
290310 for _ in 0 ..3 {
291311 // write
292312 data_file_writer. write ( to_write. clone ( ) ) . await ?;
293313 let res = data_file_writer. flush ( ) . await ?;
294314 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 ( ) ;
302316
303317 // check
304318 check_parquet_data_file ( & file_io, & data_file, & to_write) . await ;
0 commit comments