[DISCUSSION] Rethinking default read_parquet partitioning
See original GitHub issueThis is mostly a brain dump, so I realize this may be difficult to process. With that said, I welcome any thoughts and feedback (both positive and negative)
I spent some time yesterday working through the following exercise: If I were defining and writing the dask.dataframe.read_parquet
API from scratch, what would it look like?
I experimented for a few hours before arriving at an API that was not all that different from what we have today. The key difference was that my “ideal” API was prioritizing efficient* partition sizes, while dd.read_parquet
is only prioritizing metadata-processing latency. That is, dd.read_parquet
is doing its very best to avoid processing Parquet metadata, just in case the data is in remote storage. Although this decision makes sense in context, I believe we are much better off prioritizing efficient partition sizes, and warning (and instructing) the user (ahead of time) when we expect the defaults to case pain.
To better-prioritize efficient partitioning, I suspect we could make the following changes:
- Remove
split_row_groups
completely- This is the most-significant breaking change considered here. However, it is probably possible to translate most
split_row_groups
uses into other options (clearing a manageable path for deprecation)
- This is the most-significant breaking change considered here. However, it is probably possible to translate most
- Remove
aggregate_files
completely- This option is already “pre-deprecated,” so this should be manageable
- I’d deprecate the current meaning of
chunksize
- It should mean “the maximum-desired row-count in a partition” to be more consistent with similarly-named arguments in pandas/pyarrow
- The default for
chunksize
(or maybechunk_size
) should beNone
- The existing
chunksize
option is already “pre-deprecated,” so this may be manageable
- I’d add a new
blocksize
argument- Meaning: “the maximum desired (uncompressed) storage size to map to each partition”
- The default for
blocksize
would be set by the engine (so it could be larger in dask_cudf) chunksize
would take precedence overblocksize
(since row-count is a more-specific criteria)
- I’d add a
granularity
argument- Options:
"row-group"
,"file"
,("row-group", <int>)
, or("file", <int>)
- Note: I’m not crazy about the
("row-group", <int>)
or("file", <int>)
options, but this would allow us to avoid breaking existing code that usessplit_row_groups=<int>
- Note: I’m not crazy about the
- Meaning: What kind of “metadata fragments” should we work with when processing metadata and generating a “partitioning plan”
- The default would be
"file"
when bothblocksize
andchunksize
are set toNone
, and"row-group"
otherwise
- Options:
- I’d add a
file_groups
argument- Meaning: Whether any directory-partitioned columns should be used to prevent two row-groups or files from being mapped to the same output DataFrame partition. For example, the user might specify something like groups=[“year”, “month”] (leaving out “day”) to indicate that they want every row in a given partition to correspond to the same year and month.
- Default would be
None
True
could be taken to mean that an output partition can only map to a single file (i.e. “all files are a distinct group”)
- I’d use
dask.delayed
to eagerly collect “metadata-fragment” records from the dataset in parallelgranularity
would determine if each metadata-fragment record corresponds to a file, row-group, or integer-stride of files/row-groups- These records would be used (on the client) to construct a partitioning plan that satisfies
chunksize
,blocksize
andfile_groups
- For large datasets (where we known that there are many records to collect) on remote storage, I would raise a
UserWarning
to suggest that the user setblocksize=None
or usegranularity="file"
if latency is too high and/or files are sufficiently small (this warning would be silenced if any of the partitioning arguments were explicitly set)- This would allow us to prioritize good default partition sizes, and leave it up to the user to opt into a simple per-file mapping if they must avoid parsing metadata at all costs
Issue Analytics
- State:
- Created 10 months ago
- Comments:5 (5 by maintainers)
Top GitHub Comments
Thanks for writing this up @rjzamora – I look forward to grokking things here
cc @fjetter as I suspect you’ll also find this interesting
That makes sense. It is certainly most efficient for the engine to read many contiguous row-groups at once (as long as those row-groups fit comfortably in memory). I think this is the goal for default behavior regardless of whether we choose to “break up” files or not.
Note that when I say “”break” up files, I just mean that we would parse the size of each row-group in the dataset (which we already need to do to calculate the uncompressed storage size of each file), and use that per-row-group information to make sure that we are not producing output partitions that are larger than the maximum-desired size.
In https://github.com/dask/dask/pull/9637, we make the assumption that the size of all row-groups and files in the dataset can be approximated by the metadata in the first file, and we use that metadata to set a maximum row-group count for each partition (split_row_groups). If split_row_groups ends up being set to 10, for example, then a file with 15 row groups will be mapped into two partitions (one with 10 row-groups, and another with 5). This algorithm can be tweaked to break the 15 row-group file into 8 and 7 row-group files instead. However, that PR does not address the case that the file and row-group sizes vary significantly across the dataset.
Sound good. This is how things work now (and how things should work in the “proposed” API).
This feature improves the performance of hive/directory-partitioned data reading in RAPIDS by orders of magnitude. However, I can understand the interest being more limited for CPU.
I suppose it may not be completely out of scope if we can agree on a user-friendly API to cover the “advanced” partitioning cases that I’m hoping to (somehow/someday) capture. For example, I can imagine a world in which we continue deprecating most “advanced” options from
dd.read_parquet
, and introduce newDataset
/DatasetScan
classes to make custom partitioning easier to attack. This is just my first (rough) attempt at imagining something like this, but I’m sure I would also be happy with an API like…I suppose
ParquetDataset
could also provide ato_dataframe
method with an implicit scan operation, so thatcould also be written as