Skip to content

Change Parquet API interaction to use u64 (support files larger than 4GB in WASM) #7371

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

Merged
merged 18 commits into from
Apr 8, 2025
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 1 addition & 3 deletions parquet/examples/read_with_rowgroup.rs
Original file line number Diff line number Diff line change
Expand Up @@ -166,9 +166,7 @@ impl InMemoryRowGroup {
for (leaf_idx, meta) in self.metadata.columns().iter().enumerate() {
if self.mask.leaf_included(leaf_idx) {
let (start, len) = meta.byte_range();
let data = reader
.get_bytes(start as usize..(start + len) as usize)
.await?;
let data = reader.get_bytes(start..(start + len)).await?;

vs[leaf_idx] = Some(Arc::new(ColumnChunkData {
offset: start as usize,
Expand Down
12 changes: 6 additions & 6 deletions parquet/src/arrow/arrow_reader/selection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,8 @@ impl RowSelection {
/// Note: this method does not make any effort to combine consecutive ranges, nor coalesce
/// ranges that are close together. This is instead delegated to the IO subsystem to optimise,
/// e.g. [`ObjectStore::get_ranges`](object_store::ObjectStore::get_ranges)
pub fn scan_ranges(&self, page_locations: &[crate::format::PageLocation]) -> Vec<Range<usize>> {
let mut ranges = vec![];
pub fn scan_ranges(&self, page_locations: &[crate::format::PageLocation]) -> Vec<Range<u64>> {
let mut ranges: Vec<Range<u64>> = vec![];
let mut row_offset = 0;

let mut pages = page_locations.iter().peekable();
Expand All @@ -175,8 +175,8 @@ impl RowSelection {

while let Some((selector, page)) = current_selector.as_mut().zip(current_page) {
if !(selector.skip || current_page_included) {
let start = page.offset as usize;
let end = start + page.compressed_page_size as usize;
let start = page.offset as u64;
let end = start + page.compressed_page_size as u64;
ranges.push(start..end);
current_page_included = true;
}
Expand All @@ -200,8 +200,8 @@ impl RowSelection {
}
} else {
if !(selector.skip || current_page_included) {
let start = page.offset as usize;
let end = start + page.compressed_page_size as usize;
let start = page.offset as u64;
let end = start + page.compressed_page_size as u64;
ranges.push(start..end);
}
current_selector = selectors.next()
Expand Down
29 changes: 18 additions & 11 deletions parquet/src/arrow/async_reader/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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
/// }
Expand All @@ -66,11 +67,11 @@ 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>> {
fn fetch(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> {
self.get_bytes(range)
}
}
Expand All @@ -82,7 +83,7 @@ pub trait MetadataSuffixFetch: MetadataFetch {
///
/// Note the returned type is a boxed future, often created by
/// [FutureExt::boxed]. See the trait documentation for an example
fn fetch_suffix(&mut self, suffix: usize) -> BoxFuture<'_, Result<Bytes>>;
fn fetch_suffix(&mut self, suffix: u64) -> BoxFuture<'_, Result<Bytes>>;
}

/// An asynchronous interface to load [`ParquetMetaData`] from an async source
Expand Down Expand Up @@ -117,7 +118,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];
Expand All @@ -137,7 +138,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;
Expand Down Expand Up @@ -192,7 +195,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
Expand Down Expand Up @@ -256,8 +263,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()
}
}

Expand Down
59 changes: 35 additions & 24 deletions parquet/src/arrow/async_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -81,10 +81,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>>;
Copy link
Contributor

Choose a reason for hiding this comment

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

👍


/// 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());

Expand All @@ -110,11 +110,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)
}

Expand All @@ -127,14 +127,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 as u64 != to_read {
return Err(eof_err!("expected to read {} bytes, got {}", to_read, read));
}

Expand Down Expand Up @@ -441,7 +441,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()))?
Expand All @@ -450,16 +450,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(_) => {
Expand All @@ -478,14 +478,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?
}
};
Expand Down Expand Up @@ -897,7 +897,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
Expand All @@ -910,11 +910,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);
}
_ => (),
}
Expand Down Expand Up @@ -942,7 +942,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(),
}))
}
}
Expand All @@ -955,7 +959,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();

Expand Down Expand Up @@ -1125,9 +1129,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<'a>(
Expand Down
13 changes: 4 additions & 9 deletions parquet/src/arrow/async_reader/store.rs
Original file line number Diff line number Diff line change
Expand Up @@ -161,9 +161,9 @@ impl ParquetObjectReader {
}

impl MetadataSuffixFetch for &mut ParquetObjectReader {
fn fetch_suffix(&mut self, suffix: usize) -> BoxFuture<'_, Result<Bytes>> {
fn fetch_suffix(&mut self, suffix: u64) -> BoxFuture<'_, Result<Bytes>> {
let options = GetOptions {
range: Some(GetRange::Suffix(suffix as u64)),
range: Some(GetRange::Suffix(suffix)),
..Default::default()
};
self.spawn(|store, path| {
Expand All @@ -177,19 +177,14 @@ impl MetadataSuffixFetch for &mut ParquetObjectReader {
}

impl AsyncFileReader for ParquetObjectReader {
fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> {
let range = range.start as u64..range.end as u64;
fn get_bytes(&mut self, range: Range<u64>) -> BoxFuture<'_, Result<Bytes>> {
self.spawn(|store, path| store.get_range(path, 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>>>
where
Self: Send,
{
let ranges = ranges
.into_iter()
.map(|range| range.start as u64..range.end as u64)
.collect::<Vec<_>>();
self.spawn(|store, path| async move { store.get_ranges(path, &ranges).await }.boxed())
}

Expand Down
Loading
Loading