-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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
[Python][Parquet] Parquet deserialization speeds slower on Linux #38389
Comments
Here is a rendered notebook from my Ubuntu Thinkpad to make the numbers more concrete: https://gist.github.com/mrocklin/526120bb5231cc5d9d4e3ca87fc09c68 This run was actually a bit faster than I usually see. On Cloud VMs (even very nice ones) it's hard to get above 150MB/s on real data. |
For reference, I ran your snippet above and repeated the timing part multiple times on a Linux (Ubuntu 20.04) Dell XPS 13 9380 (more than 4 years old, 8th gen Intel Core i7, 4 cores / 8 threads), and I get almost 2 GB/s for disk speed and around 1 GB/s for reading (just under for from file, just above for in-memory). (so at least it's not a simple mac vs linux issue) One environment characteristic that will significantly influence those numbers is the parallelization (the Parquet reading will by default use all the available cores). So it might be worth to run those timings with and without threads enabled, to check if also single-threaded performance is bad and ensure it's not related to bad scaling on that front. |
I ran a couple of pyspy benchmarks on pure This profile shows the case where I'm reading a file and are selecting about half it's columns (a mix between different dtypes) Note how the read_table request is split into three parts
So, that's 30% where we're doing nothing/not a lot? I'm not sure at which point the pre_buffering can kick on or how this works. This stuff does not show up in my profile since it's the arrow native threadpool. At least this initial HEAD request appears to be bad, particularly if we're fetching just a couple of columns from otherwise already small-ish files. The file I was looking at is one of the TPCH lineitem files which in our dataset version is 22.4MiB large. Edit: This all ran on a Coiled VM and I was basically running the read request in a for loop. No multi threading on my side, just the pyarrow native stuff. |
About (1) some optimization will be included later, see:
Seems we can enable larger prefetch-depth to arrow fetching multiple files concurrently. |
Sorry, I just realize that my comment is also slightly off topic. The OP discusses pure deserialization without S3 in between |
FWIW I slightly modified the above script to run each operation N times since I noticed quite some variance on my machine (M1 2020 MacBook) # Create dataset
import pyarrow as pa
import pyarrow.parquet as pq
import numpy as np
import pandas as pd
import time
import io
x = np.random.randint(0, 100000, size=(1000000, 100))
df = pd.DataFrame(x)
t = pa.Table.from_pandas(df)
niterations = 20
# Write to local parquet file
pq.write_table(t, "foo.parquet")
# Time Disk speeds
start = time.perf_counter()
for _ in range(niterations):
with open("foo.parquet", mode="rb") as f:
bytes = f.read()
nbytes = len(bytes)
stop = time.perf_counter()
print("Disk Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time Arrow Parquet Speeds
start = time.perf_counter()
for _ in range(niterations):
_ = pq.read_table("foo.parquet")
stop = time.perf_counter()
print("PyArrow Read Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time In-Memory Read Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(io.BytesIO(bytes))
stop = time.perf_counter()
print("PyArrow In-Memory Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time In-Memory Read Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(io.BytesIO(bytes)).to_pandas()
stop = time.perf_counter()
print("PyArrow (to_pandas) Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s") and I get M1 (using
while on the cloud using a
|
@jorisvandenbossche have you used the same conda forge build for your measurements or did you build it yourself? It would be nice to rule out any build differences |
Ok, fun experiment. I wrapped the above script in a function Looks like the simple fact that we're running this in the dask environment is slowing us down quite a bit. This also biases most/all Coiled-based cloud benchmarks Code# Create dataset
import pyarrow as pa
import pyarrow.parquet as pq
import numpy as np
import pandas as pd
import time
import io
def run_benchmark():
from distributed.worker import print
x = np.random.randint(0, 100000, size=(1000000, 100))
df = pd.DataFrame(x)
t = pa.Table.from_pandas(df)
niterations = 20
# Write to local parquet file
pq.write_table(t, "foo.parquet")
# Time Disk speeds
start = time.perf_counter()
for _ in range(niterations):
with open("foo.parquet", mode="rb") as f:
bytes = f.read()
nbytes = len(bytes)
stop = time.perf_counter()
print("Disk Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time Arrow Parquet Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table("foo.parquet")
stop = time.perf_counter()
print("PyArrow Read Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time In-Memory Read Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(io.BytesIO(bytes))
stop = time.perf_counter()
print("PyArrow In-Memory Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time In-Memory Read Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(io.BytesIO(bytes)).to_pandas()
stop = time.perf_counter()
print("PyArrow (to_pandas) Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
run_benchmark()
from distributed import Client
client = Client()
client.submit(run_benchmark).result() |
Is pyarrow using either one of Edit: Looking at the code base, I see references and documentation that suggests that Dask is setting those to one and when I remove that, I'm back to the non-dask speed. |
In #5098 we set a malloc trim threshold by default to more aggressively control memory trimming. also related #7177 At the same time, we included these default settings but didn't have incredibly solid arguments for it. It's been a long standing best practice when using dask to disable this nested parallelism. We haven't received a lot of user feedback about this. However, we had some internal reports of users who were struggling with this because this was quite unexpected behavior for them and non-trivial to debug for the ordinary end user. In apache/arrow#38389 (comment) this also suggests to negatively impact read performance of parquet tables. We should consider removing this again
To be clear, my experiment in this issue was run without Dask and without Coiled. I wanted to isolate things. |
Turning threads off I get around 170MB/s on my linux machine, 600 MB/s on my OS-X machine.
I'm curious, for deserializing integers is this expected performance? I would have thought that for this kind of data (just ints) we'd be closer to GB/s speeds. I'm curious, what is the slow part here? Is there some intense compression or something?
I'm also quite curious about this. |
I was using a conda-forge install for the above numbers, no custom build.
But for linux you mentioned above a similar number with threads. So that means you see hardly any perf improvement with threads on linux? |
Yes, it seems we are using |
Yes. That's correct. To be clear though, I'm currently more confused about only getting 150-200 MB/s deserializing integers on a single thread. That seems very strange to me. |
Yes, thanks. I already found that pyarrow is setting the CPU threadpool to one inside of dask regardless of the env settings. I already tested a little with |
To find a standardized shared compute environment I tried this on Google Colab. |
I'm not sure if this comparison is actually fair and valid. Parquet -> Arrow has to do a nontrivial amount of work. Even your random data is encoded and compressed. (See I also ran this on colab and got something like this
from your benchmark output. I went along and ran import pickle
pickled_df = pickle.dumps(x)
compressedb = pa.compress(pickled_df, "SNAPPY")
nbytes = len(compressedb)
start = time.time()
pa.decompress(compressedb, decompressed_size=len(pickled_df), codec="SNAPPY")
stop = time.time()
print("SNAPPY Decompress Bandwidth:", int(nbytes / (stop - start) / 2**20), "MiB/s") which gives me
so we're moving in the same vicinity as the parquet read. |
Cool. What I'm reading from you are a couple of insights:
I'll add my understanding, which is that I also expect SNAPPY to operate at GB/s speeds, but for some reason it's not here. On Colab I also get 200MB/s as you do, but on my MacBook I get 767 MB/s This probably allows us to dive a bit deeper into the problem. |
Yes, I understand (and dask uses
Parquet is indeed a complex file format. In addition to the decompression, there is also the decoding (although the file here will use dictionary encoding, and that should be quite fast I would expect. Also quickly testing plain and delta_binary_packed encodings, and that actually gives slower reads than the default in this case). I was also wondering if we could have an idea which bandwidth one can expect for just the decompression, to have some point of comparison. The snappy readme (https://github.com/google/snappy) itself mentions decompression at 500MB/s for Intel Core i7. Running the snippet of Florian above, I actually only get around 100MB/s for the SNAPPY decompression..
Quickly testing with another compression ( Using no compression at all ( |
Re-running the benchmarks with a slightly adapted script from above (single threaded, different compressions), and ensuring I run it while having no other applications running, I actually get quite decent single threaded performance: Codeimport pyarrow as pa
import pyarrow.parquet as pq
import numpy as np
import pandas as pd
import time
import io
# Create datasets
x = np.random.randint(0, 100000, size=(1000000, 100))
df = pd.DataFrame(x)
t = pa.Table.from_pandas(df)
pq.write_table(t, "foo.parquet")
pq.write_table(t, "foo-lz4.parquet", compression="lz4")
pq.write_table(t, "foo-uncompressed.parquet", compression="none")
def run_benchmark(fname):
niterations = 20
# Time Disk speeds
start = time.perf_counter()
for _ in range(niterations):
with open(fname, mode="rb") as f:
bytes = f.read()
nbytes = len(bytes)
stop = time.perf_counter()
print("Disk Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time Arrow Parquet Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(fname, use_threads=False)
stop = time.perf_counter()
print("PyArrow Read Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time In-Memory Read Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(io.BytesIO(bytes), use_threads=False)
stop = time.perf_counter()
print("PyArrow In-Memory Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
# Time In-Memory Read Speeds
start = time.perf_counter()
for _ in range(niterations):
pq.read_table(io.BytesIO(bytes), use_threads=False).to_pandas(use_threads=False)
stop = time.perf_counter()
print("PyArrow (to_pandas) Bandwidth:", int(nbytes / ((stop - start) / niterations) / 2**20), "MiB/s")
And the file sizes are 258, 255 and 293 MB, respectively (so the actual speedup for uncompressed is a bit lower than what the above gives, because it's reading more MBs. But it's still faster in terms of seconds to read) |
This seems like a rather low number for uncompressed integer data. What is the exact encoding used by the integer column? The metadata display above is a bit ambiguous as it lists three encodings :-) |
Ok, so after investigating this, one factor is that However, the benchmark numbers are also largely skewed by the fact that the MB/s figure is computed relatively to the compressed and encoded size, not on the final in-memory size. |
With that in mind, here are updated results against the actual in-memory size:
|
And now the file sizes:
... meaning that DELTA_BINARY_PACKED is probably a very good choice for integer columns. |
Okay, I took a quick look. One thing that takes some time is dictionary decoding for the dictionary-encoded columns. Once that is done, I've got the following breakdown, roughly:
I did these measurements on one CPU core. The overall read speed is around 220 MB/s from the local filesystem. |
Something weird is that most columns out of this file have a single chunk, even though the file has 21 row groups. This doesn't look right: >>> [(name, a.num_chunks) for name, a in zip(tab.column_names, tab.columns)]
[('l_orderkey', 1),
('l_partkey', 1),
('l_suppkey', 1),
('l_linenumber', 1),
('l_quantity', 1),
('l_extendedprice', 1),
('l_discount', 1),
('l_tax', 1),
('l_returnflag', 21),
('l_linestatus', 21),
('l_shipdate', 1),
('l_commitdate', 1),
('l_receiptdate', 1),
('l_shipinstruct', 21),
('l_shipmode', 21),
('l_comment', 1)]
>>> pf = pq.ParquetFile('~/arrow/data/lineitem/lineitem_0002072d-7283-43ae-b645-b26640318053.parquet')
>>> pf.metadata
<pyarrow._parquet.FileMetaData object at 0x7f236076dcb0>
created_by: DuckDB
num_columns: 16
num_rows: 2568534
num_row_groups: 21
format_version: 1.0
serialized_size: 29792 |
I mean that something is weird in the way the Parquet reader behaves, btw. The files don't seem to be at fault (except perhaps for using Snappy :-)). |
That's because of the use of This # this file has 21 row groups
>>> file_path = "lineitem_0002072d-7283-43ae-b645-b26640318053.parquet"
>>> f = pq.ParquetFile(file_path)
# reading with ParquetFile.read gives a single chunk of data
>>> f.read()["l_orderkey"].num_chunks
1
# even when using the read_row_groups API
>>> f.read_row_groups([0, 1])["l_orderkey"].num_chunks
1
# only when using iter_batches, it's of course multiple chunks. The default batch_size here is 2**16,
# which even results in more batches than the number of row groups
>>> pa.Table.from_batches(f.iter_batches())["l_orderkey"].num_chunks
40
# we can make the batch_size larger
>>> pa.Table.from_batches(f.iter_batches(batch_size=128000))["l_orderkey"].num_chunks
21
# strangely it still seems to concatenate *across* row groups when further increasing the batch size
>>> pa.Table.from_batches(f.iter_batches(batch_size=2**17))["l_orderkey"].num_chunks
20
# pq.read_table uses the datasets API, but doesn't allow passing a batch size
>>> pq.read_table(file_path)["l_orderkey"].num_chunks
21
# in the datasets API, now the default batch size is 2**17 instead of 2**16 ...
>>> import pyarrow.dataset as ds
>>> ds.dataset(file_path, format="parquet").to_table()["l_orderkey"].num_chunks
21
# we can lower it (now each individual row group gets split, no combination of data of multiple row groups,
# I think because the GetRecordBatchGenerator uses a sub-generator per row group instead of a single iterator
# for the whole file as GetRecordBatchReader does)
>>> ds.dataset(file_path, format="parquet").to_table(batch_size=2**16)["l_orderkey"].num_chunks
42 So in summary, this is also a bit of a mess on our side (there are many different ways to read a parquet file ..). I had been planning to bring up that you might want to not use On the Arrow side, we should maybe consider to make the default batch size a bit more uniform, and see if we want to use an actual batch size for the ReadTable code path as well. |
I'm quite sure I was using |
I ran perf against a debug build locally (Linux) and on an m6i VM and nothing significantly different seemed to appear. However, it naively seemed like the destructor of Status being the largest overhead was unexpected. Thought it may have been related to https://issues.apache.org/jira/browse/ARROW-2400, and so reverted that w/ no significant change in the perf report. So just wanted to clarify if this is also expected? |
@milesgranger You should run perf against a release build (with optimizations) otherwise the results will probably not be relevant. A good practice is to select the |
I'm guessing that converting these to pandas dataframes would result in them being categorical dtype series. Is that correct?
What would folks recommend as default compression? LZ4? If so, @milesgranger maybe it's easy to change the data generation scripts in some way with this change? I'd be fine changing things in the benchmark if we think it's a good global recommendation. (For context, I don't like changing things in benchmarks to make performance better because it results in over-tuning and non-realistic results, but if the change is good general practice as recommended by other people then it feels better I think). |
@pitrou if it's easy for you, I'm curious how well this parallelizes. If you do it on local four cores do you get ~800MB/s? (assuming that your filesystem can go that fast, which seems likely) |
And what machine did you get that number? (because above (#38389 (comment)) you reported much higher single-threaded numbers) |
@jorisvandenbossche It wasn't the same file. |
Sorry for the delay. Basically, yes, at least with the Snappy-compressed file: >>> %timeit pq.read_table('~/arrow/data/lineitem/lineitem_0002072d-7283-43ae-b645-b26640318053.parquet', use_threads=False, read_dictionary=['l_returnflag
...: ', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'])
575 ms ± 5.49 ms per loop (mean ± std. dev. of 7 runs, 1 loop each) >>> pool = ThreadPoolExecutor(4)
>>> %timeit list(pool.map(lambda _: pq.read_table('~/arrow/data/lineitem/lineitem_0002072d-7283-43ae-b645-b26640318053.parquet', use_threads=False, read_d
...: ictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode']), range(4)))
596 ms ± 19.8 ms per loop (mean ± std. dev. of 7 runs, 1 loop each) Also, do note that the speed I'm reporting here (225 MB/s on one core) is relative to file size. Relative to in-memory Table size, this is more around 525 MB/s on one core. |
Some low-level stats using
At more than 2 instructions per cycle, it seems that we are not suffering much from cache or branch prediction misses. |
Cool. This is encouraging. Thank you for following up on this.
…On Wed, Dec 13, 2023 at 1:12 PM Antoine Pitrou ***@***.***> wrote:
Some low-level stats using perf stat -d -d -d:
Performance counter stats for 'taskset -c 1 python -c import pyarrow.parquet as pq; [pq.read_table("~/arrow/data/lineitem/lineitem_0002072d-7283-43ae-b645-b26640318053.parquet", use_threads=False, read_dictionary=["l_returnflag", "l_linestatus", "l_shipinstruct", "l_shipmode"]).nbytes for i in range(10)]':
6 051,90 msec task-clock # 0,996 CPUs utilized
806 context-switches # 133,181 /sec
1 cpu-migrations # 0,165 /sec
1 340 104 page-faults # 221,435 K/sec
26 695 694 765 cycles # 4,411 GHz (20,17%)
2 085 747 402 stalled-cycles-frontend # 7,81% frontend cycles idle (20,22%)
9 165 304 095 stalled-cycles-backend # 34,33% backend cycles idle (20,25%)
59 629 749 052 instructions # 2,23 insn per cycle
# 0,15 stalled cycles per insn (20,13%)
10 417 615 407 branches # 1,721 G/sec (20,07%)
102 060 991 branch-misses # 0,98% of all branches (20,27%)
27 063 402 822 L1-dcache-loads # 4,472 G/sec (20,25%)
431 791 165 L1-dcache-load-misses # 1,60% of all L1-dcache accesses (20,12%)
<not supported> LLC-loads
<not supported> LLC-load-misses
1 614 170 143 L1-icache-loads # 266,721 M/sec (20,06%)
30 033 666 L1-icache-load-misses # 1,86% of all L1-icache accesses (20,13%)
24 183 320 dTLB-loads # 3,996 M/sec (20,21%)
7 451 324 dTLB-load-misses # 30,81% of all dTLB cache accesses (20,20%)
1 845 633 iTLB-loads # 304,968 K/sec (20,20%)
97 899 iTLB-load-misses # 5,30% of all iTLB cache accesses (20,13%)
200 049 623 L1-dcache-prefetches # 33,056 M/sec (20,11%)
<not supported> L1-dcache-prefetch-misses
6,074363585 seconds time elapsed
4,552905000 seconds user
1,494984000 seconds sys
At more than 2 instructions per cycle, it seems that we are not suffering
much from cache or branch prediction misses.
—
Reply to this email directly, view it on GitHub
<#38389 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTCEMVBKGO7CCZML2JTYJIK5JAVCNFSM6AAAAAA6K5VD2SVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMYTQNJUG4YDQMRSHA>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
I have also tried to regenerate the given file using different compressions and then compared reading performance: >>> !ls -la lineitem-*
-rw-rw-r-- 1 antoine antoine 133922479 déc. 13 22:40 lineitem-lz4.pq
-rw-rw-r-- 1 antoine antoine 129419248 déc. 13 22:38 lineitem-snappy.pq
-rw-rw-r-- 1 antoine antoine 266474815 déc. 13 22:40 lineitem-uncompressed.pq
-rw-rw-r-- 1 antoine antoine 93395071 déc. 13 22:40 lineitem-zstd.pq
>>> %timeit pq.read_table('lineitem-snappy.pq', use_threads=False, read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'])
512 ms ± 12.5 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
>>> %timeit pq.read_table('lineitem-uncompressed.pq', use_threads=False, read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'])
252 ms ± 11 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
>>> %timeit pq.read_table('lineitem-lz4.pq', use_threads=False, read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'])
311 ms ± 13.2 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
>>> %timeit pq.read_table('lineitem-zstd.pq', use_threads=False, read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'])
456 ms ± 13.2 ms per loop (mean ± std. dev. of 7 runs, 1 loop each) So we get:
Note that only the speeds relative to in-memory Table size are comparable, since the file sizes vary. Snappy is the slowest of all options, while producing a file size not much better than lz4 and strictly worse than zstd. |
That said, ~1 GB/s for uncompressed PLAIN-encoded fixed-width data is still very mediocre. I think this has to with the fact that >>> tab = pq.read_table('lineitem-uncompressed.pq', use_threads=False, read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'], columns=['l_or
...: derkey', 'l_partkey', 'l_suppkey', 'l_linenumber', 'l_shipdate', 'l_commitdate'])
>>> [{n: c.num_chunks} for n, c in zip(tab.column_names, tab.columns)]
[{'l_orderkey': 1},
{'l_partkey': 1},
{'l_suppkey': 1},
{'l_linenumber': 1},
{'l_shipdate': 1},
{'l_commitdate': 1}] If I deliberately read row groups separately for these PLAIN-encoded columns, I get almost twice the speed: >>> %timeit pq.read_table('lineitem-uncompressed.pq', use_threads=False, read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'], columns=['l_
...: orderkey', 'l_partkey', 'l_suppkey', 'l_linenumber', 'l_shipdate', 'l_commitdate'])
67.1 ms ± 1.39 ms per loop (mean ± std. dev. of 7 runs, 10 loops each)
>>> f = pq.ParquetFile('lineitem-uncompressed.pq', read_dictionary=['l_returnflag', 'l_linestatus', 'l_shipinstruct', 'l_shipmode'])
>>> %timeit [f.read_row_group(i, use_threads=False, columns=['l_orderkey', 'l_partkey', 'l_suppkey', 'l_linenumber', 'l_shipdate', 'l_commitdate']) for i in range(f.nu
...: m_row_groups)]
36.6 ms ± 841 µs per loop (mean ± std. dev. of 7 runs, 10 loops each) |
…nit (#40143) ### Rationale for this change Closes #40142 I'm developing a new dask integration with pyarrow parquet reader (see dask/dask-expr#882) and want to rely on the pyarrow Filesystem more. Right now, we are performing a list operation ourselves to get all touched files and I would like to pass the retrieved `FileInfo` objects directly to the dataset constructor. This API is already exposed in C++ and this PR is adding the necessary python bindings. The benefit of this is that there is API is that it cuts the need to perform additional HEAD requests to a remote storage. This came up in #38389 (comment) and there's been related work already with #37857 ### What changes are included in this PR? Python bindings for the `DatasetFactory` constructor that accepts a list/vector of `FileInfo` objects. ### Are these changes tested? ~I slightly modified the minio test setup such that the prometheus endpoint is exposed. This can be used to assert that there hasn't been any HEAD requests.~ I ended up removing this again since parsing the response is a bit brittle. ### Are there any user-facing changes? * Closes: #40142 Lead-authored-by: fjetter <[email protected]> Co-authored-by: Joris Van den Bossche <[email protected]> Signed-off-by: Joris Van den Bossche <[email protected]>
…aset init (apache#40143) ### Rationale for this change Closes apache#40142 I'm developing a new dask integration with pyarrow parquet reader (see dask/dask-expr#882) and want to rely on the pyarrow Filesystem more. Right now, we are performing a list operation ourselves to get all touched files and I would like to pass the retrieved `FileInfo` objects directly to the dataset constructor. This API is already exposed in C++ and this PR is adding the necessary python bindings. The benefit of this is that there is API is that it cuts the need to perform additional HEAD requests to a remote storage. This came up in apache#38389 (comment) and there's been related work already with apache#37857 ### What changes are included in this PR? Python bindings for the `DatasetFactory` constructor that accepts a list/vector of `FileInfo` objects. ### Are these changes tested? ~I slightly modified the minio test setup such that the prometheus endpoint is exposed. This can be used to assert that there hasn't been any HEAD requests.~ I ended up removing this again since parsing the response is a bit brittle. ### Are there any user-facing changes? * Closes: apache#40142 Lead-authored-by: fjetter <[email protected]> Co-authored-by: Joris Van den Bossche <[email protected]> Signed-off-by: Joris Van den Bossche <[email protected]>
…aset init (apache#40143) ### Rationale for this change Closes apache#40142 I'm developing a new dask integration with pyarrow parquet reader (see dask/dask-expr#882) and want to rely on the pyarrow Filesystem more. Right now, we are performing a list operation ourselves to get all touched files and I would like to pass the retrieved `FileInfo` objects directly to the dataset constructor. This API is already exposed in C++ and this PR is adding the necessary python bindings. The benefit of this is that there is API is that it cuts the need to perform additional HEAD requests to a remote storage. This came up in apache#38389 (comment) and there's been related work already with apache#37857 ### What changes are included in this PR? Python bindings for the `DatasetFactory` constructor that accepts a list/vector of `FileInfo` objects. ### Are these changes tested? ~I slightly modified the minio test setup such that the prometheus endpoint is exposed. This can be used to assert that there hasn't been any HEAD requests.~ I ended up removing this again since parsing the response is a bit brittle. ### Are there any user-facing changes? * Closes: apache#40142 Lead-authored-by: fjetter <[email protected]> Co-authored-by: Joris Van den Bossche <[email protected]> Signed-off-by: Joris Van den Bossche <[email protected]>
@pitrou the different compression benchmarks were running using hot file cache, right? I'd be surprised if uncompressed would be faster than lz4 without the linux file cache (even if you have the latest NVMe) as reading from disk should be slower than decompression. Also LZ4 should be (roughly) 5GB/s, Snappy 2Gb/s and ZSTD should be 1GB/s on a i7-9700k (2018) on a single thread. I know it hurts data locality and copying around data will never help, but the results keep me thinking what's missing. |
Yes, running on hot file cache. Benchmarking with cold cache is tedious and makes numbers much more difficult to compare (IO speeds vary probably much more than single-thread CPU speeds).
Agreed, though do note that it's on a high-clocked CPU (server CPUs tend to have much lower clock speeds). And in practice it will depend on the corpus as well. |
Describe the bug, including details regarding any error messages, version, and platform.
I'm debugging slow performance in Dask DataFrame and have tracked things down, I think, to slow parquet deserialization in PyArrow. Based on what I know of Arrow I expect to get GB/s and I'm getting more in the range of 100-200 MB/s. What's more is that this seems to depend strongly on the environment (Linux / OSX) I'm using. I could use help tracking this down.
Experiment
I've isolated the performance difference down to the following simple experiment (notebook here):
Results
I've tried this on a variety of cloud machines (intel/arm, VMs/metal, 8-core/64-core, AWS/GCP) and they all get fast disk speeds (probably cached), but only about 150MB/s parquet deserialization speeds. I've tried this on two laptops, one a MBP and one a ThinkPad running Ubuntu and I get ...
In all cases I've installed latest release, PyArrow 13 from conda-forge
Summary
I'm confused by this. I've seen Arrow go way faster than this. I've tried to isolate the problem as much as possible to identify something in my environment that is the cause, but I can't. Everything seems to point to the conclusion that "PyArrow Parquet is just slow on Linux" which doesn't make any sense to me.
I'd welcome any help. Thank you all for your work historically.
Component(s)
Parquet, Python
The text was updated successfully, but these errors were encountered: