Dandandan commented on code in PR #1537:
URL:
https://github.com/apache/datafusion-ballista/pull/1537#discussion_r3033374775
##########
ballista/core/src/utils.rs:
##########
@@ -159,46 +160,83 @@ pub fn default_config_producer() -> SessionConfig {
SessionConfig::new_with_ballista()
}
-/// Stream data to disk in Arrow IPC format
+/// Stream data to disk in Arrow IPC format.
+///
+/// Batches are read from the async stream and forwarded through a bounded
+/// channel to a `spawn_blocking` task that performs all synchronous file I/O,
+/// keeping the tokio worker thread unblocked.
pub async fn write_stream_to_disk(
stream: &mut Pin<Box<dyn RecordBatchStream + Send>>,
- path: &str,
+ path: &Path,
disk_write_metric: &metrics::Time,
+ channel_capacity: usize,
) -> Result<PartitionStats> {
- let file = BufWriter::new(File::create(path).map_err(|e| {
- error!("Failed to create partition file at {path}: {e:?}");
- BallistaError::IoError(e)
- })?);
-
- let mut num_rows = 0;
- let mut num_batches = 0;
- let mut num_bytes = 0;
+ let schema = stream.schema();
+ let path_owned = path.to_owned();
+ let write_metric = disk_write_metric.clone();
- let options = IpcWriteOptions::default()
- .try_with_compression(Some(CompressionType::LZ4_FRAME))?;
+ let (tx, mut rx) =
tokio::sync::mpsc::channel::<RecordBatch>(channel_capacity);
- let mut writer =
- StreamWriter::try_new_with_options(file, stream.schema().as_ref(),
options)?;
+ let handle = tokio::task::spawn_blocking(move || -> Result<u64> {
+ let file = BufWriter::new(File::create(&path_owned).map_err(|e| {
+ error!(
+ "Failed to create partition file at {}: {e:?}",
+ path_owned.display()
+ );
+ BallistaError::IoError(e)
+ })?);
- while let Some(result) = stream.next().await {
- let batch = result?;
+ let options = IpcWriteOptions::default()
+ .try_with_compression(Some(CompressionType::LZ4_FRAME))?;
- let batch_size_bytes: usize = batch.get_array_memory_size();
- num_batches += 1;
- num_rows += batch.num_rows();
- num_bytes += batch_size_bytes;
+ let mut writer =
Review Comment:
For local disk IO stalling the runtime is usually not a big factor, as there
is not so much it can do at the same time (in the same worker thread). I bet if
you would *only* use spawn_blocking or tokio::fs for writes you would see
smaller gains?
That in itself is interesting to learn.
Maybe I am wrong and mostly projecting my experience with Parquet reads :)
For Shufflereader, i think its a bit tricky given that the current format is
entirely stream based.
It probably would need to write the shuffle files in smaller blocks (e.g. of
50MB or something each) so it can parallelize / morselize reads better.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]