Skip to content
Closed
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
13 changes: 7 additions & 6 deletions datafusion/core/src/datasource/datasource.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ use crate::execution::context::SessionState;
use crate::logical_expr::Expr;
use crate::physical_plan::ExecutionPlan;

use super::sink::DataSink;

/// Source table
#[async_trait]
pub trait TableProvider: Sync + Send {
Expand Down Expand Up @@ -98,12 +100,11 @@ pub trait TableProvider: Sync + Send {
None
}

/// Insert into this table
async fn insert_into(
&self,
_state: &SessionState,
_input: Arc<dyn ExecutionPlan>,
) -> Result<Arc<dyn ExecutionPlan>> {
/// Return a [`DataSink`] suitable for writing to this table
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Here is the new interface -- I think this probably should also get some sort of "options" parameter too so that we can provide a way to pass in settings like row_group_size from the various inserts).

I prefer to add that parameter at some future point when I have an actual usecase with COPY rather than guessing exactly what is needed.

///
/// Each insert or other DML plan will call this function. Each
/// returned value can be unique.
async fn write_to(&self) -> Result<Arc<dyn DataSink>> {
let msg = "Insertion not implemented for this table".to_owned();
Err(DataFusionError::NotImplemented(msg))
}
Expand Down
129 changes: 85 additions & 44 deletions datafusion/core/src/datasource/memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

use futures::StreamExt;
use std::any::Any;
use std::fmt::Debug;
use std::sync::Arc;

use arrow::datatypes::SchemaRef;
Expand All @@ -30,13 +31,14 @@ use crate::datasource::{TableProvider, TableType};
use crate::error::{DataFusionError, Result};
use crate::execution::context::SessionState;
use crate::logical_expr::Expr;
use crate::physical_plan::common;
use crate::physical_plan::common::AbortOnDropSingle;
use crate::physical_plan::memory::MemoryExec;
use crate::physical_plan::memory::MemoryWriteExec;
use crate::physical_plan::ExecutionPlan;
use crate::physical_plan::{common, SendableRecordBatchStream};
use crate::physical_plan::{repartition::RepartitionExec, Partitioning};

use super::sink::DataSink;

/// Type alias for partition data
pub type PartitionData = Arc<RwLock<Vec<RecordBatch>>>;

Expand Down Expand Up @@ -164,50 +166,58 @@ impl TableProvider for MemTable {
)?))
}

/// Inserts the execution results of a given [`ExecutionPlan`] into this [`MemTable`].
/// The [`ExecutionPlan`] must have the same schema as this [`MemTable`].
///
/// # Arguments
///
/// * `state` - The [`SessionState`] containing the context for executing the plan.
/// * `input` - The [`ExecutionPlan`] to execute and insert.
///
/// # Returns
///
/// * A `Result` indicating success or failure.
async fn insert_into(
&self,
_state: &SessionState,
input: Arc<dyn ExecutionPlan>,
) -> Result<Arc<dyn ExecutionPlan>> {
// Create a physical plan from the logical plan.
// Check that the schema of the plan matches the schema of this table.
if !input.schema().eq(&self.schema) {
return Err(DataFusionError::Plan(
"Inserting query must have the same schema with the table.".to_string(),
async fn write_to(&self) -> Result<Arc<dyn DataSink>> {
if self.batches.is_empty() {
return Err(DataFusionError::Internal(
"Can not insert into table without partitions.".to_string(),
));
}
Ok(Arc::new(MemSink::new(self.batches.clone())))
}
}

if self.batches.is_empty() {
return Err(DataFusionError::Plan(
"The table must have partitions.".to_string(),
));
/// Implements for writing to a [`MemTable`]
struct MemSink {
/// Target locations for writing data
batches: Vec<PartitionData>,
}

impl Debug for MemSink {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("MemSink")
.field("num_partitions", &self.batches.len())
.finish()
}
}

impl MemSink {
fn new(batches: Vec<PartitionData>) -> Self {
Self { batches }
}
}

#[async_trait]
impl DataSink for MemSink {
async fn write_all(&self, mut data: SendableRecordBatchStream) -> Result<u64> {
let num_partitions = self.batches.len();

// buffer up the data round robin stle into num_partitions new buffers
let mut new_batches = vec![vec![]; num_partitions];
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I am quite pleased that by following @tustvold 's guidance and pushing the partitioning choice into the DataSink implementations, that the logic becomes quite a bit simpler and flexible with seemingly no performance penalty

let mut i = 0;
let mut row_count = 0;
while let Some(batch) = data.next().await.transpose()? {
row_count += batch.num_rows();
new_batches[i].push(batch);
i = (i + 1) % num_partitions;
}

let input = if self.batches.len() > 1 {
Arc::new(RepartitionExec::try_new(
input,
Partitioning::RoundRobinBatch(self.batches.len()),
)?)
} else {
input
};
// write the outputs into
for (target, mut batches) in self.batches.iter().zip(new_batches.into_iter()) {
// Append all the new batches in one go to minimize locking overhead
target.write().await.append(&mut batches);
}

Ok(Arc::new(MemoryWriteExec::try_new(
input,
self.batches.clone(),
self.schema.clone(),
)?))
Ok(row_count as u64)
}
}

Expand All @@ -218,8 +228,8 @@ mod tests {
use crate::from_slice::FromSlice;
use crate::physical_plan::collect;
use crate::prelude::SessionContext;
use arrow::array::Int32Array;
use arrow::datatypes::{DataType, Field, Schema};
use arrow::array::{AsArray, Int32Array};
use arrow::datatypes::{DataType, Field, Schema, UInt64Type};
use arrow::error::ArrowError;
use datafusion_expr::LogicalPlanBuilder;
use futures::StreamExt;
Expand Down Expand Up @@ -457,6 +467,11 @@ mod tests {
initial_data: Vec<Vec<RecordBatch>>,
inserted_data: Vec<Vec<RecordBatch>>,
) -> Result<Vec<Vec<RecordBatch>>> {
let expected_count: u64 = inserted_data
.iter()
.flat_map(|batches| batches.iter().map(|batch| batch.num_rows() as u64))
.sum();

// Create a new session context
let session_ctx = SessionContext::new();
// Create and register the initial table with the provided schema and data
Expand All @@ -480,8 +495,8 @@ mod tests {

// Execute the physical plan and collect the results
let res = collect(plan, session_ctx.task_ctx()).await?;
// Ensure the result is empty after the insert operation
assert!(res.is_empty());
assert_eq!(extract_count(res), expected_count);

// Read the data from the initial table and store it in a vector of partitions
let mut partitions = vec![];
for partition in initial_table.batches.iter() {
Expand All @@ -491,6 +506,32 @@ mod tests {
Ok(partitions)
}

/// Returns the value of results:
///
/// "+-------+",
/// "| count |",
/// "+-------+",
/// "| 6 |",
/// "+-------+",
fn extract_count(res: Vec<RecordBatch>) -> u64 {
assert_eq!(res.len(), 1, "expected one batch, got {}", res.len());
let batch = &res[0];
assert_eq!(
batch.num_columns(),
1,
"expected 1 column, got {}",
batch.num_columns()
);
let col = batch.column(0).as_primitive::<UInt64Type>();
assert_eq!(col.len(), 1, "expected 1 row, got {}", col.len());
let val = col
.iter()
.next()
.expect("had value")
.expect("expected non null");
val
}

// Test inserting a single batch of data into a single partition
#[tokio::test]
async fn test_insert_into_single_partition() -> Result<()> {
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/src/datasource/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ pub mod file_format;
pub mod listing;
pub mod listing_table_factory;
pub mod memory;
pub mod sink;
pub mod streaming;
pub mod view;

Expand Down
37 changes: 37 additions & 0 deletions datafusion/core/src/datasource/sink.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// 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.

//! Data sink traits
use async_trait::async_trait;
use datafusion_common::Result;

use crate::physical_plan::SendableRecordBatchStream;

/// `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 rows 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>;
}
Loading