Make path for reading many parquet files without prescan#3978
Make path for reading many parquet files without prescan#3978martindurant merged 4 commits intodask:masterfrom
Conversation
|
Only failure is dask\tests\test_multiprocessing.py::test_pickle_globals , a load of extra global keys turns up. |
|
Failure is this #3983 |
|
Any thoughts here? I think this is a thing worth doing, and, in addition, gives users the choice to supply a glob even when there is a |
TomAugspurger
left a comment
There was a problem hiding this comment.
Things look OK at a glance, though I didn't review thoroughly. docstrings would help me follow things a bit better I think.
| return out_type(dsk, name, meta, divisions) | ||
|
|
||
|
|
||
| def _pf_validation(pf, columns, index, categories, filters): |
There was a problem hiding this comment.
A docstring about what-all this does would be helpful for the future.
dask/dataframe/io/parquet.py
Outdated
| def _read_pf_simple(fs, path, base, index_names, all_columns, is_series, | ||
| categories, cats, scheme, storage_name_mapping): | ||
| from fastparquet import ParquetFile | ||
| print(path, base) |
|
Thanks, fixed. |
|
Anything more here, @TomAugspurger ? |
TomAugspurger
left a comment
There was a problem hiding this comment.
Gave another quick look. Mostly beyond me, but things look fine overall.
| categories=None, index=None): | ||
| """Read dataset with fastparquet by assuming metadata from first file""" | ||
| from fastparquet import ParquetFile | ||
| from fastparquet.util import analyse_paths, get_file_scheme |
There was a problem hiding this comment.
These are always available on the oldest fast parquet we support?
There was a problem hiding this comment.
Both been around over a year
|
|
||
| # Infer divisions for engines/versions that support it | ||
|
|
||
| ddf2 = dd.read_parquet(os.path.join(fn, '*'), engine=read_engine, |
There was a problem hiding this comment.
For the old one, were we using the metadata file? Should your tests using just the individual files be in addition to the old test, rather than in place of them?
There was a problem hiding this comment.
The only change here, was that the set of data-files shouldn't include the _common_metadata file by mistake.
There was a problem hiding this comment.
So with a directory
_common_metadata
part.0.parquet
part.1.parquet
_common_metadata is ignored when it's captured by the glob? Is that documented (understood that it's not immediately relevant to this PR)?
There was a problem hiding this comment.
It works fine, you get an empty partition for the non-data file, just would complicate the test a little, I feel this is cleaner. The case where there is no _common_metadata either may be more common.
| return out_type(dsk, name, meta, divisions) | ||
| def _paths_to_cats(paths, scheme): | ||
| """Extract out fields and labels from directory names""" | ||
| # can be factored out in fastparquet |
There was a problem hiding this comment.
What does this comment mean? Is there similar code in fastparquet?
There was a problem hiding this comment.
Yes, but it's tied into a class method, and I don't have the appetite to change it and put in a set of deprivations.
|
|
||
| # Infer divisions for engines/versions that support it | ||
|
|
||
| ddf2 = dd.read_parquet(os.path.join(fn, '*'), engine=read_engine, |
There was a problem hiding this comment.
So with a directory
_common_metadata
part.0.parquet
part.1.parquet
_common_metadata is ignored when it's captured by the glob? Is that documented (understood that it's not immediately relevant to this PR)?
For fastparquet when there is no
_metadata.Fixes #3974
Benchmarks for a small dataset of 100 files, 6col*10row each, all int
Master
this branch
Note that reading the metadata is now longer, because is has to parse all of the row-groups up front (even though they are loaded from one place), whereas this is deferred to the lazy/parallel task when supplying a glob/list of paths.
This code is rather long and convoluted.