masonh22 commented on code in PR #6676:
URL: https://github.com/apache/arrow-rs/pull/6676#discussion_r1828530119
##########
parquet/src/arrow/async_reader/mod.rs:
##########
@@ -660,36 +705,89 @@ where
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) ->
Poll<Option<Self::Item>> {
loop {
match &mut self.state {
- StreamState::Decoding(batch_reader) => match
batch_reader.next() {
- Some(Ok(batch)) => {
- return Poll::Ready(Some(Ok(batch)));
+ StreamState::Decoding(batch_reader) => {
+ let res: Self::Item = match batch_reader.next() {
+ Some(Ok(batch)) => Ok(batch),
+ Some(Err(e)) => {
+ self.state = StreamState::Error;
+ return
Poll::Ready(Some(Err(ParquetError::ArrowError(e.to_string()))));
+ }
+ None => {
+ self.state = StreamState::Init;
+ continue;
+ }
+ };
+
+ if !self.prefetch_row_groups
+ || self.row_groups.is_empty()
+ || self.next_reader.is_some()
+ {
+ return Poll::Ready(Some(res));
}
- Some(Err(e)) => {
- self.state = StreamState::Error;
- return
Poll::Ready(Some(Err(ParquetError::ArrowError(e.to_string()))));
+
+ let old_state = std::mem::replace(&mut self.state,
StreamState::Init);
+
+ let row_group_idx = self.row_groups.pop_front().unwrap();
// already checked that row_groups is not empty
+
+ let fut = self.read_row_group(row_group_idx);
+
+ if let StreamState::Decoding(batch_reader) = old_state {
+ self.state = StreamState::Prefetch(batch_reader, fut);
+ return Poll::Ready(Some(res));
+ } else {
+ unreachable!()
}
- None => self.state = StreamState::Init,
- },
+ }
+ StreamState::Prefetch(batch_reader, f) => {
+ let mut noop_cx =
Context::from_waker(futures::task::noop_waker_ref());
Review Comment:
I wanted to avoid any potential overhead from using the real context when
polling the future here. Since we're always returning Poll::Ready out of this
state (or transitioning to another state), we don't need to rely on the real
context to wake the main stream future.
I'm not an expert at async rust code though so if it would make more sense
to do something else here I'm happy to make that change.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]