alamb commented on code in PR #6354: URL: https://github.com/apache/arrow-datafusion/pull/6354#discussion_r1194282214
########## datafusion/core/src/physical_plan/insert.rs: ########## @@ -0,0 +1,203 @@ +// 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. + +//! Execution plan for writing data to [`DataSink`]s + +use super::expressions::PhysicalSortExpr; +use super::{ + DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, +}; +use crate::error::Result; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use arrow_array::{ArrayRef, UInt64Array}; +use arrow_schema::{DataType, Field, Schema}; +use async_trait::async_trait; +use core::fmt; +use futures::StreamExt; +use std::any::Any; +use std::sync::Arc; + +use crate::execution::context::TaskContext; +use crate::physical_plan::stream::RecordBatchStreamAdapter; +use crate::physical_plan::Distribution; +use datafusion_common::DataFusionError; + +/// `DataSink` implements writing streams of [`RecordBatch`]es to +/// destinations. +#[async_trait] +pub trait DataSink: std::fmt::Debug + Send + Sync { + // TODO add desired input ordering + // How does this sink want its input ordered? + + /// Writes the data to the sink, returns the number of values written + /// + /// This method will be called exactly once during each DML + /// statement. Thus prior to return, the sink should do any commit + /// or rollback required. + async fn write_all(&self, data: SendableRecordBatchStream) -> Result<u64>; +} + +/// Execution plan for writing record batches to a [`DataSink`] +/// +/// Returns a single row with the number of values written +pub struct InsertExec { + /// Input plan that produces the record batches to be written. + input: Arc<dyn ExecutionPlan>, + /// Sink to whic to write + sink: Arc<dyn DataSink>, + /// Schema describing the structure of the data. + schema: SchemaRef, +} + +impl fmt::Debug for InsertExec { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "InsertExec schema: {:?}", self.schema) + } +} + +impl InsertExec { + /// Create a plan to write to `sink` + pub fn new(input: Arc<dyn ExecutionPlan>, sink: Arc<dyn DataSink>) -> Self { + Self { + input, + sink, + schema: make_count_schema(), + } + } +} + +impl ExecutionPlan for InsertExec { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + /// Get the schema for this execution plan + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(1) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn required_input_distribution(&self) -> Vec<Distribution> { + vec![Distribution::SinglePartition] Review Comment: But to be clear since the API is in terms of `ExecutionPlan` if someone wanted to control the partitioning more they could do so (by defining their own ExecutionPlan) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
