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 get_byte_ranges method to AsyncFileReader trait #2115

Merged
merged 4 commits into from Jul 21, 2022
Merged
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
60 changes: 50 additions & 10 deletions parquet/src/arrow/async_reader.rs
Expand Up @@ -77,6 +77,7 @@

use std::collections::VecDeque;
use std::fmt::Formatter;

use std::io::{Cursor, SeekFrom};
use std::ops::Range;
use std::pin::Pin;
Expand All @@ -101,7 +102,7 @@ use crate::column::page::{Page, PageIterator, PageMetadata, PageReader};
use crate::compression::{create_codec, Codec};
use crate::errors::{ParquetError, Result};
use crate::file::footer::{decode_footer, decode_metadata};
use crate::file::metadata::ParquetMetaData;
use crate::file::metadata::{ColumnChunkMetaData, ParquetMetaData};
use crate::file::serialized_reader::{decode_page, read_page_header};
use crate::file::FOOTER_SIZE;
use crate::schema::types::{ColumnDescPtr, SchemaDescPtr, SchemaDescriptor};
Expand All @@ -111,6 +112,27 @@ pub trait AsyncFileReader {
/// Retrieve the bytes in `range`
fn get_bytes(&mut self, range: Range<usize>) -> 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>>>
where
Self: Send,
{
async move {
let mut result = Vec::with_capacity(ranges.len());

for range in ranges.into_iter() {
let data = self.get_bytes(range).await?;
result.push(data);
}

Ok(result)
}
.boxed()
}

/// Provides asynchronous access to the [`ParquetMetaData`] of a parquet file,
/// allowing fine-grained control over how metadata is sourced, in particular allowing
/// for caching, pre-fetching, catalog metadata, etc...
Expand Down Expand Up @@ -366,6 +388,13 @@ where
let mut column_chunks =
vec![None; row_group_metadata.columns().len()];

let mut fetch_ranges =
Vec::with_capacity(column_chunks.len());

let mut update_chunks: Vec<(
Copy link
Contributor

Choose a reason for hiding this comment

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

My gut says that it would be cleaner to just iterate through the column_chunks and use filter_map to extract the ranges, pass this to AsyncFileReader. Convert the result to an iterator and then iterate the column_chunks again, popping the next element from the iterator for each included column.

Not a big deal though

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yeah, I think this was cleaner

&mut Option<InMemoryColumnChunk>,
&ColumnChunkMetaData,
)> = Vec::with_capacity(column_chunks.len());
// TODO: Combine consecutive ranges
for (idx, chunk) in column_chunks.iter_mut().enumerate() {
if !projection.leaf_included(idx) {
Expand All @@ -375,16 +404,27 @@ where
let column = row_group_metadata.column(idx);
let (start, length) = column.byte_range();

let data = input
.get_bytes(start as usize..(start + length) as usize)
.await?;
fetch_ranges
.push(start as usize..(start + length) as usize);

*chunk = Some(InMemoryColumnChunk {
num_values: column.num_values(),
compression: column.compression(),
physical_type: column.column_type(),
data,
});
update_chunks.push((chunk, column));
}

for (idx, data) in input
.get_byte_ranges(fetch_ranges)
.await?
.into_iter()
.enumerate()
Copy link
Contributor

Choose a reason for hiding this comment

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

.zip(update_chunks.iter_mut()) might be cleaner?

{
if let Some((chunk, column)) = update_chunks.get_mut(idx)
{
**chunk = Some(InMemoryColumnChunk {
num_values: column.num_values(),
compression: column.compression(),
physical_type: column.column_type(),
data,
});
}
}

Ok((
Expand Down