Skip to content

Commit 2f9efa8

Browse files
committed
i luv cleaning up
1 parent 99af430 commit 2f9efa8

File tree

4 files changed

+54
-34
lines changed

4 files changed

+54
-34
lines changed

Cargo.lock

Lines changed: 1 addition & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

crates/integrations/datafusion/Cargo.toml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ iceberg = { workspace = true }
3737
parquet = { workspace = true }
3838
tokio = { workspace = true }
3939
serde_json = { workspace = true }
40+
uuid = { workspace = true }
4041

4142
[dev-dependencies]
4243
expect-test = { workspace = true }

crates/integrations/datafusion/src/physical_plan/commit.rs

Lines changed: 9 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -80,10 +80,12 @@ impl IcebergCommitExec {
8080
}
8181

8282
// Create a record batch with just the count of rows written
83-
fn make_count_batch(count: u64) -> RecordBatch {
83+
fn make_count_batch(count: u64) -> DFResult<RecordBatch> {
8484
let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef;
8585

86-
RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, false)]).unwrap()
86+
RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, false)]).map_err(|e| {
87+
DataFusionError::ArrowError(e, Some("Failed to make count batch!".to_string()))
88+
})
8789
}
8890

8991
fn make_count_schema() -> ArrowSchemaRef {
@@ -232,10 +234,10 @@ impl ExecutionPlan for IcebergCommitExec {
232234
total_count += count_array.iter().flatten().sum::<u64>();
233235

234236
// Deserialize all data files from the StringArray
235-
let batch_files: Vec<DFResult<DataFile>> = (0..files_array.len())
236-
.map(|i| {
237-
let files_json = files_array.value(i);
238-
serde_json::from_str::<DataFileSerde>(files_json)
237+
let batch_files: Vec<DataFile> = (0..files_array.len())
238+
.map(|i| -> DFResult<DataFile> {
239+
// Parse JSON to DataFileSerde and convert to DataFile
240+
serde_json::from_str::<DataFileSerde>(files_array.value(i))
239241
.map_err(|e| {
240242
DataFusionError::Internal(format!(
241243
"Failed to deserialize data files: {}",
@@ -245,11 +247,6 @@ impl ExecutionPlan for IcebergCommitExec {
245247
.try_into(spec_id, &partition_type, &current_schema)
246248
.map_err(to_datafusion_error)
247249
})
248-
.collect();
249-
250-
// Collect results, propagating any errors
251-
let batch_files: Vec<DataFile> = batch_files
252-
.into_iter()
253250
.collect::<datafusion::common::Result<_>>()?;
254251

255252
// Add all deserialized files to our collection
@@ -275,7 +272,7 @@ impl ExecutionPlan for IcebergCommitExec {
275272
// .await
276273
// .map_err(to_datafusion_error)?;
277274

278-
Ok(Self::make_count_batch(total_count))
275+
Self::make_count_batch(total_count)
279276
})
280277
.boxed();
281278

crates/integrations/datafusion/src/physical_plan/write.rs

Lines changed: 43 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -29,16 +29,20 @@ use datafusion::execution::{SendableRecordBatchStream, TaskContext};
2929
use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
3030
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
3131
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
32-
use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties};
32+
use datafusion::physical_plan::{
33+
DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_input_stream,
34+
};
3335
use futures::StreamExt;
34-
use iceberg::spec::{DataFile, DataFileFormat, DataFileSerde, FormatVersion};
36+
use iceberg::arrow::schema_to_arrow_schema;
37+
use iceberg::spec::{DataFileFormat, DataFileSerde, FormatVersion};
3538
use iceberg::table::Table;
3639
use iceberg::writer::CurrentFileStatus;
3740
use iceberg::writer::file_writer::location_generator::{
3841
DefaultFileNameGenerator, DefaultLocationGenerator,
3942
};
4043
use iceberg::writer::file_writer::{FileWriter, FileWriterBuilder, ParquetWriterBuilder};
4144
use parquet::file::properties::WriterProperties;
45+
use uuid::Uuid;
4246

4347
use crate::to_datafusion_error;
4448

@@ -159,14 +163,24 @@ impl ExecutionPlan for IcebergWriteExec {
159163
self.table.file_io().clone(),
160164
DefaultLocationGenerator::new(self.table.metadata().clone())
161165
.map_err(to_datafusion_error)?,
162-
// todo actual filename
163-
DefaultFileNameGenerator::new("what".to_string(), None, DataFileFormat::Parquet),
166+
// todo filename prefix/suffix should be configurable
167+
DefaultFileNameGenerator::new(
168+
"datafusion".to_string(),
169+
Some(Uuid::now_v7().to_string()),
170+
DataFileFormat::Parquet,
171+
),
164172
)
165173
.build();
166174

167-
// todo repartition
168-
let data = self.input.execute(partition, context)?;
169-
let result_schema = Arc::clone(&self.result_schema);
175+
let data = execute_input_stream(
176+
Arc::clone(&self.input),
177+
Arc::new(
178+
schema_to_arrow_schema(self.table.metadata().current_schema())
179+
.map_err(to_datafusion_error)?,
180+
),
181+
partition,
182+
Arc::clone(&context),
183+
)?;
170184

171185
// todo non-default partition spec?
172186
let spec_id = self.table.metadata().default_partition_spec_id();
@@ -175,7 +189,6 @@ impl ExecutionPlan for IcebergWriteExec {
175189

176190
let stream = futures::stream::once(async move {
177191
let mut writer = parquet_writer_fut.await.map_err(to_datafusion_error)?;
178-
179192
let mut input_stream = data;
180193

181194
while let Some(batch_res) = input_stream.next().await {
@@ -186,28 +199,36 @@ impl ExecutionPlan for IcebergWriteExec {
186199
let count = writer.current_row_num() as u64;
187200
let data_file_builders = writer.close().await.map_err(to_datafusion_error)?;
188201

189-
// Convert builders to data files
190-
let data_files = data_file_builders
191-
.into_iter()
192-
.map(|mut builder| builder.partition_spec_id(spec_id).build().unwrap())
193-
.collect::<Vec<DataFile>>();
194-
195-
let data_files = data_files
202+
// Convert builders to data files and then to JSON strings
203+
let data_files: Vec<String> = data_file_builders
196204
.into_iter()
197-
.map(|f| {
198-
let serde = DataFileSerde::try_from(f, &partition_type, is_version_1).unwrap();
199-
let json = serde_json::to_string(&serde).unwrap();
205+
.map(|mut builder| -> DFResult<String> {
206+
// Build the data file
207+
let data_file = builder.partition_spec_id(spec_id).build().map_err(|e| {
208+
DataFusionError::Execution(format!("Failed to build data file: {}", e))
209+
})?;
210+
211+
// Convert to DataFileSerde
212+
let serde = DataFileSerde::try_from(data_file, &partition_type, is_version_1).map_err(|e| {
213+
DataFusionError::Execution(format!("Failed to convert to DataFileSerde: {}", e))
214+
})?;
215+
216+
// Serialize to JSON
217+
let json = serde_json::to_string(&serde).map_err(|e| {
218+
DataFusionError::Execution(format!("Failed to serialize to JSON: {}", e))
219+
})?;
220+
200221
println!("Serialized data file: {}", json); // todo remove log
201-
json
222+
Ok(json)
202223
})
203-
.collect::<Vec<String>>();
224+
.collect::<DFResult<Vec<String>>>()?;
204225

205-
Ok(Self::make_result_batch(count, data_files)?)
226+
Self::make_result_batch(count, data_files)
206227
})
207228
.boxed();
208229

209230
Ok(Box::pin(RecordBatchStreamAdapter::new(
210-
result_schema,
231+
Arc::clone(&self.result_schema),
211232
stream,
212233
)))
213234
}

0 commit comments

Comments
 (0)