fix: abort parquet writer (#1785)

* fix: sst file size

* fix: avoid creating file when no row's been written

* chore: rename tests

* fix: some clippy issues

* fix: some cr comments
This commit is contained in:
Lei, HUANG
2023-06-19 11:19:31 +08:00
committed by GitHub
parent 69854c07c5
commit 960b84262b
4 changed files with 151 additions and 80 deletions

View File

@@ -12,24 +12,26 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::future::Future;
use arrow::record_batch::RecordBatch;
use async_trait::async_trait;
use datafusion::parquet::format::FileMetaData;
use object_store::Writer;
use snafu::{OptionExt, ResultExt};
use tokio::io::{AsyncWrite, AsyncWriteExt};
use tokio_util::compat::Compat;
use crate::error::{self, Result};
use crate::share_buffer::SharedBuffer;
pub struct BufferedWriter<T, U> {
writer: T,
/// None stands for [`BufferedWriter`] closed.
pub struct LazyBufferedWriter<T, U, F> {
path: String,
writer_factory: F,
writer: Option<T>,
/// None stands for [`LazyBufferedWriter`] closed.
encoder: Option<U>,
buffer: SharedBuffer,
rows_written: usize,
bytes_written: u64,
flushed: bool,
threshold: usize,
}
@@ -42,58 +44,79 @@ pub trait ArrowWriterCloser {
async fn close(mut self) -> Result<FileMetaData>;
}
pub type DefaultBufferedWriter<E> = BufferedWriter<Compat<Writer>, E>;
impl<T: AsyncWrite + Send + Unpin, U: DfRecordBatchEncoder + ArrowWriterCloser>
BufferedWriter<T, U>
impl<
T: AsyncWrite + Send + Unpin,
U: DfRecordBatchEncoder + ArrowWriterCloser,
F: FnMut(String) -> Fut,
Fut: Future<Output = Result<T>>,
> LazyBufferedWriter<T, U, F>
{
/// Closes `LazyBufferedWriter` and optionally flushes all data to underlying storage
/// if any row's been written.
pub async fn close_with_arrow_writer(mut self) -> Result<(FileMetaData, u64)> {
let encoder = self
.encoder
.take()
.context(error::BufferedWriterClosedSnafu)?;
let metadata = encoder.close().await?;
let written = self.try_flush(true).await?;
// Use `rows_written` to keep a track of if any rows have been written.
// If no row's been written, then we can simply close the underlying
// writer without flush so that no file will be actually created.
if self.rows_written != 0 {
self.bytes_written += self.try_flush(true).await?;
}
// It's important to shut down! flushes all pending writes
self.close().await?;
Ok((metadata, written))
self.close_inner_writer().await?;
Ok((metadata, self.bytes_written))
}
}
impl<T: AsyncWrite + Send + Unpin, U: DfRecordBatchEncoder> BufferedWriter<T, U> {
pub async fn close(&mut self) -> Result<()> {
self.writer.shutdown().await.context(error::AsyncWriteSnafu)
impl<
T: AsyncWrite + Send + Unpin,
U: DfRecordBatchEncoder,
F: FnMut(String) -> Fut,
Fut: Future<Output = Result<T>>,
> LazyBufferedWriter<T, U, F>
{
/// Closes the writer without flushing the buffer data.
pub async fn close_inner_writer(&mut self) -> Result<()> {
if let Some(writer) = &mut self.writer {
writer.shutdown().await.context(error::AsyncWriteSnafu)?;
}
Ok(())
}
pub fn new(threshold: usize, buffer: SharedBuffer, encoder: U, writer: T) -> Self {
pub fn new(
threshold: usize,
buffer: SharedBuffer,
encoder: U,
path: impl AsRef<str>,
writer_factory: F,
) -> Self {
Self {
path: path.as_ref().to_string(),
threshold,
writer,
encoder: Some(encoder),
buffer,
rows_written: 0,
bytes_written: 0,
flushed: false,
writer_factory,
writer: None,
}
}
pub fn bytes_written(&self) -> u64 {
self.bytes_written
}
pub async fn write(&mut self, batch: &RecordBatch) -> Result<()> {
let encoder = self
.encoder
.as_mut()
.context(error::BufferedWriterClosedSnafu)?;
encoder.write(batch)?;
self.try_flush(false).await?;
self.rows_written += batch.num_rows();
self.bytes_written += self.try_flush(false).await?;
Ok(())
}
pub fn flushed(&self) -> bool {
self.flushed
}
pub async fn try_flush(&mut self, all: bool) -> Result<u64> {
let mut bytes_written: u64 = 0;
@@ -106,7 +129,8 @@ impl<T: AsyncWrite + Send + Unpin, U: DfRecordBatchEncoder> BufferedWriter<T, U>
};
let size = chunk.len();
self.writer
self.maybe_init_writer()
.await?
.write_all(&chunk)
.await
.context(error::AsyncWriteSnafu)?;
@@ -117,22 +141,27 @@ impl<T: AsyncWrite + Send + Unpin, U: DfRecordBatchEncoder> BufferedWriter<T, U>
if all {
bytes_written += self.try_flush_all().await?;
}
self.flushed = bytes_written > 0;
self.bytes_written += bytes_written;
Ok(bytes_written)
}
/// Only initiates underlying file writer when rows have been written.
async fn maybe_init_writer(&mut self) -> Result<&mut T> {
if let Some(ref mut writer) = self.writer {
Ok(writer)
} else {
let writer = (self.writer_factory)(self.path.clone()).await?;
Ok(self.writer.insert(writer))
}
}
async fn try_flush_all(&mut self) -> Result<u64> {
let remain = self.buffer.buffer.lock().unwrap().split();
let size = remain.len();
self.writer
self.maybe_init_writer()
.await?
.write_all(&remain)
.await
.context(error::AsyncWriteSnafu)?;
Ok(size as u64)
}
}

View File

@@ -35,12 +35,11 @@ use datafusion::physical_plan::SendableRecordBatchStream;
use futures::StreamExt;
use object_store::ObjectStore;
use snafu::ResultExt;
use tokio_util::compat::FuturesAsyncWriteCompatExt;
use self::csv::CsvFormat;
use self::json::JsonFormat;
use self::parquet::ParquetFormat;
use crate::buffered_writer::{BufferedWriter, DfRecordBatchEncoder};
use crate::buffered_writer::{DfRecordBatchEncoder, LazyBufferedWriter};
use crate::compression::CompressionType;
use crate::error::{self, Result};
use crate::share_buffer::SharedBuffer;
@@ -181,15 +180,14 @@ pub async fn stream_to_file<T: DfRecordBatchEncoder, U: Fn(SharedBuffer) -> T>(
threshold: usize,
encoder_factory: U,
) -> Result<usize> {
let writer = store
.writer(path)
.await
.context(error::WriteObjectSnafu { path })?
.compat_write();
let buffer = SharedBuffer::with_capacity(threshold);
let encoder = encoder_factory(buffer.clone());
let mut writer = BufferedWriter::new(threshold, buffer, encoder, writer);
let mut writer = LazyBufferedWriter::new(threshold, buffer, encoder, path, |path| async {
store
.writer(&path)
.await
.context(error::WriteObjectSnafu { path })
});
let mut rows = 0;
@@ -201,8 +199,7 @@ pub async fn stream_to_file<T: DfRecordBatchEncoder, U: Fn(SharedBuffer) -> T>(
// Flushes all pending writes
writer.try_flush(true).await?;
writer.close().await?;
writer.close_inner_writer().await?;
Ok(rows)
}