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

Add AsyncFileReader trait #1803

Merged
merged 3 commits into from
Jun 7, 2022
Merged

Conversation

tustvold
Copy link
Contributor

@tustvold tustvold commented Jun 6, 2022

Which issue does this PR close?

Part of #1605

Rationale for this change

I originally wanted to introduce an API that would afford clients greater control of this, but particularly with the in-flight work to support column indexes (#1749) and (#1191), it is unclear what exactly this interface should look like. Rather than potentially paint us into a corner, lets keep the interface high-level and we can introduce a lower-level API as and when desired.

What changes are included in this PR?

This extracts an AsyncChunkReader which exposes an API to fetch byte ranges. For backwards compatibility, an implementation is provided for AsyncRead + AsyncSeek. This provides a fairly straightforward location for DataFusion to plug in an object store that supports byte range fetches.

Are there any user-facing changes?

No

@github-actions github-actions bot added the parquet Changes to the parquet crate label Jun 6, 2022
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.

Looks like a nice improvement to me. Thanks @tustvold

fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>>;

/// Retrieve the [`ParquetMetaData`] for this file
fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>>;
Copy link
Contributor

Choose a reason for hiding this comment

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

I find it a little strange that the get_metadata is part of AsyncChunkReader as I would have expected the "read bytes" and "logically read and decode parquet data" more separated

Would it make sense to consider two separate traits? Something like the following perhaps 🤔

/// A reader that can asynchronously read a range of bytes
pub trait AsyncChunkReader: Send + Unpin + 'static {
    /// Retrieve the bytes in `range`
    fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>>;
}

/// Returns parquet metadata, possibly asynchronously 
pub trait AsyncParquetReader: Send + Unpin + 'static {
 /// Retrieve the [`ParquetMetaData`] for this file
    fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>>;
}

Or maybe call it AsyncChunkedParquetReader? (though I admit that is getting to be a mouthful)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

AsyncParquetFile?

I agree it is a little odd, but I want to give flexibility to how this metadata is sourced to allow for caching, pre-fetching, etc...

Copy link
Contributor

Choose a reason for hiding this comment

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

ParquetReaderAccess?

The usecase of cached / catalog'd metadata is a good one -- perhaps we can just add a comment explaining the rationale.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've split the traits up, but it feels very odd to me to have two traits implemented on the same type that is then passed to ParquetRecordBatchStreamBuilder::new...

I'll revisit in the morning, I feel this has just made it more confusing tbh...

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes I agree it is now more confusing -- hoping a good night sleep will make it clearer. I am happy with whatever you decide

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I went with AsyncFileReader and added a load of doc comments


let mut buffer = vec![0_u8; (end - start) as usize];
input.read_exact(buffer.as_mut_slice()).await?;
let data = input
Copy link
Contributor

Choose a reason for hiding this comment

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

this code is just factored into AsyncChunkReader::get_bytes correct?

let (offset_1, length_1) = metadata.row_group(0).column(1).byte_range();
let (offset_2, length_2) = metadata.row_group(0).column(2).byte_range();

assert_eq!(
Copy link
Contributor

Choose a reason for hiding this comment

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

👍

Comment on lines +119 to +120
async move {
self.seek(SeekFrom::Start(range.start as u64)).await?;
Copy link
Member

Choose a reason for hiding this comment

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

Hmm, as get_bytes and get_metadata both async, and they both call seek. Is any chance there will be race condition between then? For example, calling get_bytes first but the file pos is changed by call get_metadata next?

Copy link
Contributor Author

@tustvold tustvold Jun 6, 2022

Choose a reason for hiding this comment

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

The method takes a mutable reference, and return a future with the same lifetime, so they can't race

@tustvold tustvold changed the title Add AsyncChunkReader trait Add AsyncFileReader trait Jun 7, 2022
@codecov-commenter
Copy link

Codecov Report

Merging #1803 (c6d8230) into master (c8d4323) will decrease coverage by 0.00%.
The diff coverage is 80.06%.

❗ Current head c6d8230 differs from pull request most recent head 98515a0. Consider uploading reports for the commit 98515a0 to get more accurate results

@@            Coverage Diff             @@
##           master    #1803      +/-   ##
==========================================
- Coverage   83.42%   83.42%   -0.01%     
==========================================
  Files         198      199       +1     
  Lines       56327    56632     +305     
==========================================
+ Hits        46990    47243     +253     
- Misses       9337     9389      +52     
Impacted Files Coverage Δ
parquet/src/arrow/async_reader.rs 0.00% <0.00%> (ø)
parquet/src/file/metadata.rs 95.12% <ø> (ø)
parquet/src/file/serialized_reader.rs 94.46% <ø> (ø)
parquet/src/util/memory.rs 87.17% <0.00%> (-2.30%) ⬇️
parquet/src/file/page_index/range.rs 90.26% <90.26%> (ø)
arrow/src/ipc/writer.rs 81.78% <95.12%> (+0.85%) ⬆️
arrow/src/ipc/reader.rs 90.82% <100.00%> (+0.08%) ⬆️
parquet/src/encodings/encoding.rs 93.65% <0.00%> (+0.19%) ⬆️
arrow/src/ipc/gen/Message.rs 35.39% <0.00%> (+0.74%) ⬆️

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update c8d4323...98515a0. Read the comment docs.

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

Successfully merging this pull request may close these issues.

None yet

4 participants