-
Notifications
You must be signed in to change notification settings - Fork 970
Change Parquet API interaction for u64 #7252
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
Closed
Closed
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -48,12 +48,13 @@ use std::ops::Range; | |||||
/// file: tokio::fs::File, | ||||||
/// } | ||||||
/// impl MetadataFetch for TokioFileMetadata { | ||||||
/// fn fetch(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
/// fn fetch(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
/// // return a future that fetches data in range | ||||||
/// async move { | ||||||
/// let mut buf = vec![0; range.len()]; // target buffer | ||||||
/// let len = (range.end - range.start) as usize; | ||||||
/// let mut buf = vec![0; len]; // target buffer | ||||||
/// // seek to the start of the range and read the data | ||||||
/// self.file.seek(SeekFrom::Start(range.start as u64)).await?; | ||||||
/// self.file.seek(SeekFrom::Start(range.start)).await?; | ||||||
/// self.file.read_exact(&mut buf).await?; | ||||||
/// Ok(Bytes::from(buf)) // convert to Bytes | ||||||
/// } | ||||||
|
@@ -66,12 +67,12 @@ pub trait MetadataFetch { | |||||
/// | ||||||
/// Note the returned type is a boxed future, often created by | ||||||
/// [FutureExt::boxed]. See the trait documentation for an example | ||||||
fn fetch(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>>; | ||||||
fn fetch(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>>; | ||||||
} | ||||||
|
||||||
impl<T: AsyncFileReader> MetadataFetch for &mut T { | ||||||
fn fetch(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
self.get_bytes(range) | ||||||
fn fetch(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
self.get_bytes(range.start..range.end) | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The trait was updated to also use
Suggested change
|
||||||
} | ||||||
} | ||||||
|
||||||
|
@@ -107,7 +108,7 @@ impl<F: MetadataFetch> MetadataLoader<F> { | |||||
file_size - FOOTER_SIZE | ||||||
}; | ||||||
|
||||||
let suffix = fetch.fetch(footer_start..file_size).await?; | ||||||
let suffix = fetch.fetch(footer_start as u64..file_size as u64).await?; | ||||||
let suffix_len = suffix.len(); | ||||||
|
||||||
let mut footer = [0; FOOTER_SIZE]; | ||||||
|
@@ -127,7 +128,9 @@ impl<F: MetadataFetch> MetadataLoader<F> { | |||||
// Did not fetch the entire file metadata in the initial read, need to make a second request | ||||||
let (metadata, remainder) = if length > suffix_len - FOOTER_SIZE { | ||||||
let metadata_start = file_size - length - FOOTER_SIZE; | ||||||
let meta = fetch.fetch(metadata_start..file_size - FOOTER_SIZE).await?; | ||||||
let meta = fetch | ||||||
.fetch(metadata_start as u64..(file_size - FOOTER_SIZE) as u64) | ||||||
.await?; | ||||||
(ParquetMetaDataReader::decode_metadata(&meta)?, None) | ||||||
} else { | ||||||
let metadata_start = file_size - length - FOOTER_SIZE - footer_start; | ||||||
|
@@ -182,7 +185,11 @@ impl<F: MetadataFetch> MetadataLoader<F> { | |||||
remainder.slice(offset..range.end - *remainder_start + offset) | ||||||
} | ||||||
// Note: this will potentially fetch data already in remainder, this keeps things simple | ||||||
_ => self.fetch.fetch(range.start..range.end).await?, | ||||||
_ => { | ||||||
self.fetch | ||||||
.fetch(range.start as u64..range.end as u64) | ||||||
.await? | ||||||
} | ||||||
}; | ||||||
|
||||||
// Sanity check | ||||||
|
@@ -246,8 +253,8 @@ where | |||||
F: FnMut(Range<usize>) -> Fut + Send, | ||||||
Fut: Future<Output = Result<Bytes>> + Send, | ||||||
{ | ||||||
fn fetch(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
async move { self.0(range).await }.boxed() | ||||||
fn fetch(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
async move { self.0(range.start as usize..range.end as usize).await }.boxed() | ||||||
} | ||||||
} | ||||||
|
||||||
|
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -86,10 +86,10 @@ pub use store::*; | |||||
/// [`tokio::fs::File`]: https://docs.rs/tokio/latest/tokio/fs/struct.File.html | ||||||
pub trait AsyncFileReader: Send { | ||||||
/// Retrieve the bytes in `range` | ||||||
fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>>; | ||||||
fn get_bytes(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>>; | ||||||
|
||||||
/// Retrieve multiple byte ranges. The default implementation will call `get_bytes` sequentially | ||||||
fn get_byte_ranges(&mut self, ranges: Vec<Range<usize>>) -> BoxFuture<'_, Result<Vec<Bytes>>> { | ||||||
fn get_byte_ranges(&mut self, ranges: Vec<Range<u64>>) -> BoxFuture<'_, Result<Vec<Bytes>>> { | ||||||
async move { | ||||||
let mut result = Vec::with_capacity(ranges.len()); | ||||||
|
||||||
|
@@ -124,11 +124,11 @@ pub trait AsyncFileReader: Send { | |||||
|
||||||
/// This allows Box<dyn AsyncFileReader + '_> to be used as an AsyncFileReader, | ||||||
impl AsyncFileReader for Box<dyn AsyncFileReader + '_> { | ||||||
fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
fn get_bytes(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
self.as_mut().get_bytes(range) | ||||||
} | ||||||
|
||||||
fn get_byte_ranges(&mut self, ranges: Vec<Range<usize>>) -> BoxFuture<'_, Result<Vec<Bytes>>> { | ||||||
fn get_byte_ranges(&mut self, ranges: Vec<Range<u64>>) -> BoxFuture<'_, Result<Vec<Bytes>>> { | ||||||
self.as_mut().get_byte_ranges(ranges) | ||||||
} | ||||||
|
||||||
|
@@ -145,14 +145,14 @@ impl AsyncFileReader for Box<dyn AsyncFileReader + '_> { | |||||
} | ||||||
|
||||||
impl<T: AsyncRead + AsyncSeek + Unpin + Send> AsyncFileReader for T { | ||||||
fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
fn get_bytes(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
async move { | ||||||
self.seek(SeekFrom::Start(range.start as u64)).await?; | ||||||
self.seek(SeekFrom::Start(range.start)).await?; | ||||||
|
||||||
let to_read = range.end - range.start; | ||||||
let mut buffer = Vec::with_capacity(to_read); | ||||||
let read = self.take(to_read as u64).read_to_end(&mut buffer).await?; | ||||||
if read != to_read { | ||||||
let mut buffer = Vec::with_capacity(to_read as usize); | ||||||
let read = self.take(to_read).read_to_end(&mut buffer).await?; | ||||||
if read != to_read as usize { | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
return Err(eof_err!("expected to read {} bytes, got {}", to_read, read)); | ||||||
} | ||||||
|
||||||
|
@@ -482,7 +482,7 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> { | |||||
let metadata = self.metadata.row_group(row_group_idx); | ||||||
let column_metadata = metadata.column(column_idx); | ||||||
|
||||||
let offset: usize = if let Some(offset) = column_metadata.bloom_filter_offset() { | ||||||
let offset: u64 = if let Some(offset) = column_metadata.bloom_filter_offset() { | ||||||
offset | ||||||
.try_into() | ||||||
.map_err(|_| ParquetError::General("Bloom filter offset is invalid".to_string()))? | ||||||
|
@@ -491,16 +491,16 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> { | |||||
}; | ||||||
|
||||||
let buffer = match column_metadata.bloom_filter_length() { | ||||||
Some(length) => self.input.0.get_bytes(offset..offset + length as usize), | ||||||
Some(length) => self.input.0.get_bytes(offset..offset + length as u64), | ||||||
None => self | ||||||
.input | ||||||
.0 | ||||||
.get_bytes(offset..offset + SBBF_HEADER_SIZE_ESTIMATE), | ||||||
.get_bytes(offset..offset + SBBF_HEADER_SIZE_ESTIMATE as u64), | ||||||
} | ||||||
.await?; | ||||||
|
||||||
let (header, bitset_offset) = | ||||||
chunk_read_bloom_filter_header_and_offset(offset as u64, buffer.clone())?; | ||||||
chunk_read_bloom_filter_header_and_offset(offset, buffer.clone())?; | ||||||
|
||||||
match header.algorithm { | ||||||
BloomFilterAlgorithm::BLOCK(_) => { | ||||||
|
@@ -519,14 +519,14 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> { | |||||
} | ||||||
|
||||||
let bitset = match column_metadata.bloom_filter_length() { | ||||||
Some(_) => buffer.slice((bitset_offset as usize - offset)..), | ||||||
Some(_) => buffer.slice((bitset_offset as usize - offset as usize)..), | ||||||
None => { | ||||||
let bitset_length: usize = header.num_bytes.try_into().map_err(|_| { | ||||||
let bitset_length: u64 = header.num_bytes.try_into().map_err(|_| { | ||||||
ParquetError::General("Bloom filter length is invalid".to_string()) | ||||||
})?; | ||||||
self.input | ||||||
.0 | ||||||
.get_bytes(bitset_offset as usize..bitset_offset as usize + bitset_length) | ||||||
.get_bytes(bitset_offset..bitset_offset + bitset_length) | ||||||
.await? | ||||||
} | ||||||
}; | ||||||
|
@@ -939,7 +939,7 @@ impl InMemoryRowGroup<'_> { | |||||
if let Some((selection, offset_index)) = selection.zip(self.offset_index) { | ||||||
// If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the | ||||||
// `RowSelection` | ||||||
let mut page_start_offsets: Vec<Vec<usize>> = vec![]; | ||||||
let mut page_start_offsets: Vec<Vec<u64>> = vec![]; | ||||||
|
||||||
let fetch_ranges = self | ||||||
.column_chunks | ||||||
|
@@ -952,11 +952,11 @@ impl InMemoryRowGroup<'_> { | |||||
.flat_map(|(idx, (_chunk, chunk_meta))| { | ||||||
// If the first page does not start at the beginning of the column, | ||||||
// then we need to also fetch a dictionary page. | ||||||
let mut ranges = vec![]; | ||||||
let mut ranges: Vec<Range<u64>> = vec![]; | ||||||
let (start, _len) = chunk_meta.byte_range(); | ||||||
match offset_index[idx].page_locations.first() { | ||||||
Some(first) if first.offset as u64 != start => { | ||||||
ranges.push(start as usize..first.offset as usize); | ||||||
ranges.push(start..first.offset as u64); | ||||||
} | ||||||
_ => (), | ||||||
} | ||||||
|
@@ -984,7 +984,11 @@ impl InMemoryRowGroup<'_> { | |||||
|
||||||
*chunk = Some(Arc::new(ColumnChunkData::Sparse { | ||||||
length: metadata.column(idx).byte_range().1 as usize, | ||||||
data: offsets.into_iter().zip(chunks.into_iter()).collect(), | ||||||
data: offsets | ||||||
.into_iter() | ||||||
.map(|x| x as usize) | ||||||
.zip(chunks.into_iter()) | ||||||
.collect(), | ||||||
})) | ||||||
} | ||||||
} | ||||||
|
@@ -997,7 +1001,7 @@ impl InMemoryRowGroup<'_> { | |||||
.map(|(idx, _chunk)| { | ||||||
let column = metadata.column(idx); | ||||||
let (start, length) = column.byte_range(); | ||||||
start as usize..(start + length) as usize | ||||||
start..(start + length) | ||||||
}) | ||||||
.collect(); | ||||||
|
||||||
|
@@ -1193,9 +1197,16 @@ mod tests { | |||||
} | ||||||
|
||||||
impl AsyncFileReader for TestReader { | ||||||
fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
self.requests.lock().unwrap().push(range.clone()); | ||||||
futures::future::ready(Ok(self.data.slice(range))).boxed() | ||||||
fn get_bytes(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> { | ||||||
let range = range.clone(); | ||||||
self.requests | ||||||
.lock() | ||||||
.unwrap() | ||||||
.push(range.start as usize..range.end as usize); | ||||||
futures::future::ready(Ok(self | ||||||
.data | ||||||
.slice(range.start as usize..range.end as usize))) | ||||||
.boxed() | ||||||
} | ||||||
|
||||||
fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>> { | ||||||
|
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.