-
Notifications
You must be signed in to change notification settings - Fork 1.9k
fix: emit empty RecordBatch for empty file writes #19370
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
2a45ed8
1f96288
bccc991
cb7a994
ed62f25
c06a3a2
336f942
461a648
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -349,4 +349,46 @@ mod tests { | |
| fn fmt_batches(batches: &[RecordBatch]) -> String { | ||
| pretty::pretty_format_batches(batches).unwrap().to_string() | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn test_write_empty_json_from_sql() -> Result<()> { | ||
| let ctx = SessionContext::new(); | ||
| let tmp_dir = tempfile::TempDir::new()?; | ||
| let path = format!("{}/empty_sql.json", tmp_dir.path().to_string_lossy()); | ||
| let df = ctx.sql("SELECT CAST(1 AS BIGINT) AS id LIMIT 0").await?; | ||
| df.write_json(&path, crate::dataframe::DataFrameWriteOptions::new(), None) | ||
| .await?; | ||
| // Expected the file to exist and be empty | ||
| assert!(std::path::Path::new(&path).exists()); | ||
| let metadata = std::fs::metadata(&path)?; | ||
| assert_eq!(metadata.len(), 0); | ||
| Ok(()) | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn test_write_empty_json_from_record_batch() -> Result<()> { | ||
| let ctx = SessionContext::new(); | ||
| let schema = Arc::new(Schema::new(vec![ | ||
| Field::new("id", DataType::Int64, false), | ||
| Field::new("name", DataType::Utf8, true), | ||
| ])); | ||
| let empty_batch = RecordBatch::try_new( | ||
| schema.clone(), | ||
| vec![ | ||
| Arc::new(arrow::array::Int64Array::from(Vec::<i64>::new())), | ||
| Arc::new(arrow::array::StringArray::from(Vec::<Option<&str>>::new())), | ||
| ], | ||
| )?; | ||
|
|
||
| let tmp_dir = tempfile::TempDir::new()?; | ||
| let path = format!("{}/empty_batch.json", tmp_dir.path().to_string_lossy()); | ||
| let df = ctx.read_batch(empty_batch.clone())?; | ||
| df.write_json(&path, crate::dataframe::DataFrameWriteOptions::new(), None) | ||
| .await?; | ||
| // Expected the file to exist and be empty | ||
| assert!(std::path::Path::new(&path).exists()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Curious why there is no assertion of 0 lines here like you did for arrow, parquet files?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For JSON, empty files can't be read back to validate schema, Add file check |
||
| let metadata = std::fs::metadata(&path)?; | ||
| assert_eq!(metadata.len(), 0); | ||
| Ok(()) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -191,7 +191,11 @@ async fn row_count_demuxer( | |
| part_idx += 1; | ||
| } | ||
|
|
||
| let schema = input.schema(); | ||
| let mut is_batch_received = false; | ||
|
|
||
| while let Some(rb) = input.next().await.transpose()? { | ||
| is_batch_received = true; | ||
| // ensure we have at least minimum_parallel_files open | ||
| if open_file_streams.len() < minimum_parallel_files { | ||
| open_file_streams.push(create_new_file_stream( | ||
|
|
@@ -228,6 +232,19 @@ async fn row_count_demuxer( | |
|
|
||
| next_send_steam = (next_send_steam + 1) % minimum_parallel_files; | ||
| } | ||
|
|
||
| // if there is no batch send but with a single file, send an empty batch | ||
| if single_file_output && !is_batch_received { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. Updated the doc comment for with_single_file_output() |
||
| open_file_streams | ||
| .first_mut() | ||
| .ok_or_else(|| internal_datafusion_err!("Expected a single output file"))? | ||
| .send(RecordBatch::new_empty(schema)) | ||
| .await | ||
| .map_err(|_| { | ||
| exec_datafusion_err!("Error sending empty RecordBatch to file stream!") | ||
| })?; | ||
| } | ||
|
|
||
| Ok(()) | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Curious why there is no assertion of 0 lines here like you did for arrow, parquet files?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I missed it. Added assertions now to verify schema and 0 rows for CSV