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

Cleanup ChunkReader (#4118) #4156

Merged
merged 5 commits into from May 2, 2023

Conversation

tustvold
Copy link
Contributor

@tustvold tustvold commented Apr 28, 2023

Which issue does this PR close?

Closes #4118
Closes #1163

Rationale for this change

The length parameter provided to get_read is very confusing, and whilst attempting to document it in #4147 it became clear that we should probably just remove it. Similarly whilst looking in FileSource, I realised that it doesn't actually protect against concurrent reads #4157.

I therefore opted to just bite the bullet and close off #1163 by removing the length parameter from get_read, and removing FileSource entirely.

What changes are included in this PR?

  • ChunkReader::get_read no longer takes a length parameter
  • Removed FileSource
  • More clearly document that ChunkReader does not support concurrent or interleaved reads, much like File itself

Are there any user-facing changes?

Yes, this changes some, admittedly low-level, public interfaces

@tustvold tustvold added the api-change Changes to the arrow API label Apr 28, 2023
@github-actions github-actions bot added the parquet Changes to the parquet crate label Apr 28, 2023
@@ -112,9 +109,6 @@ impl<R: ParquetReader> FileSource<R> {

impl<R: ParquetReader> Read for FileSource<R> {
fn read(&mut self, buf: &mut [u8]) -> Result<usize> {
let bytes_to_read = cmp::min(buf.len(), (self.end - self.start) as usize);
let buf = &mut buf[0..bytes_to_read];
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This could actually benefit performance, as it will avoid artificially truncating the input buffer, causing it to not skip the FileSource buffer when it could

@tustvold tustvold marked this pull request as draft April 28, 2023 19:24
@tustvold tustvold changed the title Remove length from ChunkReader (#4118) Cleanup ChunkReader (#4118) Apr 28, 2023

/// Get a [`Read`] starting at the provided file offset
///
/// Subsequent or concurrent calls to [`Self::get_read`] or [`Self::get_bytes`] may
Copy link
Contributor Author

@tustvold tustvold Apr 28, 2023

Choose a reason for hiding this comment

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

FileSource provided protection against subsequent calls to get_read, by calling Seek on every read, but provided no protection against concurrent access. I think it is less risky to just clearly not support non-serial usage, than to only break on concurrent usage.

TBC there are no safety implications of not synchronising this access. You will just get interleaved reads, which is no different from just reading gibberish.

One option would be to add Mutex to synchronise access, however, this solution is necessarily incomplete as a user can just call File::try_clone. Ultimately there is no reliable way to synchronise file IO, I think if no synchronisation is fine for the standard library, it is fine for the parquet crate.

@tustvold tustvold marked this pull request as ready for review April 28, 2023 20:47
Copy link
Contributor

@crepererum crepererum left a comment

Choose a reason for hiding this comment

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

Simply better 👍

Copy link
Member

@viirya viirya left a comment

Choose a reason for hiding this comment

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

Looks much cleaner.

@tustvold tustvold merged commit eb5ac69 into apache:master May 2, 2023
14 checks passed
@zeevm
Copy link
Contributor

zeevm commented May 18, 2023

@tustvold the length argument was quite useful I think, my main use case is processing Parquet files from cloud storage (azure, s3 etc.) so I'm trying to minimize on downloaded data to both save time and cost, I used the length argument (with some additional extension if it was below some threshold) to only download what the reader required in a single 'read' op (cloud storage services usually charge per 10K ops + bandwidth)

Now I don't know how much to download, I'll have to develop a bunch of heuristics to get to some sensible values.

Since the reader always needs some specific entity (say page, or entire column chunk, or the file metadata) it knows what 'length' it needs, why not provide this as a 'hint' to the implementation?

@tustvold
Copy link
Contributor Author

It was removed precisely to discourage this style of usage, see the linked issues, as the ranges passed to it are fairly arbitrary. I would recommend using the metadata to determine the necessary byte ranges up front, or using the async reader which handles this for you

@zeevm
Copy link
Contributor

zeevm commented May 18, 2023

@tustvold IIUC the async reader reads into Arrow in memory format, not providing a lower level direct column or page reading correct?

This doesn't work for me as we have our own in memory format for our db, going through Arrow would incur a lot of expansive allocations and format translation.

@tustvold
Copy link
Contributor Author

Correct, the high-level API reads into arrow and supports object stores, in addition to predicate pushdown, late materialization, row filtering, etc... The low-level APIs support advanced usage, but are not nearly as batteries included. I would encourage you to try out the arrow APIs, you may find they are actually faster 😅

@zeevm
Copy link
Contributor

zeevm commented May 19, 2023 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
api-change Changes to the arrow API parquet Changes to the parquet crate
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Improve Documentation of Parquet ChunkReader Use Standard Library IO Abstractions in Parquet
4 participants