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 CDC generation for merge operations #2747

Merged
merged 3 commits into from
Aug 12, 2024
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
2 changes: 1 addition & 1 deletion crates/core/src/delta_datafusion/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ use crate::table::state::DeltaTableState;
use crate::table::Constraint;
use crate::{open_table, open_table_with_storage_options, DeltaTable};

const PATH_COLUMN: &str = "__delta_rs_path";
pub(crate) const PATH_COLUMN: &str = "__delta_rs_path";

pub mod cdf;
pub mod expr;
Expand Down
5 changes: 4 additions & 1 deletion crates/core/src/operations/cdc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,8 @@ use crate::DeltaResult;
use datafusion::prelude::*;
use datafusion_common::ScalarValue;

pub const CDC_COLUMN_NAME: &str = "_change_type";

/// The CDCTracker is useful for hooking reads/writes in a manner nececessary to create CDC files
/// associated with commits
pub(crate) struct CDCTracker {
Expand Down Expand Up @@ -88,8 +90,9 @@ mod tests {
use crate::operations::DeltaOps;
use crate::{DeltaConfigKey, DeltaTable};
use arrow::array::{ArrayRef, Int32Array, StructArray};
use arrow::datatypes::{DataType, Field};
use arrow_array::RecordBatch;
use arrow_schema::{DataType, Field, Schema};
use arrow_schema::Schema;
use datafusion::assert_batches_sorted_eq;
use datafusion::datasource::{MemTable, TableProvider};

Expand Down
3 changes: 1 addition & 2 deletions crates/core/src/operations/delete.rs
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ use serde::Serialize;
use super::cdc::should_write_cdc;
use super::datafusion_utils::Expression;
use super::transaction::{CommitBuilder, CommitProperties, PROTOCOL};
use super::write::WriterStatsConfig;

use crate::delta_datafusion::expr::fmt_expr_to_sql;
use crate::delta_datafusion::{
Expand All @@ -52,7 +51,7 @@ use crate::delta_datafusion::{
};
use crate::errors::DeltaResult;
use crate::kernel::{Action, Add, Remove};
use crate::operations::write::{write_execution_plan, write_execution_plan_cdc};
use crate::operations::write::{write_execution_plan, write_execution_plan_cdc, WriterStatsConfig};
use crate::protocol::DeltaOperation;
use crate::table::state::DeltaTableState;
use crate::{DeltaTable, DeltaTableError};
Expand Down
34 changes: 16 additions & 18 deletions crates/core/src/operations/load_cdf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ use datafusion_physical_expr::{
use std::sync::Arc;
use std::time::SystemTime;

use arrow::record_batch::RecordBatch;
use arrow_schema::{ArrowError, Field};
use chrono::{DateTime, Utc};
use datafusion::datasource::file_format::parquet::ParquetFormat;
Expand Down Expand Up @@ -329,37 +330,34 @@ impl CdfLoadBuilder {
}
}

/// Helper function to collect batches associated with reading CDF data
pub(crate) async fn collect_batches(
num_partitions: usize,
stream: DeltaCdfScan,
ctx: SessionContext,
) -> Result<Vec<RecordBatch>, Box<dyn std::error::Error>> {
let mut batches = vec![];
for p in 0..num_partitions {
let data: Vec<RecordBatch> =
crate::operations::collect_sendable_stream(stream.execute(p, ctx.task_ctx())?).await?;
batches.extend_from_slice(&data);
}
Ok(batches)
}

#[cfg(test)]
mod tests {
use super::*;
use std::error::Error;
use std::str::FromStr;

use arrow_array::RecordBatch;
use chrono::NaiveDateTime;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::prelude::SessionContext;
use datafusion_common::assert_batches_sorted_eq;

use crate::delta_datafusion::cdf::DeltaCdfScan;
use crate::operations::collect_sendable_stream;
use crate::writer::test_utils::TestResult;
use crate::DeltaOps;

async fn collect_batches(
num_partitions: usize,
stream: DeltaCdfScan,
ctx: SessionContext,
) -> Result<Vec<RecordBatch>, Box<dyn Error>> {
let mut batches = vec![];
for p in 0..num_partitions {
let data: Vec<RecordBatch> =
collect_sendable_stream(stream.execute(p, ctx.task_ctx())?).await?;
batches.extend_from_slice(&data);
}
Ok(batches)
}

#[tokio::test]
async fn test_load_local() -> TestResult {
let ctx = SessionContext::new();
Expand Down
Loading
Loading