@@ -42,11 +42,11 @@ use iceberg::spec::{
4242} ;
4343use iceberg:: table:: Table ;
4444use iceberg:: writer:: base_writer:: data_file_writer:: DataFileWriterBuilder ;
45- use iceberg:: writer:: base_writer:: rolling_writer:: RollingDataFileWriterBuilder ;
4645use iceberg:: writer:: file_writer:: ParquetWriterBuilder ;
4746use iceberg:: writer:: file_writer:: location_generator:: {
4847 DefaultFileNameGenerator , DefaultLocationGenerator ,
4948} ;
49+ use iceberg:: writer:: file_writer:: rolling_writer:: RollingFileWriterBuilder ;
5050use iceberg:: writer:: { IcebergWriter , IcebergWriterBuilder } ;
5151use iceberg:: { Error , ErrorKind } ;
5252use parquet:: file:: properties:: WriterProperties ;
@@ -205,12 +205,11 @@ impl ExecutionPlan for IcebergWriteExec {
205205 file_format,
206206 ) ,
207207 ) ;
208+ let rolling_writer_builder =
209+ RollingFileWriterBuilder :: new ( parquet_file_writer_builder, 100 * 1024 * 1024 ) ;
210+
208211 let data_file_writer_builder =
209- DataFileWriterBuilder :: new ( parquet_file_writer_builder, None , spec_id) ;
210- let rolling_writer_builder = RollingDataFileWriterBuilder :: new (
211- data_file_writer_builder,
212- 100 * 1024 * 1024 , // todo use a config
213- ) ;
212+ DataFileWriterBuilder :: new ( rolling_writer_builder, None , spec_id) ;
214213
215214 // Get input data
216215 let data = execute_input_stream (
@@ -225,7 +224,7 @@ impl ExecutionPlan for IcebergWriteExec {
225224
226225 // Create write stream
227226 let stream = futures:: stream:: once ( async move {
228- let mut writer = rolling_writer_builder
227+ let mut writer = data_file_writer_builder
229228 . build ( )
230229 . await
231230 . map_err ( to_datafusion_error) ?;
0 commit comments