You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/10/12 12:42:14 UTC

[GitHub] [arrow] pitrou commented on a diff in pull request #14018: ARROW-14161: [C++][Docs] Improve Parquet C++ docs

pitrou commented on code in PR #14018:
URL: https://github.com/apache/arrow/pull/14018#discussion_r993399368


##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 

Review Comment:
   Perhaps
   ```suggestion
   column level. By default, the writer will attempt to dictionary-encode all 
   ```



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the

Review Comment:
   An upper bound of what? The number of rows, or the number of bytes?



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.

Review Comment:
   ```suggestion
   ``uint16_t`` type.
   ```



##########
cpp/src/arrow/dataset/file_parquet_test.cc:
##########
@@ -112,8 +112,9 @@ class ParquetFormatHelper {
       const std::shared_ptr<ArrowWriterProperties>& arrow_properties =
           default_arrow_writer_properties()) {
     std::unique_ptr<parquet::arrow::FileWriter> writer;
-    RETURN_NOT_OK(parquet::arrow::FileWriter::Open(
-        *reader->schema(), pool, sink, properties, arrow_properties, &writer));
+    EXPECT_OK_AND_ASSIGN(writer,

Review Comment:
   Since this is returning a Status, should instead use `ARROW_ASSIGN_OR_RAISE` here.



##########
cpp/src/parquet/arrow/reader.h:
##########
@@ -180,6 +184,33 @@ class PARQUET_EXPORT FileReader {
   virtual ::arrow::Status GetRecordBatchReader(
       const std::vector<int>& row_group_indices, const std::vector<int>& column_indices,
       std::unique_ptr<::arrow::RecordBatchReader>* out) = 0;
+  ::arrow::Status GetRecordBatchReader(const std::vector<int>& row_group_indices,
+                                       const std::vector<int>& column_indices,
+                                       std::shared_ptr<::arrow::RecordBatchReader>* out);
+
+  /// \brief Return a RecordBatchReader of row groups selected from
+  /// row_group_indices, whose columns are selected by column_indices.
+  ///
+  /// \param row_group_indices indices of which row groups to include.
+  /// \param column_indices indices of columns to include.
+  ///
+  /// \since 10.0.0
+  ::arrow::Result<std::shared_ptr<::arrow::RecordBatchReader>> GetRecordBatchReader(

Review Comment:
   Can we also deprecate the variants taking a pointer-out parameter above?



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 
+supported columns, unless the dictionary grows too large. This behavior can
+be changed at file-level or at the column level with ``disable_dictionary()``.
+When not using dictionary encoding, it will fallback to the encoding set for 
+the column or the overall file; by default ``Encoding::PLAIN``, but this can
+be changed with ``encoding()``.
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using arrow::Compression;
+   using parquet::Encoding;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+     .compression(Compression::SNAPPY)        // Fallback
+     ->compression("colA", Compression::ZSTD) // Only applies to colA
+     ->encoding(Encoding::BIT_PACKED)         // Fallback
+     ->encoding("colB", Encoding::RLE)        // Only applies to colB
+     ->disable_dictionary("colB")             // Always use RLE, never dictionary
+     ->disable_statistics("colB")             // Stats won't be written for colB

Review Comment:
   Is it useful to show how not to write statistics?



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 
+supported columns, unless the dictionary grows too large. This behavior can
+be changed at file-level or at the column level with ``disable_dictionary()``.
+When not using dictionary encoding, it will fallback to the encoding set for 
+the column or the overall file; by default ``Encoding::PLAIN``, but this can
+be changed with ``encoding()``.
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using arrow::Compression;
+   using parquet::Encoding;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+     .compression(Compression::SNAPPY)        // Fallback
+     ->compression("colA", Compression::ZSTD) // Only applies to colA
+     ->encoding(Encoding::BIT_PACKED)         // Fallback
+     ->encoding("colB", Encoding::RLE)        // Only applies to colB

Review Comment:
   ```suggestion
        ->encoding("colB", Encoding::RLE)        // Only applies to column "colB"
   ```



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.

Review Comment:
   Hmm, I'm not sure I understand your question @ksuarez1423 . Would you like to rephrase?



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 
+supported columns, unless the dictionary grows too large. This behavior can
+be changed at file-level or at the column level with ``disable_dictionary()``.
+When not using dictionary encoding, it will fallback to the encoding set for 
+the column or the overall file; by default ``Encoding::PLAIN``, but this can
+be changed with ``encoding()``.
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using arrow::Compression;
+   using parquet::Encoding;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+     .compression(Compression::SNAPPY)        // Fallback
+     ->compression("colA", Compression::ZSTD) // Only applies to colA
+     ->encoding(Encoding::BIT_PACKED)         // Fallback
+     ->encoding("colB", Encoding::RLE)        // Only applies to colB
+     ->disable_dictionary("colB")             // Always use RLE, never dictionary
+     ->disable_statistics("colB")             // Stats won't be written for colB
+     ->build();
+
+Statistics are enabled by default for all columns. You can disable statistics for
+all columns or specific columns using ``disable_statistics`` on the builder.
+There is a ``max_statistics_size`` which limits the maximum number of bytes that
+may be used for min and max values, useful for types like strings or binary blobs.
+
+There are also Arrow-specific settings that can be configured with
+:class:`parquet::ArrowWriterProperties`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+
+   using parquet::ArrowWriterProperties;
+
+   std::shared_ptr<ArrowWriterProperties> arrow_props = ArrowWriterProperties::Builder()
+      .enable_deprecated_int96_timestamps() // default False
+      ->store_schema() // default False
+      ->enable_compliant_nested_types() // default False
+      ->build();
+
+These options mostly dictate how Arrow types are converted to Parquet types.
+Turning on ``store_schema`` will cause the writer to place the serialized Arrow

Review Comment:
   ```suggestion
   Turning on ``store_schema`` will cause the writer to store the serialized Arrow
   ```



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 
+supported columns, unless the dictionary grows too large. This behavior can
+be changed at file-level or at the column level with ``disable_dictionary()``.
+When not using dictionary encoding, it will fallback to the encoding set for 
+the column or the overall file; by default ``Encoding::PLAIN``, but this can
+be changed with ``encoding()``.
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using arrow::Compression;
+   using parquet::Encoding;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+     .compression(Compression::SNAPPY)        // Fallback
+     ->compression("colA", Compression::ZSTD) // Only applies to colA
+     ->encoding(Encoding::BIT_PACKED)         // Fallback
+     ->encoding("colB", Encoding::RLE)        // Only applies to colB
+     ->disable_dictionary("colB")             // Always use RLE, never dictionary

Review Comment:
   ```suggestion
        ->disable_dictionary("colB")             // Never dictionary-encode column "colB"
   ```



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 
+supported columns, unless the dictionary grows too large. This behavior can
+be changed at file-level or at the column level with ``disable_dictionary()``.
+When not using dictionary encoding, it will fallback to the encoding set for 
+the column or the overall file; by default ``Encoding::PLAIN``, but this can
+be changed with ``encoding()``.
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using arrow::Compression;
+   using parquet::Encoding;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+     .compression(Compression::SNAPPY)        // Fallback
+     ->compression("colA", Compression::ZSTD) // Only applies to colA

Review Comment:
   ```suggestion
        ->compression("colA", Compression::ZSTD) // Only applies to column "colA"
   ```



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};

Review Comment:
   I tend to implicitly interpret ``os`` as "output stream", which is misleading here. Just call it ``stream`` perhaps?



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::
+
+   Column compression is off by default in C++. See :ref:`below <parquet-writer-properties>` 
+   for how to choose a compression codec in the writer properties.
+
+To write out data batch-by-batch, use :class:`arrow::FileWriter`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 23-25,32,36
+   :dedent: 2
+
+StreamWriter
+------------
+
+The :class:`StreamWriter` allows for Parquet files to be written using
+standard C++ output operators.  This type-safe approach also ensures
+that rows are written without omitting fields and allows for new row
+groups to be created automatically (after certain volume of data) or
+explicitly by using the :type:`EndRowGroup` stream modifier.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to write a field using an incorrect type.
+
+* Attempt to write too many fields in a row.
+
+* Attempt to skip a required field.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_writer.h"
+
+   {
+      std::shared_ptr<arrow::io::FileOutputStream> outfile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         outfile,
+         arrow::io::FileOutputStream::Open("test.parquet"));
+
+      parquet::WriterProperties::Builder builder;
+      std::shared_ptr<parquet::schema::GroupNode> schema;
+
+      // Set up builder with required compression type etc.
+      // Define schema.
+      // ...
+
+      parquet::StreamWriter os{
+         parquet::ParquetFileWriter::Open(outfile, schema, builder.build())};
+
+      // Loop over some data structure which provides the required
+      // fields to be written and write each row.
+      for (const auto& a : getArticles())
+      {
+         os << a.name() << a.price() << a.quantity() << parquet::EndRow;
+      }
+   }
+
+.. _parquet-writer-properties:
+
+Writer properties
+-----------------
+
+To configure how Parquet files are written, use the :class:`WriterProperties::Builder`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using parquet::ParquetVersion;
+   using parquet::ParquetDataPageVersion;
+   using arrow::Compression;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+      .max_row_group_length(64 * 1024)
+      .created_by("My Application")
+      .version(ParquetVersion::PARQUET_2_6)
+      .data_page_version(ParquetDataPageVersion::V2)
+      .compression(Compression::SNAPPY)
+      .build();
+
+The ``max_row_group_length`` sets an upper bound that takes precedent over the
+``chunk_size`` passed in the write methods.
+
+You can set the version of Parquet to write with ``version``, which determines
+which logical types are available. In addition, you can set the data page version
+with ``data_page_version``. It's V1 by default; setting to V2 will allow more
+optimal compression (skipping compressing pages where there isn't a space 
+benefit), but not all readers support this data page version.
+
+Compression is off by default, but to get the most out of Parquet, you should 
+also choose a compression codec. You can choose one for the whole file or 
+choose one for individual columns. If you choose a mix, the file-level option
+will apply to columns that don't have a specific compression codec. See 
+:class:`::arrow::Compression` for options.
+
+Column data encodings can likewise be applied at the file-level or at the 
+column level. By default, the writer will attempt to dictionary encode all 
+supported columns, unless the dictionary grows too large. This behavior can
+be changed at file-level or at the column level with ``disable_dictionary()``.
+When not using dictionary encoding, it will fallback to the encoding set for 
+the column or the overall file; by default ``Encoding::PLAIN``, but this can
+be changed with ``encoding()``.
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+   #include "arrow/util/type_fwd.h"
+
+   using parquet::WriterProperties;
+   using arrow::Compression;
+   using parquet::Encoding;
+
+   std::shared_ptr<WriterProperties> props = WriterProperties::Builder()
+     .compression(Compression::SNAPPY)        // Fallback
+     ->compression("colA", Compression::ZSTD) // Only applies to colA
+     ->encoding(Encoding::BIT_PACKED)         // Fallback
+     ->encoding("colB", Encoding::RLE)        // Only applies to colB
+     ->disable_dictionary("colB")             // Always use RLE, never dictionary
+     ->disable_statistics("colB")             // Stats won't be written for colB
+     ->build();
+
+Statistics are enabled by default for all columns. You can disable statistics for
+all columns or specific columns using ``disable_statistics`` on the builder.
+There is a ``max_statistics_size`` which limits the maximum number of bytes that
+may be used for min and max values, useful for types like strings or binary blobs.
+
+There are also Arrow-specific settings that can be configured with
+:class:`parquet::ArrowWriterProperties`:
+
+.. code-block:: cpp
+
+   #include "parquet/arrow/writer.h"
+
+   using parquet::ArrowWriterProperties;
+
+   std::shared_ptr<ArrowWriterProperties> arrow_props = ArrowWriterProperties::Builder()
+      .enable_deprecated_int96_timestamps() // default False
+      ->store_schema() // default False
+      ->enable_compliant_nested_types() // default False
+      ->build();
+
+These options mostly dictate how Arrow types are converted to Parquet types.
+Turning on ``store_schema`` will cause the writer to place the serialized Arrow
+schema within the file metadata. This allows the Arrow reader to automatically
+determine which columns should be read back as dictionary-encoded columns,
+potentially saving memory.

Review Comment:
   Not only dictionary encoding is concerned. You can take a look at the existing rules [here](https://github.com/apache/arrow/blob/a9d2504b02f7c40a6c2dbed2a69ab6c447c1fa5b/cpp/src/parquet/arrow/schema.cc#L827-L996).
   
   ```suggestion
   schema within the file metadata. Since there is no bijection between Parquet
   schemas and Arrow schemas, storing the Arrow schema allows the Arrow reader
   to more faithfully recreate the original data.
   ```



##########
docs/source/cpp/parquet.rst:
##########
@@ -32,6 +32,302 @@ is a space-efficient columnar storage format for complex data.  The Parquet
 C++ implementation is part of the Apache Arrow project and benefits
 from tight integration with the Arrow C++ classes and facilities.
 
+Reading Parquet files
+=====================
+
+The :class:`arrow::FileReader` class reads data into Arrow Tables and Record
+Batches.
+
+The :class:`StreamReader` and :class:`StreamWriter` classes allow for
+data to be written using a C++ input/output streams approach to
+read/write fields column by column and row by row.  This approach is
+offered for ease of use and type-safety.  It is of course also useful
+when data must be streamed as files are read and written
+incrementally.
+
+Please note that the performance of the :class:`StreamReader` and
+:class:`StreamWriter` classes will not be as good due to the type
+checking and the fact that column values are processed one at a time.
+
+FileReader
+----------
+
+To read Parquet data into Arrow structures, use :class:`arrow::FileReader`.
+To construct, it requires a :class:`::arrow::io::RandomAccessFile` instance 
+representing the input file. To read the whole file at once, 
+use :func:`arrow::FileReader::ReadTable`:
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 9-10,14
+   :dedent: 2
+
+Finer-grained options are available through the
+:class:`arrow::FileReaderBuilder` helper class, which accepts the :class:`ReaderProperties`
+and :class:`ArrowReaderProperties` classes.
+
+For reading as a stream of batches, use the :func:`arrow::FileReader::GetRecordBatchReader`
+method to retrieve a :class:`arrow::RecordBatchReader`. It will use the batch 
+size set in :class:`ArrowReaderProperties`.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status ReadInBatches(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 25
+   :dedent: 2
+
+.. seealso::
+
+   For reading multi-file datasets or pushing down filters to prune row groups,
+   see :ref:`Tabular Datasets<cpp-dataset>`.
+
+Performance and Memory Efficiency
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+For remote filesystems, use read coalescing (pre-buffering) to reduce number of API calls:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties();
+   reader_properties.set_prebuffer(true);
+
+The defaults are generally tuned towards good performance, but parallel column
+decoding is off by default. Enable it in the constructor of :class:`ArrowReaderProperties`:
+
+.. code-block:: cpp
+
+   auto arrow_reader_props = parquet::ArrowReaderProperties(/*use_threads=*/true);
+
+If memory efficiency is more important than performance, then:
+
+#. Do *not* turn on read coalescing (pre-buffering) in :class:`parquet::ArrowReaderProperties`.
+#. Read data in batches using :func:`arrow::FileReader::GetRecordBatchReader`.
+#. Turn on ``enable_buffered_stream`` in :class:`parquet::ReaderProperties`.
+
+In addition, if you know certain columns contain many repeated values, you can
+read them as :term:`dictionary encoded<dictionary-encoding>` columns. This is 
+enabled with the ``set_read_dictionary`` setting on :class:`ArrowReaderProperties`. 
+If the files were written with Arrow C++ and the ``store_schema`` was activated,
+then the original Arrow schema will be automatically read and will override this
+setting.
+
+StreamReader
+------------
+
+The :class:`StreamReader` allows for Parquet files to be read using
+standard C++ input operators which ensures type-safety.
+
+Please note that types must match the schema exactly i.e. if the
+schema field is an unsigned 16-bit integer then you must supply a
+uint16_t type.
+
+Exceptions are used to signal errors.  A :class:`ParquetException` is
+thrown in the following circumstances:
+
+* Attempt to read field by supplying the incorrect type.
+
+* Attempt to read beyond end of row.
+
+* Attempt to read beyond end of file.
+
+.. code-block:: cpp
+
+   #include "arrow/io/file.h"
+   #include "parquet/stream_reader.h"
+
+   {
+      std::shared_ptr<arrow::io::ReadableFile> infile;
+
+      PARQUET_ASSIGN_OR_THROW(
+         infile,
+         arrow::io::ReadableFile::Open("test.parquet"));
+
+      parquet::StreamReader os{parquet::ParquetFileReader::Open(infile)};
+
+      std::string article;
+      float price;
+      uint32_t quantity;
+
+      while ( !os.eof() )
+      {
+         os >> article >> price >> quantity >> parquet::EndRow;
+         // ...
+      }
+   }
+
+Writing Parquet files
+=====================
+
+WriteTable
+----------
+
+The :func:`arrow::WriteTable` function writes an entire
+:class:`::arrow::Table` to an output file.
+
+.. literalinclude:: ../../../cpp/examples/arrow/parquet_read_write.cc
+   :language: cpp
+   :start-after: arrow::Status WriteFullFile(
+   :end-before: return arrow::Status::OK();
+   :emphasize-lines: 19-21
+   :dedent: 2
+
+.. warning::

Review Comment:
   We should keep warnings for dangerous things, experimental API markers, etc.
   
   ```suggestion
   .. note::
   ```



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org