Skip to content

Conversation

@Dandandan
Copy link
Contributor

@Dandandan Dandandan commented Nov 26, 2021

Which issue does this PR close?

Closes #1363

Rationale for this change

Parquet reading was slow. This recovers the performance regression in the TPC-H benchmark.

There is still a slowdown in query 10 - and other queries, but this is unrelated to Parquet reading #1367 (and performance still improves from roughly 10 to 7s on that query).

What changes are included in this PR?

Use the BufReadtrait instead ofRead`.

Are there any user-facing changes?

@Dandandan Dandandan changed the title Use BufRead for ChunkObjectReader to improve performance Use BufRead for ChunkObjectReader to revert performance regression in parquet reading Nov 26, 2021
@Dandandan Dandandan requested review from alamb and houqp November 26, 2021 17:30
Copy link
Contributor

@alamb alamb 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 @Dandandan -- I am not sure about the need for the BufRead trait, but otherwise looks good to me.

Very nice 🕵️ work

use std::fs::{self, File, Metadata};
use std::io::{Read, Seek, SeekFrom};
use std::io::{BufRead, BufReader, Read, Seek, SeekFrom};
Copy link
Contributor

Choose a reason for hiding this comment

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

TIL BufRead trait

file.seek(SeekFrom::Start(start))?;
Ok(Box::new(file.take(length as u64)))

let file = BufReader::new(file.take(length as u64));
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems like this is the actual fix, right? Is the change to require the BufRead trait needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not sure. Rerunning some benchmarks now without the trait.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes - looks like BufRead wasn't needed 🎉

@Dandandan Dandandan changed the title Use BufRead for ChunkObjectReader to revert performance regression in parquet reading Use BufReader for ChunkObjectReader to revert performance regression in parquet reading Nov 27, 2021
@Dandandan Dandandan changed the title Use BufReader for ChunkObjectReader to revert performance regression in parquet reading Use BufReader for LocalFileReader to revert performance regression in parquet reading Nov 27, 2021
@Dandandan Dandandan merged commit 7ee85b2 into apache:master Nov 27, 2021
@xudong963
Copy link
Member

🎉

@rdettai
Copy link
Contributor

rdettai commented Nov 28, 2021

Thanks @Dandandan ! Can you quickly explain what the reason for the slowdown was exactly?

@Dandandan
Copy link
Contributor Author

Thanks @Dandandan ! Can you quickly explain what the reason for the slowdown was exactly?

As far as I can explain:

The earlier code used the parquet-based API to read from a file, which uses a BufReader internally, which is crucial for the performance.

By introducing the object storage abstraction, we were directly reading from a File instance without any buffering in between, i.e. having lot's of extra calls to the OS (as you also hinted at in #1363).
This leads to both slowdown in loading the data but also was very expensive in the part that reads metadata /statistics (which normally takes something like <1ms locally). Probably that part does many small read calls.

By wrapping the File instance in the BufReader we avoid those calls to the OS.

Maybe a potential improvement would be having a bit more control, such as setting the capacity of the buffer.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Major performance regression in reading partitioned Parquet data on master

4 participants