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

refactor: combine normal and cdf plan until write for merge #3142

Merged
merged 2 commits into from
Jan 20, 2025
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
188 changes: 71 additions & 117 deletions crates/core/src/operations/merge/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ use std::fmt::Debug;
use std::sync::Arc;
use std::time::Instant;

use arrow_schema::{DataType, Field};
use async_trait::async_trait;
use datafusion::datasource::provider_as_source;
use datafusion::error::Result as DataFusionResult;
Expand All @@ -56,7 +57,6 @@ use datafusion_expr::{

use filter::try_construct_early_filter;
use futures::future::BoxFuture;
use itertools::Itertools;
use parquet::file::properties::WriterProperties;
use serde::Serialize;
use tracing::log::*;
Expand All @@ -80,14 +80,16 @@ use crate::logstore::LogStoreRef;
use crate::operations::cdc::*;
use crate::operations::merge::barrier::find_node;
use crate::operations::transaction::CommitBuilder;
use crate::operations::write::{write_execution_plan, write_execution_plan_cdc, WriterStatsConfig};
use crate::operations::write::WriterStatsConfig;
use crate::protocol::{DeltaOperation, MergePredicate};
use crate::table::state::DeltaTableState;
use crate::table::GeneratedColumn;
use crate::{DeltaResult, DeltaTable, DeltaTableError};
use writer::write_execution_plan_v2;

mod barrier;
mod filter;
mod writer;

const SOURCE_COLUMN: &str = "__delta_rs_source";
const TARGET_COLUMN: &str = "__delta_rs_target";
Expand Down Expand Up @@ -717,7 +719,6 @@ async fn execute(
// be disabled in the common case(s)
let should_cdc = should_write_cdc(&snapshot)?;
// Change data may be collected and then written out at the completion of the merge
let mut change_data = vec![];

if should_cdc {
debug!("Executing a merge and I should write CDC!");
Expand Down Expand Up @@ -788,7 +789,7 @@ async fn execute(
fn add_generated_columns(
mut df: DataFrame,
generated_cols: &Vec<GeneratedColumn>,
generated_cols_missing_in_source: &Vec<String>,
generated_cols_missing_in_source: &[String],
state: &SessionState,
) -> DeltaResult<DataFrame> {
debug!("Generating columns in dataframe");
Expand Down Expand Up @@ -1047,6 +1048,7 @@ async fn execute(

let mut new_columns = vec![];
let mut write_projection = Vec::new();
let mut write_projection_with_cdf = Vec::new();

for delta_field in snapshot.schema().fields() {
let mut when_expr = Vec::with_capacity(operations_size);
Expand Down Expand Up @@ -1084,9 +1086,26 @@ async fn execute(
Expr::Column(Column::from_name(name.clone())).alias(delta_field.name()),
delta_field.data_type().try_into()?,
));

write_projection_with_cdf.push(
when(
col(CDC_COLUMN_NAME).not_eq(lit("update_preimage")),
cast(
Expr::Column(Column::from_name(name.clone())),
delta_field.data_type().try_into()?,
),
)
.otherwise(cast(
Expr::Column(Column::new(Some(target_name.clone()), delta_field.name())), // We take the column from target table
delta_field.data_type().try_into()?,
))?
.alias(delta_field.name()),
);
new_columns.push((name, case));
}

write_projection_with_cdf.push(col("_change_type"));

let mut insert_when = Vec::with_capacity(ops.len());
let mut insert_then = Vec::with_capacity(ops.len());

Expand Down Expand Up @@ -1205,6 +1224,8 @@ async fn execute(
}),
});

// We should observe the metrics before we union the merge plan with the cdf_merge plan
// so that we get the metrics only for the merge plan.
let operation_count = LogicalPlan::Extension(Extension {
node: Arc::new(MetricObserver {
id: OUTPUT_COUNT_ID.into(),
Expand All @@ -1215,99 +1236,59 @@ async fn execute(

let operation_count = DataFrame::new(state.clone(), operation_count);

if should_cdc {
// Create a dataframe containing the CDC deletes which are present at this point
change_data.push(
operation_count
.clone()
.filter(col(DELETE_COLUMN))?
.select(write_projection.clone())?
.with_column(crate::operations::cdc::CDC_COLUMN_NAME, lit("delete"))?,
);
}

let filtered = operation_count.filter(col(DELETE_COLUMN).is_false())?;

if should_cdc {
debug!("The merge should triggere a CDC tracking, computing pre/insert/postimage datasets");
let cdc_projection = filtered.clone().filter(col(OPERATION_COLUMN).not_eq(
// This is a copy operation, but I'm not sure how to turn that enum into an int
lit(5),
))?;

let mut cdc_insert_df = cdc_projection
let mut projected = if should_cdc {
operation_count
.clone()
.filter(
col(SOURCE_COLUMN)
.is_true()
.and(col(TARGET_COLUMN).is_null()),
.with_column(
CDC_COLUMN_NAME,
when(col(TARGET_DELETE_COLUMN).is_null(), lit("delete")) // nulls are equal to True
.when(col(DELETE_COLUMN).is_null(), lit("source_delete"))
.when(col(TARGET_COPY_COLUMN).is_null(), lit("copy"))
.when(col(TARGET_INSERT_COLUMN).is_null(), lit("insert"))
.when(col(TARGET_UPDATE_COLUMN).is_null(), lit("update"))
.end()?,
)?
.select(write_projection.clone())?
.with_column(CDC_COLUMN_NAME, lit("insert"))?;

cdc_insert_df = add_generated_columns(
cdc_insert_df,
&generated_col_expressions,
&missing_generated_columns,
&state,
)?;

change_data.push(cdc_insert_df);

let mut after = cdc_projection
.clone()
.filter(col(TARGET_COLUMN).is_true())?
.select(write_projection.clone())?;

after = add_generated_columns(
after,
&generated_col_expressions,
&missing_generated_columns,
&state,
)?;

// Extra select_columns is required so that before and after have same schema order
// DataFusion doesn't have UnionByName yet, see https://github.com/apache/datafusion/issues/12650
let mut before = cdc_projection
.clone()
.filter(col(crate::delta_datafusion::PATH_COLUMN).is_not_null())?
.select(
target_schema
.columns()
.iter()
.filter(|c| c.name != crate::delta_datafusion::PATH_COLUMN)
.map(|c| Expr::Column(c.clone()))
.collect_vec(),
.drop_columns(&["__delta_rs_path"])? // WEIRD bug caused by interaction with unnest_columns, has to be dropped otherwise throws schema error
.with_column(
"__delta_rs_update_expanded",
when(
col(CDC_COLUMN_NAME).eq(lit("update")),
lit(ScalarValue::List(ScalarValue::new_list(
&[
ScalarValue::Utf8(Some("update_preimage".into())),
ScalarValue::Utf8(Some("update_postimage".into())),
],
&DataType::List(Field::new("element", DataType::Utf8, false).into()),
true,
))),
)
.end()?,
)?
.select_columns(
&after
.schema()
.columns()
.iter()
.map(|v| v.name())
.collect::<Vec<_>>(),
)?;

before = add_generated_columns(
before,
&generated_col_expressions,
&missing_generated_columns,
&state,
)?;

let tracker = CDCTracker::new(before, after);
change_data.push(tracker.collect()?);
}
.unnest_columns(&["__delta_rs_update_expanded"])?
.with_column(
CDC_COLUMN_NAME,
when(
col(CDC_COLUMN_NAME).eq(lit("update")),
col("__delta_rs_update_expanded"),
)
.otherwise(col(CDC_COLUMN_NAME))?,
)?
.drop_columns(&["__delta_rs_update_expanded"])?
.select(write_projection_with_cdf)?
} else {
operation_count
.filter(col(DELETE_COLUMN).is_false())?
.select(write_projection)?
};

let mut project = filtered.clone().select(write_projection)?;
project = add_generated_columns(
project,
projected = add_generated_columns(
projected,
&generated_col_expressions,
&missing_generated_columns,
&state,
)?;

let merge_final = &project.into_unoptimized_plan();
let merge_final = &projected.into_unoptimized_plan();
let write = state.create_physical_plan(merge_final).await?;

let err = || DeltaTableError::Generic("Unable to locate expected metric node".into());
Expand All @@ -1327,7 +1308,7 @@ async fn execute(
);

let rewrite_start = Instant::now();
let mut add_actions = write_execution_plan(
let mut actions: Vec<Action> = write_execution_plan_v2(
Some(&snapshot),
state.clone(),
write,
Expand All @@ -1338,38 +1319,11 @@ async fn execute(
writer_properties.clone(),
writer_stats_config.clone(),
None,
should_cdc, // if true, write execution plan splits batches in [normal, cdc] data before writing
)
.await?;

if should_cdc && !change_data.is_empty() {
let mut df = change_data
.pop()
.expect("change_data should never be empty");
// Accumulate all the changes together into a single data frame to produce the necessary
// change data files
for change in change_data {
df = df.union(change)?;
}
add_actions.extend(
write_execution_plan_cdc(
Some(&snapshot),
state.clone(),
df.create_physical_plan().await?,
table_partition_cols.clone(),
log_store.object_store(Some(operation_id)),
Some(snapshot.table_config().target_file_size() as usize),
None,
writer_properties,
writer_stats_config,
None,
)
.await?,
);
}

metrics.rewrite_time_ms = Instant::now().duration_since(rewrite_start).as_millis() as u64;

let mut actions: Vec<Action> = add_actions.clone();
metrics.num_target_files_added = actions.len();

let survivors = barrier
Expand Down
Loading
Loading