Skip to content

Commit b85a397

Browse files
Make the BatchSerializer behind Arc to avoid unnecessary struct creation (#8666)
* Make the BatchSerializer behind Arc * Commenting * Review * Incorporate review suggestions * Use old names --------- Co-authored-by: Mehmet Ozan Kabak <[email protected]>
1 parent 8ced56e commit b85a397

File tree

5 files changed

+98
-150
lines changed

5 files changed

+98
-150
lines changed

datafusion/core/src/datasource/file_format/csv.rs

+27-42
Original file line numberDiff line numberDiff line change
@@ -19,21 +19,9 @@
1919
2020
use std::any::Any;
2121
use std::collections::HashSet;
22-
use std::fmt;
23-
use std::fmt::Debug;
22+
use std::fmt::{self, Debug};
2423
use std::sync::Arc;
2524

26-
use arrow_array::RecordBatch;
27-
use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType};
28-
use datafusion_execution::TaskContext;
29-
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};
30-
31-
use bytes::{Buf, Bytes};
32-
use datafusion_physical_plan::metrics::MetricsSet;
33-
use futures::stream::BoxStream;
34-
use futures::{pin_mut, Stream, StreamExt, TryStreamExt};
35-
use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore};
36-
3725
use super::write::orchestration::stateless_multipart_put;
3826
use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD};
3927
use crate::datasource::file_format::file_compression_type::FileCompressionType;
@@ -47,11 +35,20 @@ use crate::physical_plan::insert::{DataSink, FileSinkExec};
4735
use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics};
4836
use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream};
4937

38+
use arrow::array::RecordBatch;
5039
use arrow::csv::WriterBuilder;
5140
use arrow::datatypes::{DataType, Field, Fields, Schema};
5241
use arrow::{self, datatypes::SchemaRef};
42+
use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType};
43+
use datafusion_execution::TaskContext;
44+
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};
45+
use datafusion_physical_plan::metrics::MetricsSet;
5346

5447
use async_trait::async_trait;
48+
use bytes::{Buf, Bytes};
49+
use futures::stream::BoxStream;
50+
use futures::{pin_mut, Stream, StreamExt, TryStreamExt};
51+
use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore};
5552

5653
/// Character Separated Value `FileFormat` implementation.
5754
#[derive(Debug)]
@@ -400,8 +397,6 @@ impl Default for CsvSerializer {
400397
pub struct CsvSerializer {
401398
// CSV writer builder
402399
builder: WriterBuilder,
403-
// Inner buffer for avoiding reallocation
404-
buffer: Vec<u8>,
405400
// Flag to indicate whether there will be a header
406401
header: bool,
407402
}
@@ -412,7 +407,6 @@ impl CsvSerializer {
412407
Self {
413408
builder: WriterBuilder::new(),
414409
header: true,
415-
buffer: Vec::with_capacity(4096),
416410
}
417411
}
418412

@@ -431,21 +425,14 @@ impl CsvSerializer {
431425

432426
#[async_trait]
433427
impl BatchSerializer for CsvSerializer {
434-
async fn serialize(&mut self, batch: RecordBatch) -> Result<Bytes> {
428+
async fn serialize(&self, batch: RecordBatch, initial: bool) -> Result<Bytes> {
429+
let mut buffer = Vec::with_capacity(4096);
435430
let builder = self.builder.clone();
436-
let mut writer = builder.with_header(self.header).build(&mut self.buffer);
431+
let header = self.header && initial;
432+
let mut writer = builder.with_header(header).build(&mut buffer);
437433
writer.write(&batch)?;
438434
drop(writer);
439-
self.header = false;
440-
Ok(Bytes::from(self.buffer.drain(..).collect::<Vec<u8>>()))
441-
}
442-
443-
fn duplicate(&mut self) -> Result<Box<dyn BatchSerializer>> {
444-
let new_self = CsvSerializer::new()
445-
.with_builder(self.builder.clone())
446-
.with_header(self.header);
447-
self.header = false;
448-
Ok(Box::new(new_self))
435+
Ok(Bytes::from(buffer))
449436
}
450437
}
451438

@@ -488,13 +475,11 @@ impl CsvSink {
488475
let builder_clone = builder.clone();
489476
let options_clone = writer_options.clone();
490477
let get_serializer = move || {
491-
let inner_clone = builder_clone.clone();
492-
let serializer: Box<dyn BatchSerializer> = Box::new(
478+
Arc::new(
493479
CsvSerializer::new()
494-
.with_builder(inner_clone)
480+
.with_builder(builder_clone.clone())
495481
.with_header(options_clone.writer_options.header()),
496-
);
497-
serializer
482+
) as _
498483
};
499484

500485
stateless_multipart_put(
@@ -541,15 +526,15 @@ mod tests {
541526
use crate::physical_plan::collect;
542527
use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext};
543528
use crate::test_util::arrow_test_data;
529+
544530
use arrow::compute::concat_batches;
545-
use bytes::Bytes;
546-
use chrono::DateTime;
547531
use datafusion_common::cast::as_string_array;
548-
use datafusion_common::internal_err;
549532
use datafusion_common::stats::Precision;
550-
use datafusion_common::FileType;
551-
use datafusion_common::GetExt;
533+
use datafusion_common::{internal_err, FileType, GetExt};
552534
use datafusion_expr::{col, lit};
535+
536+
use bytes::Bytes;
537+
use chrono::DateTime;
553538
use futures::StreamExt;
554539
use object_store::local::LocalFileSystem;
555540
use object_store::path::Path;
@@ -836,8 +821,8 @@ mod tests {
836821
.collect()
837822
.await?;
838823
let batch = concat_batches(&batches[0].schema(), &batches)?;
839-
let mut serializer = CsvSerializer::new();
840-
let bytes = serializer.serialize(batch).await?;
824+
let serializer = CsvSerializer::new();
825+
let bytes = serializer.serialize(batch, true).await?;
841826
assert_eq!(
842827
"c2,c3\n2,1\n5,-40\n1,29\n1,-85\n5,-82\n4,-111\n3,104\n3,13\n1,38\n4,-38\n",
843828
String::from_utf8(bytes.into()).unwrap()
@@ -860,8 +845,8 @@ mod tests {
860845
.collect()
861846
.await?;
862847
let batch = concat_batches(&batches[0].schema(), &batches)?;
863-
let mut serializer = CsvSerializer::new().with_header(false);
864-
let bytes = serializer.serialize(batch).await?;
848+
let serializer = CsvSerializer::new().with_header(false);
849+
let bytes = serializer.serialize(batch, true).await?;
865850
assert_eq!(
866851
"2,1\n5,-40\n1,29\n1,-85\n5,-82\n4,-111\n3,104\n3,13\n1,38\n4,-38\n",
867852
String::from_utf8(bytes.into()).unwrap()

datafusion/core/src/datasource/file_format/json.rs

+30-47
Original file line numberDiff line numberDiff line change
@@ -23,40 +23,34 @@ use std::fmt::Debug;
2323
use std::io::BufReader;
2424
use std::sync::Arc;
2525

26-
use super::{FileFormat, FileScanConfig};
27-
use arrow::datatypes::Schema;
28-
use arrow::datatypes::SchemaRef;
29-
use arrow::json;
30-
use arrow::json::reader::infer_json_schema_from_iterator;
31-
use arrow::json::reader::ValueIter;
32-
use arrow_array::RecordBatch;
33-
use async_trait::async_trait;
34-
use bytes::Buf;
35-
36-
use bytes::Bytes;
37-
use datafusion_physical_expr::PhysicalExpr;
38-
use datafusion_physical_expr::PhysicalSortRequirement;
39-
use datafusion_physical_plan::ExecutionPlan;
40-
use object_store::{GetResultPayload, ObjectMeta, ObjectStore};
41-
42-
use crate::datasource::physical_plan::FileGroupDisplay;
43-
use crate::physical_plan::insert::DataSink;
44-
use crate::physical_plan::insert::FileSinkExec;
45-
use crate::physical_plan::SendableRecordBatchStream;
46-
use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics};
47-
4826
use super::write::orchestration::stateless_multipart_put;
49-
27+
use super::{FileFormat, FileScanConfig};
5028
use crate::datasource::file_format::file_compression_type::FileCompressionType;
5129
use crate::datasource::file_format::write::BatchSerializer;
5230
use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD;
31+
use crate::datasource::physical_plan::FileGroupDisplay;
5332
use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec};
5433
use crate::error::Result;
5534
use crate::execution::context::SessionState;
35+
use crate::physical_plan::insert::{DataSink, FileSinkExec};
36+
use crate::physical_plan::{
37+
DisplayAs, DisplayFormatType, SendableRecordBatchStream, Statistics,
38+
};
5639

40+
use arrow::datatypes::Schema;
41+
use arrow::datatypes::SchemaRef;
42+
use arrow::json;
43+
use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter};
44+
use arrow_array::RecordBatch;
5745
use datafusion_common::{not_impl_err, DataFusionError, FileType};
5846
use datafusion_execution::TaskContext;
47+
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};
5948
use datafusion_physical_plan::metrics::MetricsSet;
49+
use datafusion_physical_plan::ExecutionPlan;
50+
51+
use async_trait::async_trait;
52+
use bytes::{Buf, Bytes};
53+
use object_store::{GetResultPayload, ObjectMeta, ObjectStore};
6054

6155
/// New line delimited JSON `FileFormat` implementation.
6256
#[derive(Debug)]
@@ -201,31 +195,22 @@ impl Default for JsonSerializer {
201195
}
202196

203197
/// Define a struct for serializing Json records to a stream
204-
pub struct JsonSerializer {
205-
// Inner buffer for avoiding reallocation
206-
buffer: Vec<u8>,
207-
}
198+
pub struct JsonSerializer {}
208199

209200
impl JsonSerializer {
210201
/// Constructor for the JsonSerializer object
211202
pub fn new() -> Self {
212-
Self {
213-
buffer: Vec::with_capacity(4096),
214-
}
203+
Self {}
215204
}
216205
}
217206

218207
#[async_trait]
219208
impl BatchSerializer for JsonSerializer {
220-
async fn serialize(&mut self, batch: RecordBatch) -> Result<Bytes> {
221-
let mut writer = json::LineDelimitedWriter::new(&mut self.buffer);
209+
async fn serialize(&self, batch: RecordBatch, _initial: bool) -> Result<Bytes> {
210+
let mut buffer = Vec::with_capacity(4096);
211+
let mut writer = json::LineDelimitedWriter::new(&mut buffer);
222212
writer.write(&batch)?;
223-
//drop(writer);
224-
Ok(Bytes::from(self.buffer.drain(..).collect::<Vec<u8>>()))
225-
}
226-
227-
fn duplicate(&mut self) -> Result<Box<dyn BatchSerializer>> {
228-
Ok(Box::new(JsonSerializer::new()))
213+
Ok(Bytes::from(buffer))
229214
}
230215
}
231216

@@ -272,10 +257,7 @@ impl JsonSink {
272257
let writer_options = self.config.file_type_writer_options.try_into_json()?;
273258
let compression = &writer_options.compression;
274259

275-
let get_serializer = move || {
276-
let serializer: Box<dyn BatchSerializer> = Box::new(JsonSerializer::new());
277-
serializer
278-
};
260+
let get_serializer = move || Arc::new(JsonSerializer::new()) as _;
279261

280262
stateless_multipart_put(
281263
data,
@@ -312,16 +294,17 @@ impl DataSink for JsonSink {
312294
#[cfg(test)]
313295
mod tests {
314296
use super::super::test_util::scan_format;
315-
use datafusion_common::cast::as_int64_array;
316-
use datafusion_common::stats::Precision;
317-
use futures::StreamExt;
318-
use object_store::local::LocalFileSystem;
319-
320297
use super::*;
321298
use crate::physical_plan::collect;
322299
use crate::prelude::{SessionConfig, SessionContext};
323300
use crate::test::object_store::local_unpartitioned_file;
324301

302+
use datafusion_common::cast::as_int64_array;
303+
use datafusion_common::stats::Precision;
304+
305+
use futures::StreamExt;
306+
use object_store::local::LocalFileSystem;
307+
325308
#[tokio::test]
326309
async fn read_small_batches() -> Result<()> {
327310
let config = SessionConfig::new().with_batch_size(2);

datafusion/core/src/datasource/file_format/write/mod.rs

+4-12
Original file line numberDiff line numberDiff line change
@@ -24,20 +24,16 @@ use std::sync::Arc;
2424
use std::task::{Context, Poll};
2525

2626
use crate::datasource::file_format::file_compression_type::FileCompressionType;
27-
2827
use crate::error::Result;
2928

3029
use arrow_array::RecordBatch;
31-
3230
use datafusion_common::DataFusionError;
3331

3432
use async_trait::async_trait;
3533
use bytes::Bytes;
36-
3734
use futures::future::BoxFuture;
3835
use object_store::path::Path;
3936
use object_store::{MultipartId, ObjectStore};
40-
4137
use tokio::io::AsyncWrite;
4238

4339
pub(crate) mod demux;
@@ -149,15 +145,11 @@ impl<W: AsyncWrite + Unpin + Send> AsyncWrite for AbortableWrite<W> {
149145

150146
/// A trait that defines the methods required for a RecordBatch serializer.
151147
#[async_trait]
152-
pub trait BatchSerializer: Unpin + Send {
148+
pub trait BatchSerializer: Sync + Send {
153149
/// Asynchronously serializes a `RecordBatch` and returns the serialized bytes.
154-
async fn serialize(&mut self, batch: RecordBatch) -> Result<Bytes>;
155-
/// Duplicates self to support serializing multiple batches in parallel on multiple cores
156-
fn duplicate(&mut self) -> Result<Box<dyn BatchSerializer>> {
157-
Err(DataFusionError::NotImplemented(
158-
"Parallel serialization is not implemented for this file type".into(),
159-
))
160-
}
150+
/// Parameter `initial` signals whether the given batch is the first batch.
151+
/// This distinction is important for certain serializers (like CSV).
152+
async fn serialize(&self, batch: RecordBatch, initial: bool) -> Result<Bytes>;
161153
}
162154

163155
/// Returns an [`AbortableWrite`] which writes to the given object store location

0 commit comments

Comments
 (0)