apache / arrow

Apache Arrow is a multi-language toolbox for accelerated data interchange and in-memory processing
https://arrow.apache.org/
Apache License 2.0
14.3k stars 3.48k forks source link

[Python][Parquet] Parquet deserialization speeds slower on Linux #38389

Open mrocklin opened 11 months ago

mrocklin commented 11 months ago

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):

# 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)

# Write to local parquet file

pq.write_table(t, "foo.parquet")

# Time Disk speeds

start = time.time()
with open("foo.parquet", mode="rb") as f:
    bytes = f.read()
    nbytes = len(bytes)
stop = time.time()
print("Disk Bandwidth:", int(nbytes / (stop - start) / 2**20), "MiB/s")

# Time Arrow Parquet Speeds

start = time.time()
_ = pq.read_table("foo.parquet")
stop = time.time()
print("PyArrow Read Bandwidth:", int(nbytes / (stop - start) / 2**20), "MiB/s")

# Time In-Memory Read Speeds

start = time.time()
pq.read_table(io.BytesIO(bytes))
stop = time.time()

print("PyArrow In-Memory Bandwidth:", int(nbytes / (stop - start) / 2**20), "MiB/s")

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

pitrou commented 10 months ago

Why would we use 2*numcpus for IO threads? There's an issue open somewhere for per-filesystem thread pools which I think would be a fruitful avenue.

mrocklin commented 10 months ago

For S3, we've found that 2-3x numcpus is pretty good. One get about 50MB/s per S3 connection, and total aggregate S3 bandwidth on Amazon is correlated with machine size (larger machines with more cores have more bandwidth). This scales linearly for modestly sized machines, such that 2-3x ends up being a good general rule.

This is made more explicit at the top of the notebook I shared (using more threads with raw S3 access results in greater aggregate bandwidth).

pitrou commented 10 months ago

Issue for IO thread pool sizes: https://github.com/apache/arrow/issues/30552

pitrou commented 10 months ago

This is made more explicit at the top of the notebook I shared

Can you give a link again? There are many notebooks and benchmark numbers in this issue already :-)

mrocklin commented 10 months ago

Here is a new one with a dataset with larger file sizes (128 MB on disk):

https://gist.github.com/mrocklin/c1fd89575b40c055a9be77b2a47894df

Arrow filesystem does well, but still only about half of what I suspect is possible.

pitrou commented 10 months ago

Thanks @mrocklin . Perhaps this is asking too much, but what are the numbers if you first download the files locally and have Arrow work on the local filesystem?

mrocklin commented 10 months ago

Sure. One moment.

mrocklin commented 10 months ago

I've updated the same notebook, see the bottom:

https://gist.github.com/mrocklin/c1fd89575b40c055a9be77b2a47894df

pitrou commented 10 months ago

Thanks. Those numbers are concerning. I'll try to investigate local performance when I find some time.

mrocklin commented 10 months ago

Thanks Antoine. I appreciate it. I'm not expecting anything from you, but if you happen to come up with something before the middle of next week I'd be excited to use it in some benchmarks I'm currently working on.

mrocklin commented 10 months ago

OK, I think I've gotten the rest of the Dask toolchain to the point where I'm now CPU-bound by Arrow deserialization. If there is anything I can do to to accelerate investigation/improvements here (for example, buying the right person a beer or a vacation on a tropical island) I'd be happy to do so.

mapleFU commented 10 months ago

I'm now CPU-bound by Arrow deserialization

What do you mean by "Arrow deserialization"? Is this parquet related (.i.e decompress, loading rep-def, decoding ) or parquet to arrow (building arrow data type) or casting arrow to other?

mrocklin commented 10 months ago

Sorry for being non-specific. I meant PyArrow Parquet performance specifically. My apologies for the lack of precision.

In particular, the configuration setting I'm using is

pq.ParquetFile(s3_filename, pre_buffer=True).read(
    columns=columns,
    use_threads=False,
    use_pandas_metadata=True,
)

As mentioned above on a 4 core machine when running this in parallel with 4-8 threads I can get up to about 200-250 MB/s. CPU utilization during this period is high. I can get around 500 MB/s S3 bandwidth on the same machine.

I'm also open to other APIs, but that has performed the best for me in my circumstances so far.

mapleFU commented 10 months ago

nice, glad to see that!

eeroel commented 10 months ago

This issue might be related? https://github.com/apache/arrow/issues/38591

mrocklin commented 10 months ago

@eeroel that's an interesting issue. Thank you for sharing and fixing. The chart you share makes the problem super-clear. I'll be curious to see how it impacts observed performance.

I'm still curious about the high CPU-utilization. I was chatting separately with Wes and he mentioned the following (I don't think I'm sharing anythign private)

I would guess that collecting C++ profile data via perf/flamegraph would tell an interesting story about what's taking up the most time in the Parquet deserialization (you generally need to recompile things with -fno-omit-frame-pointer to get reasonable looking profiles). Snappy is a common culprit

Maybe I should try with non-snappy-compressed data and see how things behave. That would probably help to bisect the problem space.

I'm going to summarize a few possible directions to investigate that came out of this discussion:

  1. Try again after #38591
  2. Try without snappy compression
  3. Collect C++ profile data with perf/flamegraph
  4. Look at changing download chunk sizes from 2MB to something larger like 5MB or 10 MB for object-store filesystems (I think I recall someone saying 2MB was the default, but that that was probably decided for local posix filesystems)

No obligation of course for anyone to do this work. This comment is as much for people who I work with as it is for possible Arrow maintainers if they're interested (I hope that you're interested!). For convenience, the notebook I was using above. cc @fjetter

jorisvandenbossche commented 10 months ago
  1. Try again after https://github.com/apache/arrow/issues/38591

Note that I assume you were doing your initial benchmarks with pyarrow 13.0 (because 14.0 was not yet released when the thread started), and I think this is only a regression in 14.0

  1. Try without snappy compression

Antoine posted some numbers above (eg https://github.com/apache/arrow/issues/38389#issuecomment-1777406762) with default of snappy compression, with lz4, and uncompressed (snappy clearly being the slowest, but the actual difference depends on which encoding is being used)

pitrou commented 10 months ago

@mrocklin It seems like those files are not available using S3 anonymous access, am I right? Can you perhaps make one of those files available?

phofl commented 10 months ago

We have a public version here: s3://coiled-data/tpch/scale-1000/ That should be available with anonymous access

pitrou commented 10 months ago

Okay, I took a quick look.

One thing that takes some time is dictionary decoding for the dictionary-encoded columns. You can keep those columns dictionary-encoded in Arrow by passing read_dictionary=["l_returnflag", "l_linestatus", "l_shipinstruct", "l_shipmode"]. This seems to save around 25% CPU time (and also makes the data much more compact in memory).

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.

pitrou commented 10 months ago

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
pitrou commented 10 months ago

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 :-)).

jorisvandenbossche commented 10 months ago

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:

That's because of the use of pq.ParquetFile.read(..) (assuming you were using that here; @mrocklin's gist is using that, but your gist from earlier was using pq.read_table, which should result in much more chunks)

This ParquetFile.read() functions binds to parquet::arrow::FileReader::ReadTable, and I have noticed before that for some reason this concatenates the chunks somewhere in the read path. On the other hand, ParquetFile.iter_batches() binds to FileReader::GetRecordBatchReader and the Dataset API / pq.read_table to FileReader::GetRecordBatchGenerator, and those two APIs will returns smaller batches (first per row group, but they also have a batch_size and will typically also return multiple chunks per row group).

# 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 ParquetFile().read() in dask, because it's a bit slower because of returning a single chunk. Although if it's for the use case to convert to pandas later on, it might also not matter that much (although when using pyarrow strings, then it can matter).

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.

pitrou commented 10 months ago

I'm quite sure I was using pq.read_table above.

milesgranger commented 10 months ago

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?

image

pitrou commented 10 months ago

@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 RelWithDebInfo build type with CMake.

mrocklin commented 9 months ago

You can keep those columns dictionary-encoded in Arrow by passing read_dictionary=["l_returnflag", "l_linestatus", "l_shipinstruct", "l_shipmode"]. This seems to save around 25% CPU time (and also makes the data much more compact in memory).

I'm guessing that converting these to pandas dataframes would result in them being categorical dtype series. Is that correct?

The files don't seem to be at fault (except perhaps for using Snappy :-)).

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).

mrocklin commented 9 months ago

I did these measurements on one CPU core. The overall read speed is around 220 MB/s from the local filesystem

@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)

jorisvandenbossche commented 9 months ago

And what machine did you get that number? (because above (https://github.com/apache/arrow/issues/38389#issuecomment-1777406762) you reported much higher single-threaded numbers)

pitrou commented 9 months ago

@jorisvandenbossche It wasn't the same file.

pitrou commented 9 months ago

I did these measurements on one CPU core. The overall read speed is around 220 MB/s from the local filesystem

@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)

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.

pitrou commented 9 months ago

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.

mrocklin commented 9 months ago

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%)
LLC-loads 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%) 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 , or unsubscribe . You are receiving this because you were mentioned.Message ID: ***@***.***>
pitrou commented 9 months ago

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.

pitrou commented 9 months ago

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 pq.read_table concatenates the row groups together instead of building one more chunk per row group:

>>> 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)
alippai commented 2 months ago

@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.

pitrou commented 2 months ago

@pitrou the different compression benchmarks were running using hot file cache, right?

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).

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.

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.