Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Datasets] Add bulk Parquet file reader API #23179

Merged
merged 2 commits into from
Apr 29, 2022

Conversation

pdames
Copy link
Member

@pdames pdames commented Mar 15, 2022

Why are these changes needed?

Adds a reader suitable for quickly reading a large number (e.g. 1-100K+) of Parquet files into a Ray Dataset from either local disk or cloud storage.

Latest Benchmarks:

  • Environment 1:
    • Cluster: 5 x r5n.8xlarge (160 vCPUs)
    • Storage: S3
    • 2000 Tiny (~4KB) Parquet Files:
      • Test Case 1 (read_parquet_fast() - default fast metadata provider - parallel 200)
        • Read Dataset Latency: 1.646850564982742
        • Write Dataset Latency: 5.7583091059932485
        • Read Dataset Latency: 2.4654725349973887
        • Write Dataset Latency: 6.635675027035177
        • Read Dataset Latency: 2.460472490056418
        • Write Dataset Latency: 6.1093138850992545
      • Test Case 2 (read_parquet() - default parquet metadata provider - parallel 200)
        • Read Dataset Latency: 15.828366486006416
        • Write Dataset Latency: 12.378745757043362
        • Read Dataset Latency: 15.968373686890118
        • Write Dataset Latency: 12.28318011702504
        • Read Dataset Latency: 16.455996707081795
        • Write Dataset Latency: 12.62632444105111
    • 10,000 Tiny (~4KB) Parquet Files:
      • Test Case 1 (read_parquet_fast() - default fast metadata provider - parallel 160)
        • Read Dataset Latency: 5.766188704059459
        • Write Dataset Latency: 6.506135637057014
      • Test Case 2 (read_parquet_fast() - default fast metadata provider - parallel 320)
        • Read Dataset Latency: 3.841439099982381
        • Write Dataset Latency: 6.254379783989862
      • Test Case 3 (read_parquet_fast() - default fast metadata provider - parallel 640)
        • Read Dataset Latency: 2.392879810067825
        • Write Dataset Latency: 6.607215114985593
      • Test Case 4 (read_parquet_fast() - default fast metadata provider - parallel 1280)
        • Read Dataset Latency: 2.1967075469437987
        • Write Dataset Latency: 9.095152657013386
      • Test Case 5 (read_parquet_fast() - default fast metadata provider - parallel 1600)
        • Read Dataset Latency: 2.277380953077227
        • Write Dataset Latency: 9.146176831098273
      • Test Case 6 (read_parquet_fast() - default fast metadata provider - parallel 2560)
        • Read Dataset Latency: 2.137215052964166
        • Write Dataset Latency: 14.531858821981587
    • 50,000 Tiny (~4KB) Parquet Files:
      • Test Case 1 (read_parquet_fast() - default fast metadata provider - parallel 320)
        • Read Dataset Latency: 11.416414194973186
        • Write Dataset Latency: 12.532715738983825
        • Read Dataset Latency: 11.54179179109633
        • Write Dataset Latency: 12.113915671012364
      • Test Case 2 (read_parquet_fast() - default fast metadata provider - parallel 640)
        • Read Dataset Latency: 7.514457287965342
        • Write Dataset Latency: 12.104804654954933
        • Read Dataset Latency: 8.027748159016483
        • Write Dataset Latency: 10.249830988934264
      • Test Case 3 (read_parquet_fast() - default fast metadata provider - parallel 1280)
        • Read Dataset Latency: 5.188601649017073
        • Write Dataset Latency: 15.750625844928436
        • Read Dataset Latency: 5.264186333981343
        • Write Dataset Latency: 10.911084560095333
    • 600,000 Tiny (~4KB) Parquet Files:
      • Test Case 1 (read_parquet_fast() - default fast metadata provider - parallel 1280)
        • Read Dataset Latency: 42.07806533499979
        • Write Dataset Latency: 202.45852285499996
  • Environment 2:
    • Cluster: 5 x c5n.9xlarge (180 vCPUs)
    • Storage: S3
    • 50,000 Tiny (~4KB) Parquet Files:
      • Test Case 1 (read_parquet_fast() - default fast metadata provider - parallel 180)
        • Read Dataset Latency: 20.308360855000046
        • Write Dataset Latency: 18.48248909499989
      • Test Case 2 (read_parquet_fast() - default fast metadata provider - parallel 360)
        • Read Dataset Latency: 8.663979919000212
        • Write Dataset Latency: 18.941202082000018
      • Test Case 3 (read_parquet_fast() - default fast metadata provider - parallel 720)
        • Read Dataset Latency: 6.119230162000349
        • Write Dataset Latency: 18.112871180000184
      • Test Case 4 (read_parquet_fast() - default fast metadata provider - parallel 1440)
        • Read Dataset Latency: 4.157292100000177
        • Write Dataset Latency: 18.078249656000025
      • Test Case 5 (read_parquet_fast() - default fast metadata provider - parallel 2880)
        • Read Dataset Latency: 3.098416549999911
        • Write Dataset Latency: 20.54457710700035
    • 600,000 Tiny (~4KB) Parquet Files:
      • Test Case 1 (read_parquet_fast() - default fast metadata provider - parallel 1440)
        • Read Dataset Latency: 32.844527049000135
        • Write Dataset Latency: 185.16646222
      • Test Case 2 (read_parquet_fast() - default fast metadata provider - parallel 2880)
        • Read Dataset Latency: 18.028459068999837
        • Write Dataset Latency: 181.07565845699992

Old Benchmarks:

  • Cluster: 5 x r5n.8xlarge
  • Storage: S3
  • 2000 Tiny (~2KB) Hive-Partitioned Parquet Files:
    • read_parquet() not benchmarked due to ParquetDataset merge failure on indexed partition columns: https://issues.apache.org/jira/browse/ARROW-13851
    • Test Case 1 (read_parquet_fast() - cached parquet metadata provider - expand paths skipped - no partition filter)
      • Read Dataset Latency: 3.72136511397548
      • Write Dataset Latency: 7.245090199983679
      • Read Dataset Latency: 2.96225255203899
      • Write Dataset Latency: 6.7958771920530125
      • Read Dataset Latency: 2.9959744710940868
      • Write Dataset Latency: 15.850318810902536
      • Read Dataset Latency: 3.111023452016525
      • Write Dataset Latency: 10.99442114494741
    • Test Case 2 (read_parquet_fast() - cached parquet metadata provider - expand paths skipped - partition filter keeping 1/7 files - partition_filter_fn=lambda d: d["day"] == "MO")
      • Read Dataset Latency: 1.5806240870151669
      • Write Dataset Latency: 8.724663103889024
      • Read Dataset Latency: 2.334774918970652
      • Write Dataset Latency: 8.310331499087624
    • Test Case 3 (read_parquet_fast() - cached parquet metadata provider - expand paths skipped - partition filter keeping all files - partition_filter_fn=lambda d: d["day"] != "NA")
      • Read Dataset Latency: 3.1833109749713913
      • Write Dataset Latency: 8.957277923938818
      • Read Dataset Latency: 1.950843447004445
      • Write Dataset Latency: 8.899622384924442
      • Read Dataset Latency: 2.891347081051208
      • Write Dataset Latency: 14.80685784202069
  • 2000 Tiny (~4KB) Parquet Files:
    • Test Case 1 (read_parquet_fast() - cached parquet metadata provider - expand paths skipped)
      • Read Dataset Latency: 2.8592376470332965
      • Write Dataset Latency: 8.016067871009
      • Read Dataset Latency: 2.98329175892286
      • Write Dataset Latency: 7.4663043189793825
    • Test Case 2 (read_parquet() - cached parquet metadata provider - expand paths skipped)
      • Read Dataset Latency: 1.5026919830124825
      • Write Dataset Latency: 17.154331294004805
      • Read Dataset Latency: 1.437328452942893
      • Write Dataset Latency: 21.36053073592484
      • Read Dataset Latency: 1.3846774999983609
      • Write Dataset Latency: 13.24175373907201
    • Test Case 3 (read_parquet())
      • Read Dataset Latency: 18.912850142922252
      • Write Dataset Latency: 12.525802098913118
      • Read Dataset Latency: 17.814308558939956
      • Write Dataset Latency: 13.623326486092992
      • Read Dataset Latency: 15.942234317073599
      • Write Dataset Latency: 13.625373031012714
    • Test Case 4 (read_parquet_fast())
      • Read Dataset Latency: 21.10480642109178
      • Write Dataset Latency: 6.64842400001362
      • Read Dataset Latency: 30.31197776598856
      • Write Dataset Latency: 7.285999955958687

Related issue number

Closes #22910

Checks

  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@ericl
Copy link
Contributor

ericl commented Mar 15, 2022

Very exciting! @clarkzinzow @naijoaix want to help shepherd this?

partition filter is defined, then each unpartitioned file will be passed
in as an empty dictionary.

For Directory Partitioning, all directories under the base directory will
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, is this a common partitioning style? If we only support the first (like Spark does) it may simplify the code.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is relatively common, but not as ubiquitous as hive-style partitioning. CloudTrail and Kinesis Firehose are two notable AWS service examples that use directory partitioning, so this is useful for parsing/filtering partitioned datasets that they produce.

self,
paths: List[str],
filesystem: "pyarrow.fs.FileSystem",
) -> List[str]:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice.

Copy link
Contributor

@ericl ericl left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@pdames this looks pretty good at a high level. Any major items missing beyond tests?

@ericl ericl added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Mar 16, 2022
@pdames pdames changed the title [Datasets][WIP] Draft of fast Parquet file reader and partition filter [Datasets] Add fast Parquet file reader and partition filter Mar 17, 2022
@pdames
Copy link
Member Author

pdames commented Mar 17, 2022

@pdames this looks pretty good at a high level. Any major items missing beyond tests?

I just updated the code to a version that passed a small integration test suite parsing and filtering hive-style partitioned datasets exported from Redshift. I think it's feature-complete at this point, so I'll start working on some associated unit tests.

@pdames
Copy link
Member Author

pdames commented Mar 18, 2022

Based on the 2K tiny file benchmarks I just ran, the good news is that (1) the partition filter adds little-to-no overhead (and appropriately reduces overall read latency when filtering out files) and (2) the read_parquet_fast() implementation reduces overall read/write round-trip latency by approximately 50% vs read_parquet() (where write is bound by waiting for read tasks to complete) as long as path expansion is skipped.

The bad news is that, without a custom metadata provider to skip path expansion (when only file paths are given), read_parquet_fast() is almost equivalently slow in terms of round-trip read/write latency, and slower than read_parquet() for metadata resolution on S3 due to the overhead of default path expansion. A couple immediate ways to improve the default latency could be to (1) parallelize path expansion via ray.remote and (2) apply partition filters eagerly during path expansion (right now they're applied after path expansion).

@ericl
Copy link
Contributor

ericl commented Mar 18, 2022

Hmm, so the issue is that if you provide a list of files, we're needing to double check that each file is not a directory? I guess maybe indeed we should make path expansion a flag that we can turn off, maybe even logging a warning if the user is passing a list and has expansion on.

@pdames
Copy link
Member Author

pdames commented Mar 19, 2022

So, instead of providing a boolean flag to skip path expansion, I created a default FastFileMetadataProvider for read_parquet_fast() that skips path expansion, and logged associated warnings under both the DefaultFileMetadataProvider about high-latency metadata resolution, and no metadata resolution or path expansion under the FastFileMetadataProvider.

The end results are looking pretty good here, with ~80% performance improvement for 2K files vs read_parquet(), and with read_parquet_fast() completing in ~18 seconds for 600K tiny S3 files (and a little over 3 minutes for a round trip read/write @ parallel 2880 on a 5 x c5n.9xlarge cluster).

Copy link
Contributor

@jianoaix jianoaix left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you Patric for making this improvement!

block = existing_block_udf(block)
return block
By default, ONLY file paths should be provided as input (i.e. no directory paths).
If your use-case requires directory paths, then the metadata provider should be
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you document what will happen if users do feed this with directories, crash or erroring? (I think we should avoid crash in general)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

An OSError (identical to the OSError for a non-existent file path) is raised currently - I've added this to the documentation.

input files due to collecting all file metadata on a single node.

Also supports a wider variety of input Parquet file types than `read_parquet` due
to not trying to merge and resolve a unified schema for all files.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will bubble up to and break the schema semantic of Dataset API right? That is, Dataset.schema() undefined.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this uses the base prepare_read() method from FileBasedDatasource, so the schema remains undefined by default (just like with CSV/NumPy/JSON/Binary datasources). If known, the schema can be added by either specifying it as a keyword argument and/or via a custom metadata provider.

partition path of the form "{value1}/{value2}/..." or an empty dictionary for
unpartitioned files.

Requires a corresponding ordered list of partition key field names to map the
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: extra space

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed!



@DeveloperAPI
class PathPartitioning:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would it be more accurate to call this a filter (e.g. PathPartitionFilter)?
From my reading of code, the role of this class is to filter paths based on provided filter function, rather than partitioning a bunch of paths.

Copy link
Member Author

@pdames pdames Mar 22, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My thinking here is that this class is really a path-based partition parser that currently only exposes a filter to end-users. Ideally, it should be extended with methods to cover additional tasks in subsequent PRs like (1) group(paths: List[str]) -> Dict[List[Tuple[str, str]], List[str]] to group paths by partition (e.g. to facilitate a partitioned dataset read), (2) partition_dir(base_dir: str, partition_values: List[str]) -> str to find the correct directory for a list of partition values (e.g. for a partitioned dataset block write), (3) convert(paths: List[str], from: PartitionStyle, to: PartitionStyle) to convert a list of paths between partition styles, etc.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree this is extendable. Shall we name it so, e.g. PathPartitionParser, or PathPartitionProcessor?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PathPartitionParser sounds good to me.

return paths, np.empty(len(paths), dtype=object)


class ParquetFastDatasource(FileBasedDatasource):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if this is too generic, e.g. maybe there are other ways to optimize file reading in the future. Shall we make it more specific (e.g. s/Fast/Directoryless/g, or s/Fast/DirectFile/g)?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had some similar thoughts. However, the class itself supports directories, so I'm not sure about either Directoryless or DirectFile. It's just the FastFileMetadataProvider default in the read_parquet_fast() API that alters it to not handle directories.

ParquetNoMetaDatasource was one thing I was thinking of, since not overriding prepare_read() means it's not populated with verbose metadata, but I can also imagine this changing in the future.

I think it may be best to just call this something like ParquetBasicDatasource to indicate that it is a minimal implementation that reuses as much code as possible from FileBasedDatasource.

As far as read_parquet_fast() API naming goes, I've been thinking that read_parquet_bulk() would be more accurate, since the current implementation offers little to no performance benefits when reading a small number of files (esp. if each file is relatively large), but I expect it to continue to offer long-term benefits for bulk reads of a large number of files.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My last commit changes read_parquet_fast() to read_parquet_bulk(), ParquetFastDatasource to ParquetBaseDatasource, and then has ParquetDatasource inherit from ParquetBaseDatasource since they share the same file format and write path.

What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good.

@@ -217,6 +220,11 @@ def expand_paths(
) -> Tuple[List[str], List[Optional[int]]]:
from pyarrow.fs import FileType

logger.warning(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure we should log this unconditionally: the input paths may contain directory so it's not appropriate to suggest FastFileMetadataProvider. How about commenting the class that if users are dealing with only files, they should use FastFileMetadataProvider?

@@ -217,6 +220,11 @@ def expand_paths(
) -> Tuple[List[str], List[Optional[int]]]:
from pyarrow.fs import FileType

logger.warning(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
logger.warning(
if len(paths) > 1:
logger.warning(

This should hide the warning for the common case, where a single directory is passed as the path.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Another option that I was considering here is to only log this warning after a fixed amount of time has elapsed expanding directory paths (e.g. if we've been expanding directories for more than 15 seconds, then log this warning).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, it seems fine to keep this simple since this is a precaution.

@@ -217,6 +220,11 @@ def expand_paths(
) -> Tuple[List[str], List[Optional[int]]]:
from pyarrow.fs import FileType

logger.warning(
f"Expanding {len(paths)} path(s). This may be a HIGH LATENCY operation "
f"on some cloud storage services. If faster reads are required, try "
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
f"on some cloud storage services. If faster reads are required, try "
f"on some cloud storage services. If the specified paths all point to files and never directories, use "

Copy link
Contributor

@jianoaix jianoaix left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of adding a big PR and then a follow-up big unit test PR, would it be possible to shard this PR horizontally and add in corresponding unit tests for each smaller PRs?
E.g. 1) partitioning.py etc.; 2) parque_base_datasource.py etc; 3) integration into read_api.py.

python/ray/data/datasource/file_based_datasource.py Outdated Show resolved Hide resolved


@DeveloperAPI
class PathPartitioning:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree this is extendable. Shall we name it so, e.g. PathPartitionParser, or PathPartitionProcessor?

@pdames
Copy link
Member Author

pdames commented Mar 28, 2022

Instead of adding a big PR and then a follow-up big unit test PR, would it be possible to shard this PR horizontally and add in corresponding unit tests for each smaller PRs? E.g. 1) partitioning.py etc.; 2) parque_base_datasource.py etc; 3) integration into read_api.py.

Sounds good to me. Now that we understand the total scope of changes from this PR, I can start opening up smaller PRs and linking them back to this PR and the parent issue that we're working on. I think the proposed ordering of PRs given above also makes sense.

Copy link
Contributor

@jianoaix jianoaix left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for addressing comments, LGTM once it's split into smaller PRs with unit tests.

@pdames
Copy link
Member Author

pdames commented Apr 1, 2022

Rebased on top of #23624 to ensure end-to-end compatibility of that PR with upcoming PRs. No need to review here.

@pdames pdames force-pushed the parquet-fast branch 2 times, most recently from c1d0229 to 0bc35f1 Compare April 27, 2022 21:03
@zhe-thoughts zhe-thoughts added the P1 Issue that should be fixed within a few weeks label Apr 28, 2022
@zhe-thoughts zhe-thoughts added this to the Datasets GA milestone Apr 28, 2022
Copy link
Contributor

@jianoaix jianoaix left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The delta part looks good. Will take another look after second PR merged.


# Expect directory path expansion to fail.
with pytest.raises(OSError):
ray.data.read_parquet_bulk(data_path, filesystem=fs)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a test for reading directory with right metadata provider, so it should have no issue?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added test_parquet_read_bulk_meta_provider for this in my latest commit.

@pdames pdames changed the title [Datasets] Add fast Parquet file reader and partition filter [Datasets] Add bulk Parquet file reader API Apr 29, 2022
clarkzinzow pushed a commit that referenced this pull request Apr 29, 2022
…ource (#24094)

Adds a fast file metadata provider that trades comprehensive file metadata collection for speed of metadata collection, and which also disabled directory path expansion which can be very slow on some cloud storage service providers. This PR also refactors the Parquet datasource to be able to take advantage of both these changes and the content-type agnostic partitioning support from #23624.

This is the second PR of a series originally proposed in #23179.
@pdames
Copy link
Member Author

pdames commented Apr 29, 2022

@jianoaix @clarkzinzow Rebased on top of the latest from master and ready for final review.

Copy link
Contributor

@clarkzinzow clarkzinzow left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM!

python/ray/data/datasource/file_based_datasource.py Outdated Show resolved Hide resolved
python/ray/data/datasource/file_based_datasource.py Outdated Show resolved Hide resolved
paths: Union[str, List[str]],
*,
filesystem: Optional["pyarrow.fs.FileSystem"] = None,
columns: Optional[List[str]] = None,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder do we perform the column projection at the s3 (maybe with file offset), or have to fetch to local first?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jianoaix I believe that column projection will happen via Arrow's pq.read_table() API, where the column selection is passed through to that call when using this bulk Parquet API.

@clarkzinzow
Copy link
Contributor

Datasets tests look good, merging! Great work @pdames! 🙌

@clarkzinzow clarkzinzow merged commit f337f04 into ray-project:master Apr 29, 2022
clarkzinzow pushed a commit that referenced this pull request May 12, 2022
…ata providers. (#24354)

API doc updates for #23179 and #24094. All data docs related to #23179 should be up-to-date once this PR and #24203 are merged.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. P1 Issue that should be fixed within a few weeks
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Feature] Ray dataset loading large list of parquet files is extremely slow
5 participants