-
Notifications
You must be signed in to change notification settings - Fork 939
feat(parquet): Implement AsyncFileWriter for object_store::buffered::BufWriter
#6013
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
Merged
Merged
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
35e463a
feat(parquet): Implement AsyncFileWriter for obejct_store::BufWriter
Xuanwo bc9b510
Fix build
Xuanwo 46f05c4
Bump object_store
Xuanwo 8f7fcb5
Merge remote-tracking branch 'origin/master' into impl-async-file-write
Xuanwo ecf2a3b
Apply suggestions from code review
Xuanwo 9b63527
Address comments
Xuanwo 43c1395
Add comments
Xuanwo 978d8ef
Make it better to read
Xuanwo bc6810e
Fix docs
Xuanwo File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,157 @@ | ||
// Licensed to the Apache Software Foundation (ASF) under one | ||
// or more contributor license agreements. See the NOTICE file | ||
// distributed with this work for additional information | ||
// regarding copyright ownership. The ASF licenses this file | ||
// to you under the Apache License, Version 2.0 (the | ||
// "License"); you may not use this file except in compliance | ||
// with the License. You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, | ||
// software distributed under the License is distributed on an | ||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
// KIND, either express or implied. See the License for the | ||
// specific language governing permissions and limitations | ||
// under the License. | ||
|
||
use bytes::Bytes; | ||
use futures::future::BoxFuture; | ||
use std::sync::Arc; | ||
|
||
use crate::arrow::async_writer::AsyncFileWriter; | ||
use crate::errors::{ParquetError, Result}; | ||
use object_store::buffered::BufWriter; | ||
use object_store::path::Path; | ||
use object_store::ObjectStore; | ||
use tokio::io::AsyncWriteExt; | ||
|
||
Xuanwo marked this conversation as resolved.
Show resolved
Hide resolved
|
||
/// [`ParquetObjectWriter`] for writing to parquet to [`ObjectStore`] | ||
/// | ||
/// ``` | ||
/// # use arrow_array::{ArrayRef, Int64Array, RecordBatch}; | ||
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. ❤️ -- this is great -- thank you |
||
/// # use object_store::memory::InMemory; | ||
/// # use object_store::path::Path; | ||
/// # use object_store::ObjectStore; | ||
/// # use std::sync::Arc; | ||
/// | ||
/// # use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; | ||
/// # use parquet::arrow::async_writer::ParquetObjectWriter; | ||
/// # use parquet::arrow::AsyncArrowWriter; | ||
/// | ||
/// # #[tokio::main(flavor="current_thread")] | ||
/// # async fn main() { | ||
/// let store = Arc::new(InMemory::new()); | ||
/// | ||
/// let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef; | ||
/// let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap(); | ||
/// | ||
/// let object_store_writer = ParquetObjectWriter::new(store.clone(), Path::from("test")); | ||
/// let mut writer = | ||
/// AsyncArrowWriter::try_new(object_store_writer, to_write.schema(), None).unwrap(); | ||
/// writer.write(&to_write).await.unwrap(); | ||
/// writer.close().await.unwrap(); | ||
/// | ||
/// let buffer = store | ||
/// .get(&Path::from("test")) | ||
/// .await | ||
/// .unwrap() | ||
/// .bytes() | ||
/// .await | ||
/// .unwrap(); | ||
/// let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer) | ||
/// .unwrap() | ||
/// .build() | ||
/// .unwrap(); | ||
/// let read = reader.next().unwrap().unwrap(); | ||
/// | ||
/// assert_eq!(to_write, read); | ||
/// # } | ||
/// ``` | ||
#[derive(Debug)] | ||
pub struct ParquetObjectWriter { | ||
w: BufWriter, | ||
} | ||
|
||
impl ParquetObjectWriter { | ||
/// Create a new [`ParquetObjectWriter`] that writes to the specified path in the given store. | ||
/// | ||
/// To configure the writer behavior, please build [`BufWriter`] and then use [`Self::from_buf_writer`] | ||
pub fn new(store: Arc<dyn ObjectStore>, path: Path) -> Self { | ||
Self::from_buf_writer(BufWriter::new(store, path)) | ||
} | ||
|
||
/// Construct a new ParquetObjectWriter via a existing BufWriter. | ||
pub fn from_buf_writer(w: BufWriter) -> Self { | ||
Self { w } | ||
} | ||
|
||
/// Consume the writer and return the underlying BufWriter. | ||
pub fn into_inner(self) -> BufWriter { | ||
self.w | ||
} | ||
} | ||
|
||
impl AsyncFileWriter for ParquetObjectWriter { | ||
fn write(&mut self, bs: Bytes) -> BoxFuture<'_, Result<()>> { | ||
Box::pin(async { | ||
self.w | ||
.put(bs) | ||
.await | ||
.map_err(|err| ParquetError::External(Box::new(err))) | ||
}) | ||
} | ||
|
||
fn complete(&mut self) -> BoxFuture<'_, Result<()>> { | ||
Box::pin(async { | ||
self.w | ||
.shutdown() | ||
.await | ||
.map_err(|err| ParquetError::External(Box::new(err))) | ||
}) | ||
} | ||
} | ||
impl From<BufWriter> for ParquetObjectWriter { | ||
fn from(w: BufWriter) -> Self { | ||
Self::from_buf_writer(w) | ||
} | ||
} | ||
#[cfg(test)] | ||
mod tests { | ||
use arrow_array::{ArrayRef, Int64Array, RecordBatch}; | ||
use object_store::memory::InMemory; | ||
use std::sync::Arc; | ||
|
||
use super::*; | ||
use crate::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; | ||
use crate::arrow::AsyncArrowWriter; | ||
|
||
#[tokio::test] | ||
async fn test_async_writer() { | ||
let store = Arc::new(InMemory::new()); | ||
|
||
let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef; | ||
let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap(); | ||
|
||
let object_store_writer = ParquetObjectWriter::new(store.clone(), Path::from("test")); | ||
let mut writer = | ||
AsyncArrowWriter::try_new(object_store_writer, to_write.schema(), None).unwrap(); | ||
writer.write(&to_write).await.unwrap(); | ||
writer.close().await.unwrap(); | ||
|
||
let buffer = store | ||
.get(&Path::from("test")) | ||
.await | ||
.unwrap() | ||
.bytes() | ||
.await | ||
.unwrap(); | ||
let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer) | ||
.unwrap() | ||
.build() | ||
.unwrap(); | ||
let read = reader.next().unwrap().unwrap(); | ||
|
||
assert_eq!(to_write, read); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.