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

fix: add config for parquet pushdown on delta scan #2364

Merged
merged 3 commits into from
Mar 31, 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/benchmarks/src/bin/merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -199,8 +199,8 @@ async fn benchmark_merge_tpcds(
table.snapshot()?.clone(),
table.log_store(),
DeltaScanConfig {
wrap_partition_values: true,
file_column_name: Some("file_path".to_string()),
..Default::default()
},
)
.unwrap();
Expand Down
35 changes: 33 additions & 2 deletions crates/core/src/delta_datafusion/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ pub(crate) fn df_logical_schema(
Ok(Arc::new(ArrowSchema::new(fields)))
}

#[derive(Debug, Clone, Default)]
#[derive(Debug, Clone)]
/// Used to specify if additional metadata columns are exposed to the user
pub struct DeltaScanConfigBuilder {
/// Include the source path for each record. The name of this column is determined by `file_column_name`
Expand All @@ -343,6 +343,18 @@ pub struct DeltaScanConfigBuilder {
file_column_name: Option<String>,
/// Whether to wrap partition values in a dictionary encoding to potentially save space
wrap_partition_values: Option<bool>,
enable_parquet_pushdown: bool,
}

impl Default for DeltaScanConfigBuilder {
fn default() -> Self {
DeltaScanConfigBuilder {
include_file_column: false,
file_column_name: None,
wrap_partition_values: None,
enable_parquet_pushdown: true,
}
}
}

impl DeltaScanConfigBuilder {
Expand Down Expand Up @@ -372,6 +384,13 @@ impl DeltaScanConfigBuilder {
self
}

/// Allow pushdown of the scan filter
/// When disabled the filter will only be used for pruning files
pub fn with_parquet_pushdown(mut self, pushdown: bool) -> Self {
self.enable_parquet_pushdown = pushdown;
self
}

/// Build a DeltaScanConfig and ensure no column name conflicts occur during downstream processing
pub fn build(&self, snapshot: &DeltaTableState) -> DeltaResult<DeltaScanConfig> {
let input_schema = snapshot.input_schema()?;
Expand Down Expand Up @@ -411,6 +430,7 @@ impl DeltaScanConfigBuilder {
Ok(DeltaScanConfig {
file_column_name,
wrap_partition_values: self.wrap_partition_values.unwrap_or(true),
enable_parquet_pushdown: self.enable_parquet_pushdown,
})
}
}
Expand All @@ -422,6 +442,8 @@ pub struct DeltaScanConfig {
pub file_column_name: Option<String>,
/// Wrap partition values in a dictionary encoding
pub wrap_partition_values: bool,
/// Allow pushdown of the scan filter
pub enable_parquet_pushdown: bool,
}

#[derive(Debug)]
Expand Down Expand Up @@ -592,6 +614,15 @@ impl<'a> DeltaScanBuilder<'a> {
.datafusion_table_statistics()
.unwrap_or(Statistics::new_unknown(&schema));

// Sometimes (i.e Merge) we want to prune files that don't make the
// filter and read the entire contents for files that do match the
// filter
let parquet_pushdown = if config.enable_parquet_pushdown {
logical_filter.clone()
} else {
None
};

let scan = ParquetFormat::new()
.create_physical_plan(
self.state,
Expand All @@ -605,7 +636,7 @@ impl<'a> DeltaScanBuilder<'a> {
table_partition_cols,
output_ordering: vec![],
},
logical_filter.as_ref(),
parquet_pushdown.as_ref(),
)
.await?;

Expand Down
126 changes: 126 additions & 0 deletions crates/core/src/operations/merge/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -983,6 +983,7 @@ async fn execute(

let scan_config = DeltaScanConfigBuilder::default()
.with_file_column(true)
.with_parquet_pushdown(false)
.build(snapshot)?;

let target_provider = Arc::new(DeltaTableProvider::try_new(
Expand Down Expand Up @@ -2973,6 +2974,131 @@ mod tests {
assert_batches_sorted_eq!(&expected, &actual);
}

#[tokio::test]
async fn test_merge_row_groups_parquet_pushdown() {
//See https://github.com/delta-io/delta-rs/issues/2362
let schema = vec![
StructField::new(
"id".to_string(),
DataType::Primitive(PrimitiveType::String),
true,
),
StructField::new(
"cost".to_string(),
DataType::Primitive(PrimitiveType::Float),
true,
),
StructField::new(
"month".to_string(),
DataType::Primitive(PrimitiveType::String),
true,
),
];

let arrow_schema = Arc::new(ArrowSchema::new(vec![
Field::new("id", ArrowDataType::Utf8, true),
Field::new("cost", ArrowDataType::Float32, true),
Field::new("month", ArrowDataType::Utf8, true),
]));

let table = DeltaOps::new_in_memory()
.create()
.with_columns(schema)
.await
.unwrap();

let ctx = SessionContext::new();
let batch1 = RecordBatch::try_new(
Arc::clone(&arrow_schema.clone()),
vec![
Arc::new(arrow::array::StringArray::from(vec!["A", "B"])),
Arc::new(arrow::array::Float32Array::from(vec![Some(10.15), None])),
Arc::new(arrow::array::StringArray::from(vec![
"2023-07-04",
"2023-07-04",
])),
],
)
.unwrap();

let batch2 = RecordBatch::try_new(
Arc::clone(&arrow_schema.clone()),
vec![
Arc::new(arrow::array::StringArray::from(vec!["C", "D"])),
Arc::new(arrow::array::Float32Array::from(vec![
Some(11.0),
Some(12.0),
])),
Arc::new(arrow::array::StringArray::from(vec![
"2023-07-04",
"2023-07-04",
])),
],
)
.unwrap();

let table = DeltaOps(table)
.write(vec![batch1, batch2])
.with_write_batch_size(2)
.with_save_mode(SaveMode::Append)
.await
.unwrap();
assert_eq!(table.version(), 1);
assert_eq!(table.get_files_count(), 1);

let batch = RecordBatch::try_new(
Arc::clone(&arrow_schema.clone()),
vec![
Arc::new(arrow::array::StringArray::from(vec!["C", "E"])),
Arc::new(arrow::array::Float32Array::from(vec![
Some(12.15),
Some(11.15),
])),
Arc::new(arrow::array::StringArray::from(vec![
"2023-07-04",
"2023-07-04",
])),
],
)
.unwrap();
let source = ctx.read_batch(batch).unwrap();

let (table, _metrics) = DeltaOps(table)
.merge(source, "target.id = source.id and target.id >= 'C'")
.with_source_alias("source")
.with_target_alias("target")
.when_matched_update(|insert| {
insert
.update("id", "target.id")
.update("cost", "source.cost")
.update("month", "target.month")
})
.unwrap()
.when_not_matched_insert(|insert| {
insert
.set("id", "source.id")
.set("cost", "source.cost")
.set("month", "source.month")
})
.unwrap()
.await
.unwrap();

let expected = vec![
"+----+-------+------------+",
"| id | cost | month |",
"+----+-------+------------+",
"| A | 10.15 | 2023-07-04 |",
"| B | | 2023-07-04 |",
"| C | 12.15 | 2023-07-04 |",
"| D | 12.0 | 2023-07-04 |",
"| E | 11.15 | 2023-07-04 |",
"+----+-------+------------+",
];
let actual = get_data(&table).await;
assert_batches_sorted_eq!(&expected, &actual);
}

#[tokio::test]
async fn test_merge_pushdowns_partitioned() {
//See #2158
Expand Down
Loading