@@ -48,7 +48,7 @@ use crate::file::properties::{WriterProperties, WriterPropertiesPtr};
4848use crate :: file:: reader:: { ChunkReader , Length } ;
4949use crate :: file:: writer:: { SerializedFileWriter , SerializedRowGroupWriter } ;
5050use crate :: parquet_thrift:: { ThriftCompactOutputProtocol , WriteThrift } ;
51- use crate :: schema:: types:: { ColumnDescPtr , SchemaDescriptor } ;
51+ use crate :: schema:: types:: { ColumnDescPtr , SchemaDescPtr , SchemaDescriptor } ;
5252use levels:: { ArrayLevels , calculate_array_levels} ;
5353
5454mod byte_array;
@@ -252,7 +252,7 @@ impl<W: Write + Send> ArrowWriter<W> {
252252 SerializedFileWriter :: new ( writer, schema. root_schema_ptr ( ) , Arc :: clone ( & props_ptr) ) ?;
253253
254254 let row_group_writer_factory =
255- ArrowRowGroupWriterFactory :: new ( & file_writer, schema , arrow_schema. clone ( ) , props_ptr ) ;
255+ ArrowRowGroupWriterFactory :: new ( & file_writer, arrow_schema. clone ( ) ) ;
256256
257257 Ok ( Self {
258258 writer : file_writer,
@@ -423,7 +423,10 @@ impl<W: Write + Send> ArrowWriter<W> {
423423 }
424424
425425 /// Create a new row group writer and return its column writers.
426- #[ deprecated( since = "56.2.0" , note = "Use into_serialized_writer instead" ) ]
426+ #[ deprecated(
427+ since = "56.2.0" ,
428+ note = "Use `ArrowRowGroupWriterFactory` instead, see `ArrowColumnWriter` for an example"
429+ ) ]
427430 pub fn get_column_writers ( & mut self ) -> Result < Vec < ArrowColumnWriter > > {
428431 self . flush ( ) ?;
429432 let in_progress = self
@@ -433,7 +436,10 @@ impl<W: Write + Send> ArrowWriter<W> {
433436 }
434437
435438 /// Append the given column chunks to the file as a new row group.
436- #[ deprecated( since = "56.2.0" , note = "Use into_serialized_writer instead" ) ]
439+ #[ deprecated(
440+ since = "56.2.0" ,
441+ note = "Use `SerializedFileWriter` directly instead, see `ArrowColumnWriter` for an example"
442+ ) ]
437443 pub fn append_row_group ( & mut self , chunks : Vec < ArrowColumnChunk > ) -> Result < ( ) > {
438444 let mut row_group_writer = self . writer . next_row_group ( ) ?;
439445 for chunk in chunks {
@@ -445,6 +451,10 @@ impl<W: Write + Send> ArrowWriter<W> {
445451
446452 /// Converts this writer into a lower-level [`SerializedFileWriter`] and [`ArrowRowGroupWriterFactory`].
447453 /// This can be useful to provide more control over how files are written.
454+ #[ deprecated(
455+ since = "57.0.0" ,
456+ note = "Construct a `SerializedFileWriter` and `ArrowRowGroupWriterFactory` directly instead"
457+ ) ]
448458 pub fn into_serialized_writer (
449459 mut self ,
450460 ) -> Result < ( SerializedFileWriter < W > , ArrowRowGroupWriterFactory ) > {
@@ -693,6 +703,8 @@ impl ArrowColumnChunk {
693703
694704/// Encodes [`ArrowLeafColumn`] to [`ArrowColumnChunk`]
695705///
706+ /// `ArrowColumnWriter` instances can be created using an [`ArrowRowGroupWriterFactory`];
707+ ///
696708/// Note: This is a low-level interface for applications that require
697709/// fine-grained control of encoding (e.g. encoding using multiple threads),
698710/// see [`ArrowWriter`] for a higher-level interface
@@ -704,7 +716,7 @@ impl ArrowColumnChunk {
704716/// # use arrow_array::*;
705717/// # use arrow_schema::*;
706718/// # use parquet::arrow::ArrowSchemaConverter;
707- /// # use parquet::arrow::arrow_writer::{ArrowLeafColumn, compute_leaves, get_column_writers, ArrowColumnChunk };
719+ /// # use parquet::arrow::arrow_writer::{compute_leaves, ArrowColumnChunk, ArrowLeafColumn, ArrowRowGroupWriterFactory };
708720/// # use parquet::file::properties::WriterProperties;
709721/// # use parquet::file::writer::{SerializedFileWriter, SerializedRowGroupWriter};
710722/// #
@@ -720,8 +732,17 @@ impl ArrowColumnChunk {
720732/// .convert(&schema)
721733/// .unwrap();
722734///
723- /// // Create writers for each of the leaf columns
724- /// let col_writers = get_column_writers(&parquet_schema, &props, &schema).unwrap();
735+ /// // Create parquet writer
736+ /// let root_schema = parquet_schema.root_schema_ptr();
737+ /// // write to memory in the example, but this could be a File
738+ /// let mut out = Vec::with_capacity(1024);
739+ /// let mut writer = SerializedFileWriter::new(&mut out, root_schema, props.clone())
740+ /// .unwrap();
741+ ///
742+ /// // Create a factory for building Arrow column writers
743+ /// let row_group_factory = ArrowRowGroupWriterFactory::new(&writer, Arc::clone(&schema));
744+ /// // Create column writers for the 0th row group
745+ /// let col_writers = row_group_factory.create_column_writers(0).unwrap();
725746///
726747/// // Spawn a worker thread for each column
727748/// //
@@ -744,13 +765,6 @@ impl ArrowColumnChunk {
744765/// })
745766/// .collect();
746767///
747- /// // Create parquet writer
748- /// let root_schema = parquet_schema.root_schema_ptr();
749- /// // write to memory in the example, but this could be a File
750- /// let mut out = Vec::with_capacity(1024);
751- /// let mut writer = SerializedFileWriter::new(&mut out, root_schema, props.clone())
752- /// .unwrap();
753- ///
754768/// // Start row group
755769/// let mut row_group_writer: SerializedRowGroupWriter<'_, _> = writer
756770/// .next_row_group()
@@ -894,69 +908,65 @@ impl ArrowRowGroupWriter {
894908
895909/// Factory that creates new column writers for each row group in the Parquet file.
896910pub struct ArrowRowGroupWriterFactory {
897- schema : SchemaDescriptor ,
911+ schema : SchemaDescPtr ,
898912 arrow_schema : SchemaRef ,
899913 props : WriterPropertiesPtr ,
900914 #[ cfg( feature = "encryption" ) ]
901915 file_encryptor : Option < Arc < FileEncryptor > > ,
902916}
903917
904918impl ArrowRowGroupWriterFactory {
905- # [ cfg ( feature = "encryption" ) ]
906- fn new < W : Write + Send > (
919+ /// Create a new [`ArrowRowGroupWriterFactory`] for the provided file writer and Arrow schema
920+ pub fn new < W : Write + Send > (
907921 file_writer : & SerializedFileWriter < W > ,
908- schema : SchemaDescriptor ,
909922 arrow_schema : SchemaRef ,
910- props : WriterPropertiesPtr ,
911923 ) -> Self {
924+ let schema = Arc :: clone ( file_writer. schema_descr_ptr ( ) ) ;
925+ let props = Arc :: clone ( file_writer. properties ( ) ) ;
912926 Self {
913927 schema,
914928 arrow_schema,
915929 props,
930+ #[ cfg( feature = "encryption" ) ]
916931 file_encryptor : file_writer. file_encryptor ( ) ,
917932 }
918933 }
919934
920- #[ cfg( not( feature = "encryption" ) ) ]
921- fn new < W : Write + Send > (
922- _file_writer : & SerializedFileWriter < W > ,
923- schema : SchemaDescriptor ,
924- arrow_schema : SchemaRef ,
925- props : WriterPropertiesPtr ,
926- ) -> Self {
927- Self {
928- schema,
929- arrow_schema,
930- props,
935+ fn create_row_group_writer ( & self , row_group_index : usize ) -> Result < ArrowRowGroupWriter > {
936+ let writers = self . create_column_writers ( row_group_index) ?;
937+ Ok ( ArrowRowGroupWriter :: new ( writers, & self . arrow_schema ) )
938+ }
939+
940+ /// Create column writers for a new row group.
941+ pub fn create_column_writers ( & self , row_group_index : usize ) -> Result < Vec < ArrowColumnWriter > > {
942+ let mut writers = Vec :: with_capacity ( self . arrow_schema . fields . len ( ) ) ;
943+ let mut leaves = self . schema . columns ( ) . iter ( ) ;
944+ let column_factory = self . column_writer_factory ( row_group_index) ;
945+ for field in & self . arrow_schema . fields {
946+ column_factory. get_arrow_column_writer (
947+ field. data_type ( ) ,
948+ & self . props ,
949+ & mut leaves,
950+ & mut writers,
951+ ) ?;
931952 }
953+ Ok ( writers)
932954 }
933955
934956 #[ cfg( feature = "encryption" ) ]
935- fn create_row_group_writer ( & self , row_group_index : usize ) -> Result < ArrowRowGroupWriter > {
936- let writers = get_column_writers_with_encryptor (
937- & self . schema ,
938- & self . props ,
939- & self . arrow_schema ,
940- self . file_encryptor . clone ( ) ,
941- row_group_index,
942- ) ?;
943- Ok ( ArrowRowGroupWriter :: new ( writers, & self . arrow_schema ) )
957+ fn column_writer_factory ( & self , row_group_idx : usize ) -> ArrowColumnWriterFactory {
958+ ArrowColumnWriterFactory :: new ( )
959+ . with_file_encryptor ( row_group_idx, self . file_encryptor . clone ( ) )
944960 }
945961
946962 #[ cfg( not( feature = "encryption" ) ) ]
947- fn create_row_group_writer ( & self , _row_group_index : usize ) -> Result < ArrowRowGroupWriter > {
948- let writers = get_column_writers ( & self . schema , & self . props , & self . arrow_schema ) ?;
949- Ok ( ArrowRowGroupWriter :: new ( writers, & self . arrow_schema ) )
950- }
951-
952- /// Create column writers for a new row group.
953- pub fn create_column_writers ( & self , row_group_index : usize ) -> Result < Vec < ArrowColumnWriter > > {
954- let rg_writer = self . create_row_group_writer ( row_group_index) ?;
955- Ok ( rg_writer. writers )
963+ fn column_writer_factory ( & self , _row_group_idx : usize ) -> ArrowColumnWriterFactory {
964+ ArrowColumnWriterFactory :: new ( )
956965 }
957966}
958967
959968/// Returns [`ArrowColumnWriter`]s for each column in a given schema
969+ #[ deprecated( since = "57.0.0" , note = "Use `ArrowRowGroupWriterFactory` instead" ) ]
960970pub fn get_column_writers (
961971 parquet : & SchemaDescriptor ,
962972 props : & WriterPropertiesPtr ,
@@ -976,30 +986,6 @@ pub fn get_column_writers(
976986 Ok ( writers)
977987}
978988
979- /// Returns the [`ArrowColumnWriter`] for a given schema and supports columnar encryption
980- #[ cfg( feature = "encryption" ) ]
981- fn get_column_writers_with_encryptor (
982- parquet : & SchemaDescriptor ,
983- props : & WriterPropertiesPtr ,
984- arrow : & SchemaRef ,
985- file_encryptor : Option < Arc < FileEncryptor > > ,
986- row_group_index : usize ,
987- ) -> Result < Vec < ArrowColumnWriter > > {
988- let mut writers = Vec :: with_capacity ( arrow. fields . len ( ) ) ;
989- let mut leaves = parquet. columns ( ) . iter ( ) ;
990- let column_factory =
991- ArrowColumnWriterFactory :: new ( ) . with_file_encryptor ( row_group_index, file_encryptor) ;
992- for field in & arrow. fields {
993- column_factory. get_arrow_column_writer (
994- field. data_type ( ) ,
995- props,
996- & mut leaves,
997- & mut writers,
998- ) ?;
999- }
1000- Ok ( writers)
1001- }
1002-
1003989/// Creates [`ArrowColumnWriter`] instances
1004990struct ArrowColumnWriterFactory {
1005991 #[ cfg( feature = "encryption" ) ]
0 commit comments