wjones127 commented on a change in pull request #12112:
URL: https://github.com/apache/arrow/pull/12112#discussion_r831240137
##########
File path: docs/source/python/dataset.rst
##########
@@ -613,6 +613,77 @@ guidelines apply. Row groups can provide parallelism when
reading and allow data
based on statistics, but very small groups can cause metadata to be a
significant portion
of file size. Arrow's file writer provides sensible defaults for group sizing
in most cases.
+Configuring files open during a write
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+When writing data to the disk, there are a few parameters that can be
+important to optimize the writes, such as the number of rows per file and
+the number of files open during write.
+
+Set the maximum number of files opened with the ``max_open_files`` parameter of
+:meth:`write_dataset`.
+
+If ``max_open_files`` is set greater than 0 then this will limit the maximum
+number of files that can be left open. This only applies to writing partitioned
+datasets, where rows are dispatched to the appropriate file depending on their
+partition values. If an attempt is made to open too many files then the least
+recently used file will be closed. If this setting is set too low you may end
+up fragmenting your data into many small files.
+
+If your process is concurrently using other file handlers, either with a
+dataset scanner or otherwise, you may hit a system file handler limit. For
+example, if you are scanning a dataset with 300 files and writing out to
+900 files, the total of 1200 files may be over a system limit. (On Linux,
+this might be a "Too Many Open Files" error.) You can either reduce this
+``max_open_files`` setting or increasing your file handler limit on your
+system. The default value is 900 which also allows some number of files
+to be open by the scanner before hitting the default Linux limit of 1024.
+
+Another important configuration used in :meth:`write_dataset` is
``max_rows_per_file``.
+
+Set the maximum number of rows written in each file with the
``max_rows_per_files``
+parameter of :meth:`write_dataset`.
+
+If ``max_rows_per_file`` is set greater than 0 then this will limit how many
+rows are placed in any single file. Otherwise there will be no limit and one
+file will be created in each output directory unless files need to be closed
to respect
+``max_open_files``. This setting is the primary way to control file size.
+For workloads writing a lot of data, files can get very large without a
+row count cap, leading to out-of-memory errors in downstream readers. The
+relationship between row count and file size depends on the dataset schema
+and how well compressed (if at all) the data is. For most applications,
+it's best to keep file sizes below 1GB.
+
+Configuring rows per group during a write
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+When writing data to disk, depending on the volume of data obtained,
+(in a mini-batch setting where, records are obtained in batch by batch)
+the volume of data written to disk per each group can be configured.
+This can be configured using a minimum and maximum parameter.
+
+Set the maximum number of files opened with the ``min_rows_per_group``
parameter of
+:meth:`write_dataset`.
+
+Note: if ``min_rows_per_group`` is set greater than 0 then this will cause the
+dataset writer to batch incoming data and only write the row groups to the
+disk when sufficient rows have accumulated. The final row group size may be
+less than this value and other options such as ``max_open_files`` or
+``max_rows_per_file`` lead to smaller row group sizes.
+
+Set the maximum number of files opened with the ``max_rows_per_group``
parameter of
+:meth:`write_dataset`.
+
+Note: if ``max_rows_per_group`` is set greater than 0 then the dataset writer
may split
+up large incoming batches into multiple row groups. If this value is set then
+``min_rows_per_group`` should also be set or else you may end up with very
small
+row groups (e.g. if the incoming row group size is just barely larger than
this value).
+In addition row_groups are a factor which impacts write/read of Parquest,
Feather and IPC
+formats. The main purpose of these formats are to provide high performance
data structures
+for I/O operations on larger datasets. The row_group concept allows the
write/read operations
+to be optimized and gather a defined number of rows at once and execute the
I/O operation.
+But row_groups are not integrated to support JSON or CSV formats.
Review comment:
I think it could use a little more direct advice to help users see the
symptoms of when they've done something wrong. Here's my suggestion:
> Row groups are build into the Parquet and IPC/Feather formats, but don't
affect JSON or CSV. When reading back Parquet and IPC formats in Arrow, the row
group boundaries become the record batch boundaries, determining the default
batch size of downstream readers. Additionally, row groups in Parquet files
have column statistics which can help readers skip irrelevant data but can add
size to the file. As an extreme example, if one sets `max_rows_per_group=1` in
Parquet, they will have large files because most of the file will be row group
statistics.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]