@@ -29,7 +29,7 @@ use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
29
29
use datafusion:: physical_plan:: execution_plan:: { Boundedness , EmissionType } ;
30
30
use datafusion:: physical_plan:: stream:: RecordBatchStreamAdapter ;
31
31
use datafusion:: physical_plan:: {
32
- DisplayAs , DisplayFormatType , ExecutionPlan , PlanProperties , execute_stream_partitioned ,
32
+ DisplayAs , DisplayFormatType , ExecutionPlan , ExecutionPlanProperties , PlanProperties ,
33
33
} ;
34
34
use futures:: StreamExt ;
35
35
use iceberg:: Catalog ;
@@ -42,10 +42,11 @@ use crate::to_datafusion_error;
42
42
43
43
/// IcebergCommitExec is responsible for collecting results from multiple IcebergWriteExec
44
44
/// instances and using Transaction::fast_append to commit the data files written.
45
+ #[ derive( Debug ) ]
45
46
pub ( crate ) struct IcebergCommitExec {
46
47
table : Table ,
47
48
catalog : Arc < dyn Catalog > ,
48
- write_plan : Arc < dyn ExecutionPlan > ,
49
+ input : Arc < dyn ExecutionPlan > ,
49
50
schema : ArrowSchemaRef ,
50
51
count_schema : ArrowSchemaRef ,
51
52
plan_properties : PlanProperties ,
@@ -55,15 +56,15 @@ impl IcebergCommitExec {
55
56
pub fn new (
56
57
table : Table ,
57
58
catalog : Arc < dyn Catalog > ,
58
- write_plan : Arc < dyn ExecutionPlan > ,
59
+ input : Arc < dyn ExecutionPlan > ,
59
60
schema : ArrowSchemaRef ,
60
61
) -> Self {
61
62
let plan_properties = Self :: compute_properties ( schema. clone ( ) ) ;
62
63
63
64
Self {
64
65
table,
65
66
catalog,
66
- write_plan ,
67
+ input ,
67
68
schema,
68
69
count_schema : Self :: make_count_schema ( ) ,
69
70
plan_properties,
@@ -99,12 +100,6 @@ impl IcebergCommitExec {
99
100
}
100
101
}
101
102
102
- impl Debug for IcebergCommitExec {
103
- fn fmt ( & self , f : & mut Formatter < ' _ > ) -> std:: fmt:: Result {
104
- write ! ( f, "IcebergCommitExec" )
105
- }
106
- }
107
-
108
103
impl DisplayAs for IcebergCommitExec {
109
104
fn fmt_as ( & self , t : DisplayFormatType , f : & mut Formatter ) -> std:: fmt:: Result {
110
105
match t {
@@ -140,18 +135,18 @@ impl ExecutionPlan for IcebergCommitExec {
140
135
}
141
136
142
137
fn children ( & self ) -> Vec < & Arc < dyn ExecutionPlan > > {
143
- vec ! [ & self . write_plan ]
138
+ vec ! [ & self . input ]
144
139
}
145
140
146
141
fn with_new_children (
147
142
self : Arc < Self > ,
148
143
children : Vec < Arc < dyn ExecutionPlan > > ,
149
144
) -> DFResult < Arc < dyn ExecutionPlan > > {
150
145
if children. len ( ) != 1 {
151
- return Err ( DataFusionError :: Internal (
152
- "IcebergCommitExec expects exactly one child, but provided {children.len()}"
153
- . to_string ( ) ,
154
- ) ) ;
146
+ return Err ( DataFusionError :: Internal ( format ! (
147
+ "IcebergCommitExec expects exactly one child, but provided {}" ,
148
+ children . len ( )
149
+ ) ) ) ;
155
150
}
156
151
157
152
Ok ( Arc :: new ( IcebergCommitExec :: new (
@@ -176,7 +171,7 @@ impl ExecutionPlan for IcebergCommitExec {
176
171
}
177
172
178
173
let table = self . table . clone ( ) ;
179
- let input_plan = self . write_plan . clone ( ) ;
174
+ let input_plan = self . input . clone ( ) ;
180
175
let count_schema = Arc :: clone ( & self . count_schema ) ;
181
176
182
177
// todo revisit this
@@ -191,54 +186,51 @@ impl ExecutionPlan for IcebergCommitExec {
191
186
let mut data_files: Vec < DataFile > = Vec :: new ( ) ;
192
187
let mut total_record_count: u64 = 0 ;
193
188
194
- // Execute and collect results from all partitions of the input plan
195
- let batches = execute_stream_partitioned ( input_plan, context) ?;
196
-
197
- // Collect all data files from this partition's stream
198
- for mut batch_stream in batches {
199
- while let Some ( batch_result) = batch_stream. next ( ) . await {
200
- let batch = batch_result?;
201
-
202
- let files_array = batch
203
- . column_by_name ( DATA_FILES_COL_NAME )
204
- . ok_or_else ( || {
205
- DataFusionError :: Internal (
206
- "Expected 'data_files' column in input batch" . to_string ( ) ,
207
- )
208
- } ) ?
209
- . as_any ( )
210
- . downcast_ref :: < StringArray > ( )
211
- . ok_or_else ( || {
212
- DataFusionError :: Internal (
213
- "Expected 'data_files' column to be StringArray" . to_string ( ) ,
214
- )
215
- } ) ?;
216
-
217
- // todo remove log
218
- println ! ( "files_array to deserialize: {:?}" , files_array) ;
219
-
220
- // Deserialize all data files from the StringArray
221
- let batch_files: Vec < DataFile > = files_array
222
- . into_iter ( )
223
- . flatten ( )
224
- . map ( |f| -> DFResult < DataFile > {
225
- // Parse JSON to DataFileSerde and convert to DataFile
226
- deserialize_data_file_from_json (
227
- f,
228
- spec_id,
229
- & partition_type,
230
- & current_schema,
231
- )
232
- . map_err ( to_datafusion_error)
233
- } )
234
- . collect :: < datafusion:: common:: Result < _ > > ( ) ?;
235
-
236
- // add record_counts from the current batch to total record count
237
- total_record_count += batch_files. iter ( ) . map ( |f| f. record_count ( ) ) . sum :: < u64 > ( ) ;
238
-
239
- // Add all deserialized files to our collection
240
- data_files. extend ( batch_files) ;
241
- }
189
+ // Execute and collect results from the input coalesced plan
190
+ let mut batch_stream = input_plan. execute ( 0 , context) ?;
191
+
192
+ while let Some ( batch_result) = batch_stream. next ( ) . await {
193
+ let batch = batch_result?;
194
+
195
+ let files_array = batch
196
+ . column_by_name ( DATA_FILES_COL_NAME )
197
+ . ok_or_else ( || {
198
+ DataFusionError :: Internal (
199
+ "Expected 'data_files' column in input batch" . to_string ( ) ,
200
+ )
201
+ } ) ?
202
+ . as_any ( )
203
+ . downcast_ref :: < StringArray > ( )
204
+ . ok_or_else ( || {
205
+ DataFusionError :: Internal (
206
+ "Expected 'data_files' column to be StringArray" . to_string ( ) ,
207
+ )
208
+ } ) ?;
209
+
210
+ // todo remove log
211
+ println ! ( "files_array to deserialize: {:?}" , files_array) ;
212
+
213
+ // Deserialize all data files from the StringArray
214
+ let batch_files: Vec < DataFile > = files_array
215
+ . into_iter ( )
216
+ . flatten ( )
217
+ . map ( |f| -> DFResult < DataFile > {
218
+ // Parse JSON to DataFileSerde and convert to DataFile
219
+ deserialize_data_file_from_json (
220
+ f,
221
+ spec_id,
222
+ & partition_type,
223
+ & current_schema,
224
+ )
225
+ . map_err ( to_datafusion_error)
226
+ } )
227
+ . collect :: < datafusion:: common:: Result < _ > > ( ) ?;
228
+
229
+ // add record_counts from the current batch to total record count
230
+ total_record_count += batch_files. iter ( ) . map ( |f| f. record_count ( ) ) . sum :: < u64 > ( ) ;
231
+
232
+ // Add all deserialized files to our collection
233
+ data_files. extend ( batch_files) ;
242
234
}
243
235
244
236
// If no data files were collected, return an empty result
0 commit comments