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

feat: introduce opendal AsyncWriter for parquet integrations #4958

Merged
merged 10 commits into from
Aug 5, 2024
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions integrations/parquet/.gitignore
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
Cargo.lock
56 changes: 56 additions & 0 deletions integrations/parquet/Cargo.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
# 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.

[package]
description = "parquet Integration for Apache OpenDAL"
name = "parquet_opendal"

authors = ["Apache OpenDAL <[email protected]>"]
edition = "2021"
homepage = "https://opendal.apache.org/"
license = "Apache-2.0"
repository = "https://github.com/apache/opendal"
rust-version = "1.75"
version = "0.0.1"

[features]
default = ["arrow"]
WenyXu marked this conversation as resolved.
Show resolved Hide resolved
arrow = ["dep:arrow"]

[dependencies]
async-trait = "0.1"
bytes = "1"
futures = "0.3"
opendal = { version = "0.48.0", path = "../../core" }
parquet = { version = "52.0", default-features = false, features = [
"async",
"arrow",
] }
arrow = { version = "52.0", optional = true }
WenyXu marked this conversation as resolved.
Show resolved Hide resolved

[dev-dependencies]
opendal = { version = "0.48.0", path = "../../core", features = [
"services-memory",
"services-s3",
] }
rand = "0.8.5"
tokio = { version = "1", features = ["fs", "macros", "rt-multi-thread"] }

[[example]]
name = "async_writer"
path = "examples/async_writer.rs"
required-features = ["arrow"]
45 changes: 45 additions & 0 deletions integrations/parquet/examples/async_writer.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
use std::sync::Arc;

use arrow::array::{ArrayRef, Int64Array, RecordBatch};

use opendal::{services::S3Config, Operator};
use parquet::arrow::{arrow_reader::ParquetRecordBatchReaderBuilder, AsyncArrowWriter};
use parquet_opendal::AsyncWriter;

#[tokio::main]
async fn main() {
let mut cfg = S3Config::default();
cfg.access_key_id = Some("my_access_key".to_string());
cfg.secret_access_key = Some("my_secret_key".to_string());
cfg.endpoint = Some("my_endpoint".to_string());
cfg.region = Some("my_region".to_string());
cfg.bucket = "my_bucket".to_string();

// Create a new operator
let operator = Operator::from_config(cfg).unwrap().finish();
let path = "/path/to/file.parquet";

// Create an async writer
let writer = AsyncWriter::new(
operator
.writer_with(path)
.chunk(32 * 1024 * 1024)
.concurrent(8)
.await
.unwrap(),
);

let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
let mut writer = AsyncArrowWriter::try_new(writer, to_write.schema(), None).unwrap();
writer.write(&to_write).await.unwrap();
writer.close().await.unwrap();

let buffer = operator.read(path).await.unwrap().to_bytes();
let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer)
.unwrap()
.build()
.unwrap();
let read = reader.next().unwrap().unwrap();
assert_eq!(to_write, read);
}
139 changes: 139 additions & 0 deletions integrations/parquet/src/async_writer.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,139 @@
// 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 parquet::arrow::async_writer::AsyncFileWriter;
use parquet::errors::{ParquetError, Result};

use futures::future::BoxFuture;
use opendal::Writer;

/// AsyncWriter implements AsyncFileWriter trait by using opendal.
///
/// ```no_run
/// use std::sync::Arc;
///
/// use arrow::array::{ArrayRef, Int64Array, RecordBatch};
///
/// use opendal::{services::S3Config, Operator};
/// use parquet::arrow::{arrow_reader::ParquetRecordBatchReaderBuilder, AsyncArrowWriter};
/// use parquet_opendal::AsyncWriter;
///
/// #[tokio::main]
/// async fn main() {
/// let mut cfg = S3Config::default();
/// cfg.access_key_id = Some("my_access_key".to_string());
/// cfg.secret_access_key = Some("my_secret_key".to_string());
/// cfg.endpoint = Some("my_endpoint".to_string());
/// cfg.region = Some("my_region".to_string());
/// cfg.bucket = "my_bucket".to_string();
///
/// // Create a new operator
/// let operator = Operator::from_config(cfg).unwrap().finish();
/// let path = "/path/to/file.parquet";
///
/// // Create an async writer
/// let writer = AsyncWriter::new(
/// operator
/// .writer_with(path)
/// .chunk(32 * 1024 * 1024)
/// .concurrent(8)
/// .await
/// .unwrap(),
/// );
///
/// let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
/// let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
/// let mut writer = AsyncArrowWriter::try_new(writer, to_write.schema(), None).unwrap();
/// writer.write(&to_write).await.unwrap();
/// writer.close().await.unwrap();
///
/// let buffer = operator.read(path).await.unwrap().to_bytes();
/// let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer)
/// .unwrap()
/// .build()
/// .unwrap();
/// let read = reader.next().unwrap().unwrap();
/// assert_eq!(to_write, read);
/// }
/// ```
WenyXu marked this conversation as resolved.
Show resolved Hide resolved
pub struct AsyncWriter {
inner: Writer,
}

impl AsyncWriter {
/// Create a [`OpendalAsyncWriter`] by given [`Writer`].
pub fn new(writer: Writer) -> Self {
Self { inner: writer }
}
}

impl AsyncFileWriter for AsyncWriter {
fn write(&mut self, bs: Bytes) -> BoxFuture<'_, Result<()>> {
Box::pin(async move {
self.inner
.write(bs)
.await
.map_err(|err| ParquetError::External(Box::new(err)))
})
}

fn complete(&mut self) -> BoxFuture<'_, Result<()>> {
Box::pin(async move {
self.inner
.close()
.await
.map_err(|err| ParquetError::External(Box::new(err)))
})
}
}

#[cfg(test)]
mod tests {
use super::*;
use opendal::{services, Operator};

#[tokio::test]
async fn test_basic() {
let op = Operator::new(services::Memory::default()).unwrap().finish();
let path = "data/test.txt";
let mut writer = AsyncWriter::new(op.writer(path).await.unwrap());
let bytes = Bytes::from_static(b"hello, world!");
writer.write(bytes).await.unwrap();
let bytes = Bytes::from_static(b"hello, OpenDAL!");
writer.write(bytes).await.unwrap();
writer.complete().await.unwrap();

let bytes = op.read(path).await.unwrap().to_vec();
assert_eq!(bytes, b"hello, world!hello, OpenDAL!");
}

#[tokio::test]
async fn test_abort() {
let op = Operator::new(services::Memory::default()).unwrap().finish();
let path = "data/test.txt";
let mut writer = AsyncWriter::new(op.writer(path).await.unwrap());
let bytes = Bytes::from_static(b"hello, world!");
writer.write(bytes).await.unwrap();
let bytes = Bytes::from_static(b"hello, OpenDAL!");
writer.write(bytes).await.unwrap();
drop(writer);

let exist = op.is_exist(path).await.unwrap();
assert!(!exist);
}
}
72 changes: 72 additions & 0 deletions integrations/parquet/src/lib.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
// 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.

//! parquet_opendal provides parquet IO utils.
//!
//! AsyncWriter implements AsyncFileWriter trait by using opendal.
//!
//! ```no_run
//! use std::sync::Arc;
//!
//! use arrow::array::{ArrayRef, Int64Array, RecordBatch};
//!
//! use opendal::{services::S3Config, Operator};
//! use parquet::arrow::{arrow_reader::ParquetRecordBatchReaderBuilder, AsyncArrowWriter};
//! use parquet_opendal::AsyncWriter;
//!
//! #[tokio::main]
//! async fn main() {
//! let mut cfg = S3Config::default();
//! cfg.access_key_id = Some("my_access_key".to_string());
//! cfg.secret_access_key = Some("my_secret_key".to_string());
//! cfg.endpoint = Some("my_endpoint".to_string());
//! cfg.region = Some("my_region".to_string());
//! cfg.bucket = "my_bucket".to_string();
//!
//! // Create a new operator
//! let operator = Operator::from_config(cfg).unwrap().finish();
//! let path = "/path/to/file.parquet";
//!
//! // Create an async writer
//! let writer = AsyncWriter::new(
//! operator
//! .writer_with(path)
//! .chunk(32 * 1024 * 1024)
//! .concurrent(8)
//! .await
//! .unwrap(),
//! );
//!
//! let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
//! let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
//! let mut writer = AsyncArrowWriter::try_new(writer, to_write.schema(), None).unwrap();
//! writer.write(&to_write).await.unwrap();
//! writer.close().await.unwrap();
//!
//! let buffer = operator.read(path).await.unwrap().to_bytes();
//! let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer)
//! .unwrap()
//! .build()
//! .unwrap();
//! let read = reader.next().unwrap().unwrap();
//! assert_eq!(to_write, read);
//! }
//! ```

mod async_writer;

pub use async_writer::AsyncWriter;
Loading