diff --git a/parquet/src/arrow/arrow_writer/mod.rs b/parquet/src/arrow/arrow_writer/mod.rs index c2a7a6376f9f..3e3c9108d59c 100644 --- a/parquet/src/arrow/arrow_writer/mod.rs +++ b/parquet/src/arrow/arrow_writer/mod.rs @@ -450,11 +450,11 @@ impl ArrowWriter { } /// Converts this writer into a lower-level [`SerializedFileWriter`] and [`ArrowRowGroupWriterFactory`]. - /// This can be useful to provide more control over how files are written. - #[deprecated( - since = "57.0.0", - note = "Construct a `SerializedFileWriter` and `ArrowRowGroupWriterFactory` directly instead" - )] + /// + /// Flushes any outstanding data before returning. + /// + /// This can be useful to provide more control over how files are written, for example + /// to write columns in parallel. See the example on [`ArrowColumnWriter`]. pub fn into_serialized_writer( mut self, ) -> Result<(SerializedFileWriter, ArrowRowGroupWriterFactory)> { @@ -872,6 +872,12 @@ impl ArrowColumnWriter { } /// Encodes [`RecordBatch`] to a parquet row group +/// +/// Note: this structure is created by [`ArrowRowGroupWriterFactory`] internally used to +/// create [`ArrowRowGroupWriter`]s, but it is not exposed publicly. +/// +/// See the example on [`ArrowColumnWriter`] for how to encode columns in parallel +#[derive(Debug)] struct ArrowRowGroupWriter { writers: Vec, schema: SchemaRef, @@ -907,6 +913,10 @@ impl ArrowRowGroupWriter { } /// Factory that creates new column writers for each row group in the Parquet file. +/// +/// You can create this structure via an [`ArrowWriter::into_serialized_writer`]. +/// See the example on [`ArrowColumnWriter`] for how to encode columns in parallel +#[derive(Debug)] pub struct ArrowRowGroupWriterFactory { schema: SchemaDescPtr, arrow_schema: SchemaRef, @@ -937,7 +947,7 @@ impl ArrowRowGroupWriterFactory { Ok(ArrowRowGroupWriter::new(writers, &self.arrow_schema)) } - /// Create column writers for a new row group. + /// Create column writers for a new row group, with the given row group index pub fn create_column_writers(&self, row_group_index: usize) -> Result> { let mut writers = Vec::with_capacity(self.arrow_schema.fields.len()); let mut leaves = self.schema.columns().iter();