-
Notifications
You must be signed in to change notification settings - Fork 664
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
Document Async decoder usage (#4043) (#78) #4046
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 |
---|---|---|
|
@@ -17,6 +17,8 @@ | |
|
||
//! CSV Reader | ||
//! | ||
//! # Basic Usage | ||
//! | ||
//! This CSV reader allows CSV files to be read into the Arrow memory model. Records are | ||
//! loaded in batches and are then converted from row-based data to columnar data. | ||
//! | ||
|
@@ -39,6 +41,84 @@ | |
//! let mut csv = Reader::new(file, Arc::new(schema), false, None, 1024, None, None, None); | ||
//! let batch = csv.next().unwrap().unwrap(); | ||
//! ``` | ||
//! | ||
//! # Async Usage | ||
//! | ||
//! The lower-level [`Decoder`] can be integrated with various forms of async data streams. | ||
//! | ||
//! For example, see below for how it can be used with an arbitrary `Stream` of `Bytes` | ||
//! | ||
//! ``` | ||
//! # use std::task::{Poll, ready}; | ||
//! # use bytes::{Buf, Bytes}; | ||
//! # use arrow_schema::ArrowError; | ||
//! # use futures::stream::{Stream, StreamExt}; | ||
//! # use arrow_array::RecordBatch; | ||
//! # use arrow_csv::reader::Decoder; | ||
//! # | ||
//! fn decode_stream<S: Stream<Item = Bytes> + Unpin>( | ||
//! mut decoder: Decoder, | ||
//! mut input: S, | ||
//! ) -> impl Stream<Item = Result<RecordBatch, ArrowError>> { | ||
//! let mut buffered = Bytes::new(); | ||
//! futures::stream::poll_fn(move |cx| { | ||
//! loop { | ||
//! if buffered.is_empty() { | ||
//! if let Some(b) = ready!(input.poll_next_unpin(cx)) { | ||
//! buffered = b; | ||
//! } | ||
//! } | ||
//! let decoded = match decoder.decode(buffered.as_ref()) { | ||
//! // Note: the decoder needs to be called with an empty | ||
//! // array to delimit the final record | ||
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. Where is this done in this example? I would have expected that there's some 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 fallthrough of the branch above will end up here, I'll move it up to the match on poll_next_unpin to make it more clear |
||
//! Ok(0) => break, | ||
//! Ok(decoded) => decoded, | ||
//! Err(e) => return Poll::Ready(Some(Err(e))), | ||
//! }; | ||
//! buffered.advance(decoded); | ||
//! } | ||
//! | ||
//! Poll::Ready(decoder.flush().transpose()) | ||
//! }) | ||
//! } | ||
//! | ||
//! ``` | ||
//! | ||
//! In a similar vein, it can also be used with tokio-based IO primitives | ||
//! | ||
//! ``` | ||
//! # use std::pin::Pin; | ||
//! # use std::task::{Poll, ready}; | ||
//! # use futures::Stream; | ||
//! # use tokio::io::AsyncBufRead; | ||
//! # use arrow_array::RecordBatch; | ||
//! # use arrow_csv::reader::Decoder; | ||
//! # use arrow_schema::ArrowError; | ||
//! fn decode_stream<R: AsyncBufRead + Unpin>( | ||
//! mut decoder: Decoder, | ||
//! mut reader: R, | ||
//! ) -> impl Stream<Item = Result<RecordBatch, ArrowError>> { | ||
//! futures::stream::poll_fn(move |cx| { | ||
//! loop { | ||
//! let b = match ready!(Pin::new(&mut reader).poll_fill_buf(cx)) { | ||
//! Ok(b) => b, | ||
//! Err(e) => return Poll::Ready(Some(Err(e.into()))), | ||
//! }; | ||
//! let decoded = match decoder.decode(b) { | ||
//! // Note: the decoder needs to be called with an empty | ||
//! // array to delimit the final record | ||
//! Ok(0) => break, | ||
//! Ok(decoded) => decoded, | ||
//! Err(e) => return Poll::Ready(Some(Err(e))), | ||
//! }; | ||
//! Pin::new(&mut reader).consume(decoded); | ||
//! } | ||
//! | ||
//! Poll::Ready(decoder.flush().transpose()) | ||
//! }) | ||
//! } | ||
//! ``` | ||
//! | ||
|
||
mod records; | ||
|
||
|
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.
The "various forms" here is key, one of the major challenges in accommodating the async ecosystem is there is no one-size fits all async IO primitive, the intention behind the Decoder interface is to not be opinionated about where the bytes are coming from. This is similar to the AsyncFileReader trait we have for parquet, which similarly is not opinionated about what the underlying IO primitive actually is.
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.
This information should be part of the doc-string, not hidden in some PR comment.