-
Notifications
You must be signed in to change notification settings - Fork 793
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
feat: add method for async read bloom filter #4917
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -77,7 +77,6 @@ | |
|
||
use std::collections::VecDeque; | ||
use std::fmt::Formatter; | ||
|
||
use std::io::SeekFrom; | ||
use std::ops::Range; | ||
use std::pin::Pin; | ||
|
@@ -88,7 +87,6 @@ use bytes::{Buf, Bytes}; | |
use futures::future::{BoxFuture, FutureExt}; | ||
use futures::ready; | ||
use futures::stream::Stream; | ||
|
||
use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt}; | ||
|
||
use arrow_array::RecordBatch; | ||
|
@@ -102,15 +100,16 @@ use crate::arrow::arrow_reader::{ | |
}; | ||
use crate::arrow::ProjectionMask; | ||
|
||
use crate::bloom_filter::{ | ||
read_bloom_filter_header_and_length, Sbbf, SBBF_HEADER_SIZE_ESTIMATE, | ||
}; | ||
use crate::column::page::{PageIterator, PageReader}; | ||
|
||
use crate::errors::{ParquetError, Result}; | ||
use crate::file::footer::{decode_footer, decode_metadata}; | ||
use crate::file::metadata::{ParquetMetaData, RowGroupMetaData}; | ||
use crate::file::reader::{ChunkReader, Length, SerializedPageReader}; | ||
use crate::format::PageLocation; | ||
|
||
use crate::file::FOOTER_SIZE; | ||
use crate::format::PageLocation; | ||
|
||
mod metadata; | ||
pub use metadata::*; | ||
|
@@ -302,6 +301,46 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> { | |
Self::new_builder(AsyncReader(input), metadata) | ||
} | ||
|
||
/// Read bloom filter for a column in a row group | ||
/// Returns `None` if the column does not have a bloom filter | ||
/// | ||
/// We should call this function after other forms pruning, such as projection and predicate pushdown. | ||
pub async fn get_row_group_column_bloom_filter( | ||
&mut self, | ||
row_group_idx: usize, | ||
column_idx: usize, | ||
) -> Result<Option<Sbbf>> { | ||
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() { | ||
offset.try_into().map_err(|_| { | ||
ParquetError::General("Bloom filter offset is invalid".to_string()) | ||
})? | ||
} else { | ||
return Ok(None); | ||
}; | ||
|
||
let buffer = self | ||
.input | ||
.0 | ||
.get_bytes(offset..offset + SBBF_HEADER_SIZE_ESTIMATE) | ||
.await | ||
.unwrap(); | ||
let (header, length) = read_bloom_filter_header_and_length(buffer)?; | ||
let (header, bitset_offset) = (header, offset + length as usize); | ||
|
||
// length in bytes | ||
let length: usize = header.num_bytes.try_into().map_err(|_| { | ||
ParquetError::General("Bloom filter length is invalid".to_string()) | ||
})?; | ||
let bitset = self | ||
.input | ||
.0 | ||
.get_bytes(bitset_offset..bitset_offset + length) | ||
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. I think it would be ideal if we could avoid this extra roundtrip in the common case, by fetching enough data in the first call 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 first call is used to parse |
||
.await?; | ||
Ok(Some(Sbbf::new(&bitset))) | ||
} | ||
|
||
/// Build a new [`ParquetRecordBatchStream`] | ||
pub fn build(self) -> Result<ParquetRecordBatchStream<T>> { | ||
let num_row_groups = self.metadata.row_groups().len(); | ||
|
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.
There is a new bloom_filter_length that may be present and would avoid needing to guess here
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.
Thanks, i checked the module
bloom_filter
and then updated this part.