This is an automated email from the ASF dual-hosted git repository.

alamb pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git


The following commit(s) were added to refs/heads/main by this push:
     new c468056d04 Improve ParquetDecoder docs (#8802)
c468056d04 is described below

commit c468056d04b07dc2e6e250f47992baa01760fef0
Author: Andrew Lamb <[email protected]>
AuthorDate: Tue Nov 11 17:19:36 2025 -0500

    Improve ParquetDecoder docs (#8802)
    
    # Which issue does this PR close?
    
    
    # Rationale for this change
    
    As I keep working with ParquetRecordBatchReader I keep wishing is docs
    were better
    and examples easier to find
    
    Most recently:
    https://github.com/apache/arrow-rs/pull/8715#discussion_r2500937059
    
    
    # What changes are included in this PR?
    
    So let's add more docs
    
    # Are these changes tested?
    
    By CI / Doc check
    
    # Are there any user-facing changes?
    
    No
    
    ---------
    
    Co-authored-by: Ed Seidl <[email protected]>
---
 parquet/src/arrow/arrow_reader/mod.rs | 33 +++++++++++++++++++--------------
 parquet/src/arrow/push_decoder/mod.rs |  3 +--
 2 files changed, 20 insertions(+), 16 deletions(-)

diff --git a/parquet/src/arrow/arrow_reader/mod.rs 
b/parquet/src/arrow/arrow_reader/mod.rs
index 673e9d9d7f..c6f2eaeff4 100644
--- a/parquet/src/arrow/arrow_reader/mod.rs
+++ b/parquet/src/arrow/arrow_reader/mod.rs
@@ -58,9 +58,9 @@ pub mod statistics;
 ///
 /// Most users should use one of the following specializations:
 ///
-/// * synchronous API: [`ParquetRecordBatchReaderBuilder::try_new`]
-/// * `async` API: [`ParquetRecordBatchStreamBuilder::new`]
-/// * decoder API: [`ParquetDecoderBuilder::new`]
+/// * synchronous API: [`ParquetRecordBatchReaderBuilder`]
+/// * `async` API: [`ParquetRecordBatchStreamBuilder`]
+/// * decoder API: [`ParquetPushDecoderBuilder`]
 ///
 /// # Features
 /// * Projection pushdown: [`Self::with_projection`]
@@ -95,8 +95,8 @@ pub mod statistics;
 /// You can read more about this design in the [Querying Parquet with
 /// Millisecond Latency] Arrow blog post.
 ///
-/// [`ParquetRecordBatchStreamBuilder::new`]: 
crate::arrow::async_reader::ParquetRecordBatchStreamBuilder::new
-/// [`ParquetDecoderBuilder::new`]: 
crate::arrow::push_decoder::ParquetPushDecoderBuilder::new
+/// [`ParquetRecordBatchStreamBuilder`]: 
crate::arrow::async_reader::ParquetRecordBatchStreamBuilder
+/// [`ParquetPushDecoderBuilder`]: 
crate::arrow::push_decoder::ParquetPushDecoderBuilder
 /// [Apache Arrow]: https://arrow.apache.org/
 /// [`StatisticsConverter`]: statistics::StatisticsConverter
 /// [Querying Parquet with Millisecond Latency]: 
https://arrow.apache.org/blog/2022/12/26/querying-parquet-with-millisecond-latency/
@@ -746,11 +746,12 @@ impl<T: Debug + ChunkReader> Debug for SyncReader<T> {
     }
 }
 
-/// A synchronous builder used to construct [`ParquetRecordBatchReader`] for a 
file
+/// Creates [`ParquetRecordBatchReader`] for reading Parquet files into Arrow 
[`RecordBatch`]es
 ///
-/// For an async API see 
[`crate::arrow::async_reader::ParquetRecordBatchStreamBuilder`]
-///
-/// See [`ArrowReaderBuilder`] for additional member functions
+/// # See Also
+/// * [`crate::arrow::async_reader::ParquetRecordBatchStreamBuilder`] for an 
async API
+/// * [`crate::arrow::push_decoder::ParquetPushDecoderBuilder`] for a SansIO 
decoder API
+/// * [`ArrowReaderBuilder`] for additional member functions
 pub type ParquetRecordBatchReaderBuilder<T> = 
ArrowReaderBuilder<SyncReader<T>>;
 
 impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> {
@@ -1037,12 +1038,16 @@ impl<T: ChunkReader + 'static> Iterator for 
ReaderPageIterator<T> {
 
 impl<T: ChunkReader + 'static> PageIterator for ReaderPageIterator<T> {}
 
-/// An `Iterator<Item = ArrowResult<RecordBatch>>` that yields [`RecordBatch`]
-/// read from a parquet data source
+/// Reads Parquet data as Arrow [`RecordBatch`]es
+///
+/// This struct implements the [`RecordBatchReader`] trait and is an
+/// `Iterator<Item = ArrowResult<RecordBatch>>` that yields [`RecordBatch`]es.
+///
+/// Typically, either reads from a file or an in memory buffer [`Bytes`]
+///
+/// Created by [`ParquetRecordBatchReaderBuilder`]
 ///
-/// This reader is created by [`ParquetRecordBatchReaderBuilder`], and has all
-/// the buffered state (DataPages, etc) necessary to decode the parquet data 
into
-/// Arrow arrays.
+/// [`Bytes`]: bytes::Bytes
 pub struct ParquetRecordBatchReader {
     array_reader: Box<dyn ArrayReader>,
     schema: SchemaRef,
diff --git a/parquet/src/arrow/push_decoder/mod.rs 
b/parquet/src/arrow/push_decoder/mod.rs
index b26a21132c..3e0b051c55 100644
--- a/parquet/src/arrow/push_decoder/mod.rs
+++ b/parquet/src/arrow/push_decoder/mod.rs
@@ -37,8 +37,7 @@ use std::sync::Arc;
 
 /// A builder for [`ParquetPushDecoder`].
 ///
-/// To create a new decoder, use 
[`ParquetPushDecoderBuilder::try_new_decoder`] and pass
-/// the file length and metadata of the Parquet file to decode.
+/// To create a new decoder, use 
[`ParquetPushDecoderBuilder::try_new_decoder`].
 ///
 /// You can decode the metadata from a Parquet file using either
 /// [`ParquetMetadataReader`] or [`ParquetMetaDataPushDecoder`].

Reply via email to