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

DataSink additions #7778

Merged
merged 8 commits into from
Oct 10, 2023
Merged
Show file tree
Hide file tree
Changes from all 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
9 changes: 9 additions & 0 deletions datafusion/core/src/datasource/file_format/csv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};

use async_trait::async_trait;
use bytes::{Buf, Bytes};
use datafusion_physical_plan::metrics::MetricsSet;
use futures::stream::BoxStream;
use futures::{pin_mut, Stream, StreamExt, TryStreamExt};
use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore};
Expand Down Expand Up @@ -484,6 +485,14 @@ impl CsvSink {

#[async_trait]
impl DataSink for CsvSink {
fn as_any(&self) -> &dyn Any {
self
}

fn metrics(&self) -> Option<MetricsSet> {
None
}

async fn write_all(
&self,
data: Vec<SendableRecordBatchStream>,
Expand Down
9 changes: 9 additions & 0 deletions datafusion/core/src/datasource/file_format/json.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ use datafusion_common::DataFusionError;
use datafusion_common::FileType;
use datafusion_execution::TaskContext;
use datafusion_physical_expr::PhysicalSortRequirement;
use datafusion_physical_plan::metrics::MetricsSet;
use rand::distributions::Alphanumeric;
use rand::distributions::DistString;
use std::fmt;
Expand Down Expand Up @@ -276,6 +277,14 @@ impl JsonSink {

#[async_trait]
impl DataSink for JsonSink {
fn as_any(&self) -> &dyn Any {
self
}

fn metrics(&self) -> Option<MetricsSet> {
None
}

async fn write_all(
&self,
data: Vec<SendableRecordBatchStream>,
Expand Down
9 changes: 9 additions & 0 deletions datafusion/core/src/datasource/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

//! Parquet format abstractions

use datafusion_physical_plan::metrics::MetricsSet;
use parquet::column::writer::ColumnCloseResult;
use parquet::file::writer::SerializedFileWriter;
use rand::distributions::DistString;
Expand Down Expand Up @@ -757,6 +758,14 @@ impl ParquetSink {

#[async_trait]
impl DataSink for ParquetSink {
fn as_any(&self) -> &dyn Any {
self
}

fn metrics(&self) -> Option<MetricsSet> {
None
}

async fn write_all(
&self,
mut data: Vec<SendableRecordBatchStream>,
Expand Down
9 changes: 9 additions & 0 deletions datafusion/core/src/datasource/memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

//! [`MemTable`] for querying `Vec<RecordBatch>` by DataFusion.

use datafusion_physical_plan::metrics::MetricsSet;
use futures::StreamExt;
use log::debug;
use std::any::Any;
Expand Down Expand Up @@ -259,6 +260,14 @@ impl MemSink {

#[async_trait]
impl DataSink for MemSink {
fn as_any(&self) -> &dyn Any {
self
}

fn metrics(&self) -> Option<MetricsSet> {
None
}

async fn write_all(
&self,
mut data: Vec<SendableRecordBatchStream>,
Expand Down
21 changes: 21 additions & 0 deletions datafusion/physical-plan/src/insert.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ use std::any::Any;
use std::fmt::Debug;
use std::sync::Arc;

use crate::metrics::MetricsSet;
use crate::stream::RecordBatchStreamAdapter;
use datafusion_common::{exec_err, internal_err, DataFusionError};
use datafusion_execution::TaskContext;
Expand All @@ -46,6 +47,16 @@ use datafusion_execution::TaskContext;
/// output.
#[async_trait]
pub trait DataSink: DisplayAs + Debug + Send + Sync {
/// Returns the data sink as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;

/// Return a snapshot of the [MetricsSet] for this
/// [DataSink].
///
/// See [ExecutionPlan::metrics()] for more details
fn metrics(&self) -> Option<MetricsSet>;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤔 I wonder if there is value in making some sort of End to End example for this feature (so we don't accidentally break it in the future0


// TODO add desired input ordering
// How does this sink want its input ordered?

Expand Down Expand Up @@ -151,6 +162,16 @@ impl FileSinkExec {
}
Ok(streams)
}

/// Returns insert sink
pub fn sink(&self) -> &dyn DataSink {
self.sink.as_ref()
}

/// Returns the metrics of the underlying [DataSink]
pub fn metrics(&self) -> Option<MetricsSet> {
self.sink.metrics()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ahh this was the wrong place, it should be in impl ExecutionPlan for FileSinkExec

}
}

impl DisplayAs for FileSinkExec {
Expand Down