-
Notifications
You must be signed in to change notification settings - Fork 403
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
Very slow read from a super massive table in Azure Data Lake Gen2 #1569
Comments
We have some ideas to improve read performance using our internal file system implementaions (which are in use here). In case you have some predicates that can be applied to you data, you may be able to avoid reading some of the data into memory, by using |
We have also experienced really slow reads with ADLS gen 2 and a 70 mb file taking 12 minutes. |
Hi there! We found a faster approach for this one if your goal is to only read the current data of the delta table.
dt = DeltaTable("adl://container1/table1/", storage_options=storage_options)
files = dt.file_uris()
import pyarrow.parquet as pq
import pyarrow as pa
pq_data = pq.ParquetDataset(path_or_paths= files, filesystem= fs)
table= pq_data.read() Special thanks to @Yacobolo for this one! |
@mjducutoae If you want to keep the delta schema, you will need to pass it also: pq_data = pq.ParquetDataset(path_or_paths=files, schema=dt.schema().to_pyarrow(), filesystem=fs) |
I think this is likely due to some GIL-locking in the filesystem implementation. If you pass a PyArrow filesystem down instead, does it perform as expected? dt = DeltaTable("adl://container1/table1/")
dt.to_pyarrow_dataset(filesystem=fs) |
I don't see any difference, same execution time as with or without fs. fs_abfs = fsspec.filesystem("abfs", **storage_options)
handler = pafs.FSSpecHandler(fs_abfs)
fs = pafs.PyFileSystem(handler)
sub_fs = pafs.SubTreeFileSystem(os.path.join(root_dir, file), fs)
df = dt.to_pyarrow_dataset(filesystem=sub_fs) |
@ion-elgreco Okay. Did you get faster performance using PyArrow datasets directly on the parquet files in Azure? #1569 (comment) I suspect the fsspec implementation also holds the GIL, given it's implemented in Python. So comparing against that might not be relevant. But if you get better performance by reading the parquet files with PyArrow directly, then that would point to something else. |
Yes, reading the files directly with pyarrow dataset is the fastest. |
# Description I'm not sure yet this is the root cause of the performance issues we are seeing in #1569, but this seems like an obvious change that might help. Any method that takes `&mut self` implicitly holds the GIL, so we'll need to allow threads explicitly in those methods. # Related Issue(s) <!--- For example: - closes #106 ---> # Documentation <!--- Share links to useful documentation --->
@wjones127 You can disregard my previous results, I had a network issue which slowed down the reads. I retried it today and passing a pyarrow filesystem which made it just as fast as using pyarrow.dataset.dataset directly. def read_delta_fast_lazy(
file: str,
storage_options: dict | None = None,
version: int | None = None,
) -> pl.LazyFrame:
if storage_options is None:
fs = fsspec.filesystem("file")
else:
fs = fsspec.filesystem("abfs", **storage_options)
dt = DeltaTable(file, version=version, storage_options=storage_options)
files = dt.file_uris()
pq_data = pa.dataset.dataset(
source=files,
schema=dt.schema().to_pyarrow(),
format='parquet',
filesystem=fs,
)
return pl.scan_pyarrow_dataset(pq_data) This is also just as fast: dt = DeltaTable(os.path.join(root_dir, file), storage_options=storage_options)
fs_abfs = fsspec.filesystem("abfs", **storage_options)
handler = pafs.FSSpecHandler(fs_abfs)
fs = pafs.PyFileSystem(handler)
sub_fs = pafs.SubTreeFileSystem(os.path.join(root_dir, file), fs)
df = dt.to_pyarrow_dataset(filesystem=sub_fs) |
Thanks for confirming @ion-elgreco . I’ll look into this more soon |
@wjones127 I've some done some performance checks with v0.10.1 and v0.10.2, and with my own function using pyarrow directly. Seems that it's working as expected now : ) def read_delta_fast_lazy(
file: str,
storage_options: dict | None = None,
version: int | None = None,
) -> pl.LazyFrame:
"""Fast path to read delta tables lazily, until upstream fixed: https://github.com/delta-io/delta-rs/issues/1569
Args:
file (str): file url (abfs://...)
storage_options (dict): storage_options for fsspec, must contain, client_id, client_secret, tenant_id
version (int | None, optional): Version number of table. Defaults to None.
Returns:
pl.LazyFrame: dataframe
"""
if storage_options is None:
fs = fsspec.filesystem("file")
else:
storage_options['account_name'] = file.split("@")[1].split('.')[0]
fs = fsspec.filesystem("abfs", **storage_options)
dt = DeltaTable(file, version=version, storage_options=storage_options)
files = dt.file_uris()
pq_data = pa.dataset.dataset(
source=files,
schema=dt.schema().to_pyarrow(),
partitioning="hive",
format="parquet",
filesystem=fs,
)
return pl.scan_pyarrow_dataset(pq_data) Results (pyarrow dataset): %%timeit
df = read_delta_fast(os.path.join(root_dir, "table"), storage_options=storage_options)
31.6 s ± 1.72 s per loop (mean ± std. dev. of 7 runs, 1 loop each) Results (v0.10.1): %%timeit
df2 = pl.read_delta(os.path.join(root_dir, "data"), storage_options=storage_options)
49.3 s ± 1.5 s per loop (mean ± std. dev. of 7 runs, 1 loop each) Results (v0.10.2): %%timeit
df2 = pl.read_delta(os.path.join(root_dir, "data"), storage_options=storage_options)
32 s ± 1.84 s per loop (mean ± std. dev. of 7 runs, 1 loop each) |
@rtyler @wjones127 apache/arrow#36765 enabling pre-buffer in the pyarrow parquet scan options boost the performance quite a lot. |
@wjones127 @rtyler shall we close this issue now that pre-buffer is enabled in pyarrow? |
First of all thanks for creating this awesome library and eliminating the need to use spark for everything related to deltalake.
I have been trying to read a table from Azure Data Lake gen 2 and it is super slow.
The table hast lots of versions but when instantiating DeltaTable I specify the latest one
This above code is almost fast 6s, but when I run
to_pyarrow_table
andto_pyarrow_dataset
to use the data later in DuckDB it is taking super long. Is it possible to increase the number of threads it uses for reading data?Any other possible solution?
Thanks in advance!
The text was updated successfully, but these errors were encountered: