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

Change Parquet API interaction for u64 #7252

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all 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
2 changes: 1 addition & 1 deletion parquet/examples/read_with_rowgroup.rs
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ impl InMemoryRowGroup {
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)
.get_bytes(start..(start + len))
.await?;

vs[leaf_idx] = Some(Arc::new(ColumnChunkData {
Expand Down
10 changes: 5 additions & 5 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 @@ -202,7 +202,7 @@ impl RowSelection {
if !(selector.skip || current_page_included) {
let start = page.offset as usize;
let end = start + page.compressed_page_size as usize;
ranges.push(start..end);
ranges.push(start as u64..end as u64);
}
current_selector = selectors.next()
}
Expand Down
2 changes: 1 addition & 1 deletion parquet/src/arrow/async_reader/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ pub trait MetadataFetch {

impl<T: AsyncFileReader> MetadataFetch for &mut T {
fn fetch(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> {
self.get_bytes(range)
self.get_bytes(range.start as u64..range.end 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.

I think we should also change this trait

}
}

Expand Down
49 changes: 25 additions & 24 deletions parquet/src/arrow/async_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -83,10 +83,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());

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

Expand Down Expand Up @@ -412,7 +412,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 @@ -421,16 +421,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 @@ -449,14 +449,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 @@ -866,7 +866,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 @@ -879,11 +879,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 @@ -911,7 +911,7 @@ impl InMemoryRowGroup<'_> {

*chunk = Some(Arc::new(ColumnChunkData::Sparse {
length: self.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 @@ -924,7 +924,7 @@ impl InMemoryRowGroup<'_> {
.map(|(idx, _chunk)| {
let column = self.metadata.column(idx);
let (start, length) = column.byte_range();
start as usize..(start + length) as usize
start..(start + length)
})
.collect();

Expand Down Expand Up @@ -1083,9 +1083,10 @@ 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>>> {
Expand Down
15 changes: 11 additions & 4 deletions parquet/src/arrow/async_reader/store.rs
Original file line number Diff line number Diff line change
Expand Up @@ -146,15 +146,22 @@ impl ParquetObjectReader {
}

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

// This method doesn't directly call `self.spawn` because all of the IO that is done down the
Expand Down
Loading