From e668b99ba1024ffb7e6aa762b04b3a68a7092257 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 29 May 2025 19:04:02 -0700 Subject: [PATCH 01/58] Initial commit to form PR for datafusion encryption support --- Cargo.lock | 4 + Cargo.toml | 1 + datafusion/common/Cargo.toml | 4 + datafusion/common/src/config.rs | 157 ++++++++++++++++++ .../common/src/file_options/parquet_writer.rs | 31 +++- datafusion/datasource-parquet/src/opener.rs | 8 + datafusion/datasource-parquet/src/source.rs | 9 + datafusion/proto-common/src/from_proto/mod.rs | 2 + .../proto/src/logical_plan/file_formats.rs | 2 + 9 files changed, 217 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index b4ad326358f83..d9e86f3980a1a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1986,6 +1986,7 @@ dependencies = [ "chrono", "half", "hashbrown 0.14.5", + "hex", "indexmap 2.9.0", "insta", "libc", @@ -1996,6 +1997,8 @@ dependencies = [ "pyo3", "rand 0.9.1", "recursive", + "serde", + "serde_json", "sqlparser", "tokio", "web-time", @@ -4440,6 +4443,7 @@ dependencies = [ "num-bigint", "object_store", "paste", + "ring", "seq-macro", "simdutf8", "snap", diff --git a/Cargo.toml b/Cargo.toml index 79bb2f3cc602d..9e4ac7651b999 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -155,6 +155,7 @@ parquet = { version = "55.1.0", default-features = false, features = [ "arrow", "async", "object_store", + "encryption" ] } pbjson = { version = "0.7.0" } pbjson-types = "0.7" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index d471e48be4e75..1a26d8db7ff1e 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -56,6 +56,7 @@ arrow = { workspace = true } arrow-ipc = { workspace = true } base64 = "0.22.1" half = { workspace = true } +hex = "0.4.3" hashbrown = { workspace = true } indexmap = { workspace = true } libc = "0.2.172" @@ -66,8 +67,11 @@ paste = "1.0.15" pyo3 = { version = "0.24.2", optional = true } recursive = { workspace = true, optional = true } sqlparser = { workspace = true } +serde = { version = "1.0.217", features = ["derive"] } +serde_json = {workspace = true} tokio = { workspace = true } + [target.'cfg(target_family = "wasm")'.dependencies] web-time = "1.1.0" diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 59283114e3a9e..5d09fa9bdb107 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -27,6 +27,46 @@ use std::error::Error; use std::fmt::{self, Display}; use std::str::FromStr; +use serde::{Deserialize, Serialize}; +use hex; +#[cfg(feature = "parquet")] +use parquet::encryption::encrypt::FileEncryptionProperties; +#[cfg(feature = "parquet")] +use parquet::encryption::decrypt::FileDecryptionProperties; + +#[derive(Serialize, Deserialize)] +pub struct EncryptionColumnKeys { + pub column_keys_as_hex: HashMap, +} + +impl EncryptionColumnKeys { + pub fn new(column_keys: &HashMap>) -> Self { + let mut column_keys_as_hex: HashMap = HashMap::new(); + + for (key, value) in column_keys.iter() { + column_keys_as_hex.insert(hex::encode(key.clone()), hex::encode(value.clone())); + } + + EncryptionColumnKeys {column_keys_as_hex} + } + + pub fn to_json(&self) -> String { + serde_json::to_string(self).unwrap() + } + + pub fn from_json_to_column_keys(s: &String) -> HashMap> { + let eck: EncryptionColumnKeys = serde_json::from_str(s).expect("failed to decode column keys from JSON"); + let mut hm: HashMap> = HashMap::new(); + for (key, val) in eck.column_keys_as_hex { + let col = String::from_utf8(hex::decode(key).expect("Invalid column name")).expect("Invalid column name"); + let key = hex::decode(val).expect("Invalid column key"); + hm.insert(col, key); + } + hm + } +} + + /// A macro that wraps a configuration struct and automatically derives /// [`Default`] and [`ConfigField`] for it, allowing it to be used /// in the [`ConfigOptions`] configuration tree. @@ -188,6 +228,119 @@ macro_rules! config_namespace { } } +config_namespace! { + pub struct ConfigFileDecryptionProperties { + pub footer_key_as_hex: String, default = String::new() + pub column_keys_as_json_hex: String, default = String::new() + pub aad_prefix_as_hex: String, default = String::new() + } +} + +#[cfg(feature = "parquet")] +impl Into for ConfigFileDecryptionProperties { + fn into(self) -> FileDecryptionProperties { + let eck = EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); + let mut column_names: Vec<&str> = Vec::new(); + let mut column_keys: Vec> = Vec::new(); + if !eck.is_empty() { + for (key, val) in eck.iter() { + column_names.push(key.as_str()); + column_keys.push(val.clone()); + } + } + let mut fep = FileDecryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) + .with_column_keys(column_names, column_keys).unwrap(); + + if self.aad_prefix_as_hex.len() > 0 { + let aad_prefix = hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + fep = fep.with_aad_prefix(aad_prefix); + } + + fep.build().unwrap() + } +} + +#[cfg(feature = "parquet")] +impl From for ConfigFileDecryptionProperties { + fn from(f: FileDecryptionProperties) -> Self { + let mut column_keys: HashMap> = HashMap::new(); + let (column_names_vec, column_keys_vec) = f.column_keys(); + if column_names_vec.len() > 0 { + let mut iter = std::iter::zip(column_names_vec, column_keys_vec); + column_keys = HashMap::from_iter(&mut iter); + } + + let ck = EncryptionColumnKeys::new(&column_keys); + let mut aad_prefix: Vec = Vec::new(); + if let Some(prefix) = f.aad_prefix() { + aad_prefix = prefix.clone(); + } + ConfigFileDecryptionProperties { + footer_key_as_hex: hex::encode(f.footer_key(None).unwrap_or_default().as_ref()), + column_keys_as_json_hex: ck.to_json(), + aad_prefix_as_hex: hex::encode(aad_prefix), + } + } +} + +config_namespace! { + pub struct ConfigFileEncryptionProperties { + pub encrypt_footer: bool, default = false + pub footer_key_as_hex: String, default = String::new() + pub column_keys_as_json_hex: String, default = String::new() + pub aad_prefix_as_hex: String, default = String::new() + } +} + +#[cfg(feature = "parquet")] +impl Into for ConfigFileEncryptionProperties { + fn into(self) -> FileEncryptionProperties { + let eck = EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); + let mut column_names: Vec<&str> = Vec::new(); + let mut column_keys: Vec> = Vec::new(); + if !eck.is_empty() { + for (key, val) in eck.iter() { + column_names.push(&key[..]); + column_keys.push(val.clone()); + } + } + + let mut fep = FileEncryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) + .with_column_keys(column_names, column_keys).expect("Invalid column keys") + .with_plaintext_footer(!self.encrypt_footer); + + if self.aad_prefix_as_hex.len() > 0 { + let aad_prefix: Vec = hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + fep = fep.with_aad_prefix(aad_prefix); + } + fep.build().unwrap() + } +} + +#[cfg(feature = "parquet")] +impl From for ConfigFileEncryptionProperties { + fn from(f: FileEncryptionProperties) -> Self { + let mut column_keys: HashMap> = HashMap::new(); + let (column_names_vec, column_keys_vec, _column_metas_vec) = f.column_keys(); + if column_names_vec.len() > 0 { + let mut iter = std::iter::zip(column_names_vec, column_keys_vec); + column_keys = HashMap::from_iter(&mut iter); + } + let ck = EncryptionColumnKeys::new(&column_keys); + let mut aad_prefix: Vec = Vec::new(); + if let Some(prefix) = f.aad_prefix() { + aad_prefix = prefix.clone(); + } + ConfigFileEncryptionProperties { + encrypt_footer: f.encrypt_footer(), + footer_key_as_hex: hex::encode(f.footer_key()), + column_keys_as_json_hex: ck.to_json(), + aad_prefix_as_hex: hex::encode(aad_prefix), + } + } +} + + config_namespace! { /// Options related to catalog and directory scanning /// @@ -591,6 +744,10 @@ config_namespace! { /// writing out already in-memory data, such as from a cached /// data frame. pub maximum_buffered_record_batches_per_stream: usize, default = 2 + + // Optional encryption settings + pub file_decryption_properties: Option, default = None + pub file_encryption_properties: Option, default = None } } diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 07e763f0ee6f3..ad381d74500a2 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -41,6 +41,8 @@ use parquet::{ schema::types::ColumnPath, }; +use parquet::encryption::encrypt::FileEncryptionProperties; + /// Options for writing parquet files #[derive(Clone, Debug)] pub struct ParquetWriterOptions { @@ -225,6 +227,8 @@ impl ParquetOptions { bloom_filter_on_write, bloom_filter_fpp, bloom_filter_ndv, + file_decryption_properties: _, + file_encryption_properties, // not in WriterProperties enable_page_index: _, @@ -243,6 +247,7 @@ impl ParquetOptions { skip_arrow_metadata: _, } = self; + let mut builder = WriterProperties::builder() .set_data_page_size_limit(*data_pagesize_limit) .set_write_batch_size(*write_batch_size) @@ -261,6 +266,18 @@ impl ParquetOptions { .set_data_page_row_count_limit(*data_page_row_count_limit) .set_bloom_filter_enabled(*bloom_filter_on_write); + let fep: Option = + match file_encryption_properties { + Some(fe) => + Some(fe.clone().into()), + None => None, + }; + + if fep.is_some() { + builder = builder.with_file_encryption_properties(fep.unwrap()); + } + + builder = { #[allow(deprecated)] builder.set_max_statistics_size( @@ -449,7 +466,7 @@ mod tests { }; use std::collections::HashMap; - use crate::config::{ParquetColumnOptions, ParquetOptions}; + use crate::config::{ConfigFileEncryptionProperties, ParquetColumnOptions, ParquetOptions}; use super::*; @@ -499,6 +516,8 @@ mod tests { bloom_filter_on_write: !defaults.bloom_filter_on_write, bloom_filter_fpp: Some(0.42), bloom_filter_ndv: Some(42), + file_decryption_properties: None, + file_encryption_properties: None, // not in WriterProperties, but itemizing here to not skip newly added props enable_page_index: defaults.enable_page_index, @@ -580,6 +599,14 @@ mod tests { HashMap::from([(COL_NAME.into(), configured_col_props)]) }; + let fep: Option = + match props.file_encryption_properties() { + Some(fe) => { + Some(fe.clone().into()) + }, + None => None, + }; + #[allow(deprecated)] // max_statistics_size TableParquetOptions { global: ParquetOptions { @@ -605,6 +632,8 @@ mod tests { .unwrap_or_default(), bloom_filter_fpp: default_col_props.bloom_filter_fpp, bloom_filter_ndv: default_col_props.bloom_filter_ndv, + file_encryption_properties: fep, + file_decryption_properties: None, // not in WriterProperties enable_page_index: global_options_defaults.enable_page_index, diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 9e14425074f78..a028516316408 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -42,6 +42,7 @@ use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::metadata::ParquetMetaDataReader; +use parquet::encryption::decrypt::FileDecryptionProperties; /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { @@ -82,6 +83,8 @@ pub(super) struct ParquetOpener { pub enable_row_group_stats_pruning: bool, /// Coerce INT96 timestamps to specific TimeUnit pub coerce_int96: Option, + /// Optional parquet FileDecryptionProperties + pub file_decryption_properties: Option>, } impl FileOpener for ParquetOpener { @@ -123,6 +126,8 @@ impl FileOpener for ParquetOpener { .global_counter("num_predicate_creation_errors"); let enable_page_index = self.enable_page_index; + let file_decryption_properties = self.file_decryption_properties.clone(); + Ok(Box::pin(async move { // Don't load the page index yet. Since it is not stored inline in @@ -131,6 +136,9 @@ impl FileOpener for ParquetOpener { // pruning predicates. Thus default to not requesting if from the // underlying reader. let mut options = ArrowReaderOptions::new().with_page_index(false); + if let Some(ref fd_val) = file_decryption_properties { + options = options.with_file_decryption_properties((**fd_val).clone()); + } let mut metadata_timer = file_metrics.metadata_load_time.timer(); // Begin by loading the metadata from the underlying reader (note diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 69347f440c365..5c9c73f1f407f 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -49,6 +49,7 @@ use datafusion_physical_plan::DisplayFormatType; use itertools::Itertools; use object_store::ObjectStore; +use parquet::encryption::decrypt::FileDecryptionProperties; /// Execution plan for reading one or more Parquet files. /// @@ -466,6 +467,13 @@ impl FileSource for ParquetSource { Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _ }); + let mut file_decryption_properties: Option> = None; + if self.table_parquet_options().global.file_decryption_properties.is_some() { + let fdp: FileDecryptionProperties = + self.table_parquet_options().global.file_decryption_properties.clone().unwrap().into(); + file_decryption_properties = Some(Arc::new(fdp)); + } + let coerce_int96 = self .table_parquet_options .global @@ -492,6 +500,7 @@ impl FileSource for ParquetSource { enable_row_group_stats_pruning: self.table_parquet_options.global.pruning, schema_adapter_factory, coerce_int96, + file_decryption_properties, }) } diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index bd969db316872..83c4592921940 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -988,6 +988,8 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { protobuf::parquet_options::CoerceInt96Opt::CoerceInt96(v) => Some(v), }).unwrap_or(None), skip_arrow_metadata: value.skip_arrow_metadata, + file_decryption_properties: None, + file_encryption_properties: None, }) } } diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index d3f6511ec98fa..31b525ce01e46 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -513,6 +513,8 @@ impl From<&ParquetOptionsProto> for ParquetOptions { coerce_int96: proto.coerce_int96_opt.as_ref().map(|opt| match opt { parquet_options::CoerceInt96Opt::CoerceInt96(coerce_int96) => coerce_int96.clone(), }), + file_decryption_properties: None, + file_encryption_properties: None, } } } From d38dba45df3b62350d802b88f0d981b51312ed94 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 29 May 2025 22:14:26 -0700 Subject: [PATCH 02/58] Add tests for encryption configuration --- datafusion/common/src/config.rs | 267 +++++++++++++++++++++++--------- 1 file changed, 194 insertions(+), 73 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 5d09fa9bdb107..c853217d2b24f 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -34,38 +34,6 @@ use parquet::encryption::encrypt::FileEncryptionProperties; #[cfg(feature = "parquet")] use parquet::encryption::decrypt::FileDecryptionProperties; -#[derive(Serialize, Deserialize)] -pub struct EncryptionColumnKeys { - pub column_keys_as_hex: HashMap, -} - -impl EncryptionColumnKeys { - pub fn new(column_keys: &HashMap>) -> Self { - let mut column_keys_as_hex: HashMap = HashMap::new(); - - for (key, value) in column_keys.iter() { - column_keys_as_hex.insert(hex::encode(key.clone()), hex::encode(value.clone())); - } - - EncryptionColumnKeys {column_keys_as_hex} - } - - pub fn to_json(&self) -> String { - serde_json::to_string(self).unwrap() - } - - pub fn from_json_to_column_keys(s: &String) -> HashMap> { - let eck: EncryptionColumnKeys = serde_json::from_str(s).expect("failed to decode column keys from JSON"); - let mut hm: HashMap> = HashMap::new(); - for (key, val) in eck.column_keys_as_hex { - let col = String::from_utf8(hex::decode(key).expect("Invalid column name")).expect("Invalid column name"); - let key = hex::decode(val).expect("Invalid column key"); - hm.insert(col, key); - } - hm - } -} - /// A macro that wraps a configuration struct and automatically derives /// [`Default`] and [`ConfigField`] for it, allowing it to be used @@ -228,11 +196,75 @@ macro_rules! config_namespace { } } +#[derive(Serialize, Deserialize, Debug)] +pub struct EncryptionKeyAsHex { + pub key: String, + pub key_metadata: Option, +} + +#[derive(Debug)] +pub struct EncryptionKey { + pub key: Vec, + pub key_metadata: Option>, +} + +#[derive(Serialize, Deserialize, Debug)] +pub struct EncryptionColumnKeys { + pub column_keys_as_hex: HashMap, +} + +impl EncryptionColumnKeys { + pub fn new(column_names: &Vec, keys: &Vec>, meta: &Vec>) -> Self { + let mut column_keys_as_hex: HashMap = HashMap::new(); + + for (i, column_name) in column_names.iter().enumerate() { + let key_metadata: Option = meta.get(i).map(|x| hex::encode(x)); + let encryption_key = EncryptionKeyAsHex { + key: hex::encode(keys[i].clone()), + key_metadata, + }; + column_keys_as_hex.insert(column_name.clone(), encryption_key); + } + + EncryptionColumnKeys {column_keys_as_hex} + } + + pub fn to_json(&self) -> String { + serde_json::to_string(self).unwrap() + } + + pub fn from_json_to_column_keys(s: &String) -> HashMap { + let eck: EncryptionColumnKeys = serde_json::from_str(s).expect("failed to decode column keys from JSON"); + let mut hm: HashMap = HashMap::new(); + for (key, val) in eck.column_keys_as_hex { + let col = key; + let key = hex::decode(val.key).expect("Invalid column key"); + let key_metadata = val.key_metadata.map(|x| hex::decode(x).expect("Invalid column metadata")); + let encryption_key = EncryptionKey {key, key_metadata}; + hm.insert(col, encryption_key); + } + hm + } +} + + +config_namespace! { + pub struct ConfigFileEncryptionProperties { + pub encrypt_footer: bool, default = false + pub footer_key_as_hex: String, default = String::new() + pub footer_key_metadata_as_hex: String, default = String::new() + pub column_keys_as_json_hex: String, default = String::new() + pub aad_prefix_as_hex: String, default = String::new() + pub store_aad_prefix: bool, default = false + } +} + config_namespace! { pub struct ConfigFileDecryptionProperties { pub footer_key_as_hex: String, default = String::new() pub column_keys_as_json_hex: String, default = String::new() pub aad_prefix_as_hex: String, default = String::new() + pub footer_signature_verification: bool, default = true } } @@ -243,13 +275,17 @@ impl Into for ConfigFileDecryptionProperties { let mut column_names: Vec<&str> = Vec::new(); let mut column_keys: Vec> = Vec::new(); if !eck.is_empty() { - for (key, val) in eck.iter() { - column_names.push(key.as_str()); - column_keys.push(val.clone()); + for (col_name, encryption_key) in eck.iter() { + column_names.push(col_name.as_str()); + column_keys.push(encryption_key.key.clone()); } } let mut fep = FileDecryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) .with_column_keys(column_names, column_keys).unwrap(); + + if !self.footer_signature_verification { + fep = fep.disable_footer_signature_verification(); + } if self.aad_prefix_as_hex.len() > 0 { let aad_prefix = hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); @@ -263,14 +299,9 @@ impl Into for ConfigFileDecryptionProperties { #[cfg(feature = "parquet")] impl From for ConfigFileDecryptionProperties { fn from(f: FileDecryptionProperties) -> Self { - let mut column_keys: HashMap> = HashMap::new(); let (column_names_vec, column_keys_vec) = f.column_keys(); - if column_names_vec.len() > 0 { - let mut iter = std::iter::zip(column_names_vec, column_keys_vec); - column_keys = HashMap::from_iter(&mut iter); - } - - let ck = EncryptionColumnKeys::new(&column_keys); + let column_keys_meta= Vec::new(); + let ck = EncryptionColumnKeys::new(&column_names_vec, &column_keys_vec, &column_keys_meta); let mut aad_prefix: Vec = Vec::new(); if let Some(prefix) = f.aad_prefix() { aad_prefix = prefix.clone(); @@ -279,35 +310,33 @@ impl From for ConfigFileDecryptionProperties { footer_key_as_hex: hex::encode(f.footer_key(None).unwrap_or_default().as_ref()), column_keys_as_json_hex: ck.to_json(), aad_prefix_as_hex: hex::encode(aad_prefix), + footer_signature_verification: f.check_plaintext_footer_integrity() } } } -config_namespace! { - pub struct ConfigFileEncryptionProperties { - pub encrypt_footer: bool, default = false - pub footer_key_as_hex: String, default = String::new() - pub column_keys_as_json_hex: String, default = String::new() - pub aad_prefix_as_hex: String, default = String::new() - } -} - #[cfg(feature = "parquet")] impl Into for ConfigFileEncryptionProperties { fn into(self) -> FileEncryptionProperties { let eck = EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); - let mut column_names: Vec<&str> = Vec::new(); - let mut column_keys: Vec> = Vec::new(); - if !eck.is_empty() { - for (key, val) in eck.iter() { - column_names.push(&key[..]); - column_keys.push(val.clone()); - } + let mut fep = FileEncryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) + .with_plaintext_footer(!self.encrypt_footer) + .with_aad_prefix_storage(self.store_aad_prefix); + + if self.footer_key_metadata_as_hex.len() > 0 { + fep = fep.with_footer_key_metadata(hex::decode(&self.footer_key_metadata_as_hex).expect("Invalid footer key metadata")); } - let mut fep = FileEncryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) - .with_column_keys(column_names, column_keys).expect("Invalid column keys") - .with_plaintext_footer(!self.encrypt_footer); + for (column_name, encryption_key) in eck.iter() { + match &encryption_key.key_metadata { + Some(key_metadata) => { + fep = fep.with_column_key_and_metadata(column_name, encryption_key.key.clone(), key_metadata.clone()); + }, + None => { + fep = fep.with_column_key(column_name, encryption_key.key.clone()); + } + } + } if self.aad_prefix_as_hex.len() > 0 { let aad_prefix: Vec = hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); @@ -320,13 +349,8 @@ impl Into for ConfigFileEncryptionProperties { #[cfg(feature = "parquet")] impl From for ConfigFileEncryptionProperties { fn from(f: FileEncryptionProperties) -> Self { - let mut column_keys: HashMap> = HashMap::new(); - let (column_names_vec, column_keys_vec, _column_metas_vec) = f.column_keys(); - if column_names_vec.len() > 0 { - let mut iter = std::iter::zip(column_names_vec, column_keys_vec); - column_keys = HashMap::from_iter(&mut iter); - } - let ck = EncryptionColumnKeys::new(&column_keys); + let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); + let ck = EncryptionColumnKeys::new(&column_names_vec, &column_keys_vec, &column_metas_vec); let mut aad_prefix: Vec = Vec::new(); if let Some(prefix) = f.aad_prefix() { aad_prefix = prefix.clone(); @@ -334,8 +358,10 @@ impl From for ConfigFileEncryptionProperties { ConfigFileEncryptionProperties { encrypt_footer: f.encrypt_footer(), footer_key_as_hex: hex::encode(f.footer_key()), + footer_key_metadata_as_hex: f.footer_key_metadata().map(|x| hex::encode(x)).unwrap_or_default(), column_keys_as_json_hex: ck.to_json(), aad_prefix_as_hex: hex::encode(aad_prefix), + store_aad_prefix: f.store_aad_prefix(), } } } @@ -2255,11 +2281,7 @@ impl Display for OutputFormat { mod tests { use std::any::Any; use std::collections::HashMap; - - use crate::config::{ - ConfigEntry, ConfigExtension, ConfigField, ConfigFileType, ExtensionOptions, - Extensions, TableOptions, - }; + use crate::config::{ConfigEntry, ConfigExtension, ConfigField, ConfigFileDecryptionProperties, ConfigFileEncryptionProperties, ConfigFileType, ExtensionOptions, Extensions, TableOptions}; #[derive(Default, Debug, Clone)] pub struct TestExtensionConfig { @@ -2388,6 +2410,105 @@ mod tests { ); } + #[cfg(feature = "parquet")] + #[test] + fn parquet_table_encryption() { + use parquet::encryption::decrypt::FileDecryptionProperties; + use parquet::encryption::encrypt::FileEncryptionProperties; + + let footer_key = b"0123456789012345".to_vec(); // 128bit/16 + let column_names = vec!["double_field", "float_field"]; + let column_keys = vec![b"1234567890123450".to_vec(), b"1234567890123451".to_vec()]; + + let file_encryption_properties = FileEncryptionProperties::builder(footer_key.clone()) + .with_column_keys(column_names.clone(), column_keys.clone()) + .unwrap() + .build() + .unwrap(); + + let decryption_properties = FileDecryptionProperties::builder(footer_key.clone()) + .with_column_keys(column_names.clone(), column_keys.clone()) + .unwrap() + .build() + .unwrap(); + + // Test round-trip + let config_encrypt: ConfigFileEncryptionProperties = file_encryption_properties.clone().into(); + let encryption_properties_built: FileEncryptionProperties = config_encrypt.clone().into(); + assert_eq!(file_encryption_properties, encryption_properties_built); + + let config_decrypt: ConfigFileDecryptionProperties = decryption_properties.clone().into(); + let decryption_properties_built: FileDecryptionProperties = config_decrypt.clone().into(); + assert_eq!(decryption_properties, decryption_properties_built); + + + /////////////////////////////////////////////////////////////////////////////////// + // Test encryption config + + // Display original encryption config + // println!("{:#?}", config_encrypt); + + let mut table_config = TableOptions::new(); + table_config.set_config_format(ConfigFileType::PARQUET); + table_config + .parquet + .set("file_encryption_properties.encrypt_footer", config_encrypt.encrypt_footer.to_string().as_str()) + .unwrap(); + table_config + .parquet + .set("file_encryption_properties.footer_key_as_hex", config_encrypt.footer_key_as_hex.as_str()) + .unwrap(); + + table_config + .parquet + .set("file_encryption_properties.column_keys_as_json_hex", config_encrypt.column_keys_as_json_hex.as_str()) + .unwrap(); + + // Print matching final encryption config + // println!("{:#?}", table_config.parquet.global.file_encryption_properties); + + assert_eq!( + table_config.parquet.global.file_encryption_properties, + Some(config_encrypt) + ); + + + /////////////////////////////////////////////////////////////////////////////////// + // Test decryption config + + // Display original decryption config + // println!("{:#?}", config_decrypt); + + let mut table_config = TableOptions::new(); + table_config.set_config_format(ConfigFileType::PARQUET); + table_config + .parquet + .set("file_decryption_properties.footer_key_as_hex", config_decrypt.footer_key_as_hex.as_str()) + .unwrap(); + + table_config + .parquet + .set("file_decryption_properties.column_keys_as_json_hex", config_decrypt.column_keys_as_json_hex.as_str()) + .unwrap(); + + // Print matching final decryption config + // println!("{:#?}", table_config.parquet.global.file_decryption_properties); + + assert_eq!( + table_config.parquet.global.file_decryption_properties, + Some(config_decrypt.clone()) + ); + + // Set config directly + let mut table_config = TableOptions::new(); + table_config.set_config_format(ConfigFileType::PARQUET); + table_config.parquet.global.file_decryption_properties = Some(config_decrypt.clone()); + assert_eq!( + table_config.parquet.global.file_decryption_properties, + Some(config_decrypt.clone()) + ); + } + #[cfg(feature = "parquet")] #[test] fn parquet_table_options_config_entry() { From 5a2b4562f9d6831fe6629296cdc151e3e9e582f7 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 30 May 2025 13:44:47 -0700 Subject: [PATCH 03/58] Apply cargo fmt --- datafusion/common/src/config.rs | 175 ++++++++++++------ .../common/src/file_options/parquet_writer.rs | 20 +- datafusion/datasource-parquet/src/opener.rs | 5 +- datafusion/datasource-parquet/src/source.rs | 20 +- 4 files changed, 141 insertions(+), 79 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index c853217d2b24f..ccec20e99deae 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -27,13 +27,12 @@ use std::error::Error; use std::fmt::{self, Display}; use std::str::FromStr; -use serde::{Deserialize, Serialize}; use hex; #[cfg(feature = "parquet")] -use parquet::encryption::encrypt::FileEncryptionProperties; -#[cfg(feature = "parquet")] use parquet::encryption::decrypt::FileDecryptionProperties; - +#[cfg(feature = "parquet")] +use parquet::encryption::encrypt::FileEncryptionProperties; +use serde::{Deserialize, Serialize}; /// A macro that wraps a configuration struct and automatically derives /// [`Default`] and [`ConfigField`] for it, allowing it to be used @@ -214,7 +213,11 @@ pub struct EncryptionColumnKeys { } impl EncryptionColumnKeys { - pub fn new(column_names: &Vec, keys: &Vec>, meta: &Vec>) -> Self { + pub fn new( + column_names: &Vec, + keys: &Vec>, + meta: &Vec>, + ) -> Self { let mut column_keys_as_hex: HashMap = HashMap::new(); for (i, column_name) in column_names.iter().enumerate() { @@ -226,7 +229,7 @@ impl EncryptionColumnKeys { column_keys_as_hex.insert(column_name.clone(), encryption_key); } - EncryptionColumnKeys {column_keys_as_hex} + EncryptionColumnKeys { column_keys_as_hex } } pub fn to_json(&self) -> String { @@ -234,20 +237,22 @@ impl EncryptionColumnKeys { } pub fn from_json_to_column_keys(s: &String) -> HashMap { - let eck: EncryptionColumnKeys = serde_json::from_str(s).expect("failed to decode column keys from JSON"); + let eck: EncryptionColumnKeys = + serde_json::from_str(s).expect("failed to decode column keys from JSON"); let mut hm: HashMap = HashMap::new(); for (key, val) in eck.column_keys_as_hex { let col = key; let key = hex::decode(val.key).expect("Invalid column key"); - let key_metadata = val.key_metadata.map(|x| hex::decode(x).expect("Invalid column metadata")); - let encryption_key = EncryptionKey {key, key_metadata}; + let key_metadata = val + .key_metadata + .map(|x| hex::decode(x).expect("Invalid column metadata")); + let encryption_key = EncryptionKey { key, key_metadata }; hm.insert(col, encryption_key); } hm } } - config_namespace! { pub struct ConfigFileEncryptionProperties { pub encrypt_footer: bool, default = false @@ -271,7 +276,8 @@ config_namespace! { #[cfg(feature = "parquet")] impl Into for ConfigFileDecryptionProperties { fn into(self) -> FileDecryptionProperties { - let eck = EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); + let eck = + EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); let mut column_names: Vec<&str> = Vec::new(); let mut column_keys: Vec> = Vec::new(); if !eck.is_empty() { @@ -280,15 +286,19 @@ impl Into for ConfigFileDecryptionProperties { column_keys.push(encryption_key.key.clone()); } } - let mut fep = FileDecryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) - .with_column_keys(column_names, column_keys).unwrap(); - + let mut fep = FileDecryptionProperties::builder( + hex::decode(self.footer_key_as_hex).unwrap(), + ) + .with_column_keys(column_names, column_keys) + .unwrap(); + if !self.footer_signature_verification { fep = fep.disable_footer_signature_verification(); } if self.aad_prefix_as_hex.len() > 0 { - let aad_prefix = hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + let aad_prefix = + hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); fep = fep.with_aad_prefix(aad_prefix); } @@ -300,17 +310,23 @@ impl Into for ConfigFileDecryptionProperties { impl From for ConfigFileDecryptionProperties { fn from(f: FileDecryptionProperties) -> Self { let (column_names_vec, column_keys_vec) = f.column_keys(); - let column_keys_meta= Vec::new(); - let ck = EncryptionColumnKeys::new(&column_names_vec, &column_keys_vec, &column_keys_meta); + let column_keys_meta = Vec::new(); + let ck = EncryptionColumnKeys::new( + &column_names_vec, + &column_keys_vec, + &column_keys_meta, + ); let mut aad_prefix: Vec = Vec::new(); if let Some(prefix) = f.aad_prefix() { aad_prefix = prefix.clone(); } ConfigFileDecryptionProperties { - footer_key_as_hex: hex::encode(f.footer_key(None).unwrap_or_default().as_ref()), + footer_key_as_hex: hex::encode( + f.footer_key(None).unwrap_or_default().as_ref(), + ), column_keys_as_json_hex: ck.to_json(), aad_prefix_as_hex: hex::encode(aad_prefix), - footer_signature_verification: f.check_plaintext_footer_integrity() + footer_signature_verification: f.check_plaintext_footer_integrity(), } } } @@ -318,20 +334,30 @@ impl From for ConfigFileDecryptionProperties { #[cfg(feature = "parquet")] impl Into for ConfigFileEncryptionProperties { fn into(self) -> FileEncryptionProperties { - let eck = EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); - let mut fep = FileEncryptionProperties::builder(hex::decode(self.footer_key_as_hex).unwrap()) - .with_plaintext_footer(!self.encrypt_footer) - .with_aad_prefix_storage(self.store_aad_prefix); - + let eck = + EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); + let mut fep = FileEncryptionProperties::builder( + hex::decode(self.footer_key_as_hex).unwrap(), + ) + .with_plaintext_footer(!self.encrypt_footer) + .with_aad_prefix_storage(self.store_aad_prefix); + if self.footer_key_metadata_as_hex.len() > 0 { - fep = fep.with_footer_key_metadata(hex::decode(&self.footer_key_metadata_as_hex).expect("Invalid footer key metadata")); + fep = fep.with_footer_key_metadata( + hex::decode(&self.footer_key_metadata_as_hex) + .expect("Invalid footer key metadata"), + ); } for (column_name, encryption_key) in eck.iter() { match &encryption_key.key_metadata { Some(key_metadata) => { - fep = fep.with_column_key_and_metadata(column_name, encryption_key.key.clone(), key_metadata.clone()); - }, + fep = fep.with_column_key_and_metadata( + column_name, + encryption_key.key.clone(), + key_metadata.clone(), + ); + } None => { fep = fep.with_column_key(column_name, encryption_key.key.clone()); } @@ -339,7 +365,8 @@ impl Into for ConfigFileEncryptionProperties { } if self.aad_prefix_as_hex.len() > 0 { - let aad_prefix: Vec = hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + let aad_prefix: Vec = + hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); fep = fep.with_aad_prefix(aad_prefix); } fep.build().unwrap() @@ -350,7 +377,11 @@ impl Into for ConfigFileEncryptionProperties { impl From for ConfigFileEncryptionProperties { fn from(f: FileEncryptionProperties) -> Self { let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); - let ck = EncryptionColumnKeys::new(&column_names_vec, &column_keys_vec, &column_metas_vec); + let ck = EncryptionColumnKeys::new( + &column_names_vec, + &column_keys_vec, + &column_metas_vec, + ); let mut aad_prefix: Vec = Vec::new(); if let Some(prefix) = f.aad_prefix() { aad_prefix = prefix.clone(); @@ -358,7 +389,10 @@ impl From for ConfigFileEncryptionProperties { ConfigFileEncryptionProperties { encrypt_footer: f.encrypt_footer(), footer_key_as_hex: hex::encode(f.footer_key()), - footer_key_metadata_as_hex: f.footer_key_metadata().map(|x| hex::encode(x)).unwrap_or_default(), + footer_key_metadata_as_hex: f + .footer_key_metadata() + .map(|x| hex::encode(x)) + .unwrap_or_default(), column_keys_as_json_hex: ck.to_json(), aad_prefix_as_hex: hex::encode(aad_prefix), store_aad_prefix: f.store_aad_prefix(), @@ -366,7 +400,6 @@ impl From for ConfigFileEncryptionProperties { } } - config_namespace! { /// Options related to catalog and directory scanning /// @@ -2279,9 +2312,13 @@ impl Display for OutputFormat { #[cfg(test)] mod tests { + use crate::config::{ + ConfigEntry, ConfigExtension, ConfigField, ConfigFileDecryptionProperties, + ConfigFileEncryptionProperties, ConfigFileType, ExtensionOptions, Extensions, + TableOptions, + }; use std::any::Any; use std::collections::HashMap; - use crate::config::{ConfigEntry, ConfigExtension, ConfigField, ConfigFileDecryptionProperties, ConfigFileEncryptionProperties, ConfigFileType, ExtensionOptions, Extensions, TableOptions}; #[derive(Default, Debug, Clone)] pub struct TestExtensionConfig { @@ -2415,80 +2452,99 @@ mod tests { fn parquet_table_encryption() { use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; - + let footer_key = b"0123456789012345".to_vec(); // 128bit/16 let column_names = vec!["double_field", "float_field"]; - let column_keys = vec![b"1234567890123450".to_vec(), b"1234567890123451".to_vec()]; + let column_keys = + vec![b"1234567890123450".to_vec(), b"1234567890123451".to_vec()]; - let file_encryption_properties = FileEncryptionProperties::builder(footer_key.clone()) - .with_column_keys(column_names.clone(), column_keys.clone()) - .unwrap() - .build() - .unwrap(); + let file_encryption_properties = + FileEncryptionProperties::builder(footer_key.clone()) + .with_column_keys(column_names.clone(), column_keys.clone()) + .unwrap() + .build() + .unwrap(); let decryption_properties = FileDecryptionProperties::builder(footer_key.clone()) .with_column_keys(column_names.clone(), column_keys.clone()) .unwrap() .build() .unwrap(); - + // Test round-trip - let config_encrypt: ConfigFileEncryptionProperties = file_encryption_properties.clone().into(); - let encryption_properties_built: FileEncryptionProperties = config_encrypt.clone().into(); + let config_encrypt: ConfigFileEncryptionProperties = + file_encryption_properties.clone().into(); + let encryption_properties_built: FileEncryptionProperties = + config_encrypt.clone().into(); assert_eq!(file_encryption_properties, encryption_properties_built); - - let config_decrypt: ConfigFileDecryptionProperties = decryption_properties.clone().into(); - let decryption_properties_built: FileDecryptionProperties = config_decrypt.clone().into(); + + let config_decrypt: ConfigFileDecryptionProperties = + decryption_properties.clone().into(); + let decryption_properties_built: FileDecryptionProperties = + config_decrypt.clone().into(); assert_eq!(decryption_properties, decryption_properties_built); - - + /////////////////////////////////////////////////////////////////////////////////// // Test encryption config - + // Display original encryption config // println!("{:#?}", config_encrypt); - + let mut table_config = TableOptions::new(); table_config.set_config_format(ConfigFileType::PARQUET); table_config .parquet - .set("file_encryption_properties.encrypt_footer", config_encrypt.encrypt_footer.to_string().as_str()) + .set( + "file_encryption_properties.encrypt_footer", + config_encrypt.encrypt_footer.to_string().as_str(), + ) .unwrap(); table_config .parquet - .set("file_encryption_properties.footer_key_as_hex", config_encrypt.footer_key_as_hex.as_str()) + .set( + "file_encryption_properties.footer_key_as_hex", + config_encrypt.footer_key_as_hex.as_str(), + ) .unwrap(); - + table_config .parquet - .set("file_encryption_properties.column_keys_as_json_hex", config_encrypt.column_keys_as_json_hex.as_str()) + .set( + "file_encryption_properties.column_keys_as_json_hex", + config_encrypt.column_keys_as_json_hex.as_str(), + ) .unwrap(); // Print matching final encryption config // println!("{:#?}", table_config.parquet.global.file_encryption_properties); - + assert_eq!( table_config.parquet.global.file_encryption_properties, Some(config_encrypt) ); - /////////////////////////////////////////////////////////////////////////////////// // Test decryption config // Display original decryption config // println!("{:#?}", config_decrypt); - + let mut table_config = TableOptions::new(); table_config.set_config_format(ConfigFileType::PARQUET); table_config .parquet - .set("file_decryption_properties.footer_key_as_hex", config_decrypt.footer_key_as_hex.as_str()) + .set( + "file_decryption_properties.footer_key_as_hex", + config_decrypt.footer_key_as_hex.as_str(), + ) .unwrap(); table_config .parquet - .set("file_decryption_properties.column_keys_as_json_hex", config_decrypt.column_keys_as_json_hex.as_str()) + .set( + "file_decryption_properties.column_keys_as_json_hex", + config_decrypt.column_keys_as_json_hex.as_str(), + ) .unwrap(); // Print matching final decryption config @@ -2502,7 +2558,8 @@ mod tests { // Set config directly let mut table_config = TableOptions::new(); table_config.set_config_format(ConfigFileType::PARQUET); - table_config.parquet.global.file_decryption_properties = Some(config_decrypt.clone()); + table_config.parquet.global.file_decryption_properties = + Some(config_decrypt.clone()); assert_eq!( table_config.parquet.global.file_decryption_properties, Some(config_decrypt.clone()) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index ad381d74500a2..d7f1617f7a4d0 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -247,7 +247,6 @@ impl ParquetOptions { skip_arrow_metadata: _, } = self; - let mut builder = WriterProperties::builder() .set_data_page_size_limit(*data_pagesize_limit) .set_write_batch_size(*write_batch_size) @@ -266,18 +265,15 @@ impl ParquetOptions { .set_data_page_row_count_limit(*data_page_row_count_limit) .set_bloom_filter_enabled(*bloom_filter_on_write); - let fep: Option = - match file_encryption_properties { - Some(fe) => - Some(fe.clone().into()), - None => None, - }; + let fep: Option = match file_encryption_properties { + Some(fe) => Some(fe.clone().into()), + None => None, + }; if fep.is_some() { builder = builder.with_file_encryption_properties(fep.unwrap()); } - builder = { #[allow(deprecated)] builder.set_max_statistics_size( @@ -466,7 +462,9 @@ mod tests { }; use std::collections::HashMap; - use crate::config::{ConfigFileEncryptionProperties, ParquetColumnOptions, ParquetOptions}; + use crate::config::{ + ConfigFileEncryptionProperties, ParquetColumnOptions, ParquetOptions, + }; use super::*; @@ -601,9 +599,7 @@ mod tests { let fep: Option = match props.file_encryption_properties() { - Some(fe) => { - Some(fe.clone().into()) - }, + Some(fe) => Some(fe.clone().into()), None => None, }; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index a028516316408..f97bd193e83b4 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -41,8 +41,8 @@ use log::debug; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; -use parquet::file::metadata::ParquetMetaDataReader; use parquet::encryption::decrypt::FileDecryptionProperties; +use parquet::file::metadata::ParquetMetaDataReader; /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { @@ -84,7 +84,7 @@ pub(super) struct ParquetOpener { /// Coerce INT96 timestamps to specific TimeUnit pub coerce_int96: Option, /// Optional parquet FileDecryptionProperties - pub file_decryption_properties: Option>, + pub file_decryption_properties: Option>, } impl FileOpener for ParquetOpener { @@ -128,7 +128,6 @@ impl FileOpener for ParquetOpener { let enable_page_index = self.enable_page_index; let file_decryption_properties = self.file_decryption_properties.clone(); - Ok(Box::pin(async move { // Don't load the page index yet. Since it is not stored inline in // the footer, loading the page index if it is not needed will do diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 5c9c73f1f407f..17f53a4d8bd55 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -467,13 +467,23 @@ impl FileSource for ParquetSource { Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _ }); - let mut file_decryption_properties: Option> = None; - if self.table_parquet_options().global.file_decryption_properties.is_some() { - let fdp: FileDecryptionProperties = - self.table_parquet_options().global.file_decryption_properties.clone().unwrap().into(); + let mut file_decryption_properties: Option> = None; + if self + .table_parquet_options() + .global + .file_decryption_properties + .is_some() + { + let fdp: FileDecryptionProperties = self + .table_parquet_options() + .global + .file_decryption_properties + .clone() + .unwrap() + .into(); file_decryption_properties = Some(Arc::new(fdp)); } - + let coerce_int96 = self .table_parquet_options .global From c97267681c2b615ad1ac7c9709e904e3c8348bed Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 30 May 2025 16:41:37 -0700 Subject: [PATCH 04/58] Add a roundtrip encryption test to the parquet tests. --- .../core/tests/parquet/custom_reader.rs | 1 + datafusion/core/tests/parquet/encryption.rs | 127 ++++++++++++++++++ datafusion/core/tests/parquet/mod.rs | 1 + .../datasource-parquet/src/file_format.rs | 33 ++++- 4 files changed, 159 insertions(+), 3 deletions(-) create mode 100644 datafusion/core/tests/parquet/encryption.rs diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 761a78a29fd3a..3abaa61f2a830 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -241,6 +241,7 @@ impl AsyncFileReader for ParquetFileReader { self.store.as_ref(), &self.meta, self.metadata_size_hint, + None ) .await .map_err(|e| { diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs new file mode 100644 index 0000000000000..4f90074470b19 --- /dev/null +++ b/datafusion/core/tests/parquet/encryption.rs @@ -0,0 +1,127 @@ +// 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. + +//! non trivial integration testing for parquet predicate pushdown +//! +//! Testing hints: If you run this test with --nocapture it will tell you where +//! the generated parquet file went. You can then test it and try out various queries +//! datafusion-cli like: +//! +//! ```sql +//! create external table data stored as parquet location 'data.parquet'; +//! select * from data limit 10; +//! ``` + +use std::fs::File; +use std::path::{Path, PathBuf}; +use std::sync::Arc; +use arrow::record_batch::RecordBatch; +use datafusion::prelude::{ + ParquetReadOptions, SessionContext, +}; + +use parquet::encryption::decrypt::FileDecryptionProperties; +use parquet::encryption::encrypt::FileEncryptionProperties; +use parquet::arrow::ArrowWriter; +use parquet::file::properties::WriterProperties; +use tempfile::TempDir; +use datafusion::execution::SessionStateBuilder; +use datafusion_common::config::ConfigFileDecryptionProperties; +use datafusion_execution::config::SessionConfig; + + +async fn read_parquet_test_data>(path: T, ctx: &SessionContext) -> Vec { + ctx.read_parquet(path.into(), ParquetReadOptions::default()) + .await + .unwrap() + .collect() + .await + .unwrap() +} + +pub fn write_batches( + path: PathBuf, + props: WriterProperties, + batches: impl IntoIterator, +) -> datafusion_common::Result { + let mut batches = batches.into_iter(); + let first_batch = batches.next().expect("need at least one record batch"); + let schema = first_batch.schema(); + + let file = File::create(&path)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props))?; + + writer.write(&first_batch)?; + let mut num_rows = first_batch.num_rows(); + + for batch in batches { + writer.write(&batch)?; + num_rows += batch.num_rows(); + } + writer.close()?; + Ok(num_rows) +} + +#[tokio::test] +async fn round_trip_encryption() { + let ctx: SessionContext = SessionContext::new(); + + let batches = + read_parquet_test_data("tests/data/filter_pushdown/single_file.gz.parquet", &ctx).await; + + let schema = batches[0].schema(); + let footer_key = b"0123456789012345".to_vec(); // 128bit/16 + let column_key = b"1234567890123450".to_vec(); // 128bit/16 + + let mut encrypt = + FileEncryptionProperties::builder(footer_key.clone()); + let mut decrypt = + FileDecryptionProperties::builder(footer_key.clone()); + + for field in schema.fields.iter() { + encrypt = encrypt.with_column_key(field.name().as_str(), column_key.clone()); + decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); + } + let encrypt = encrypt.build().unwrap(); + let decrypt = decrypt.build().unwrap(); + + // Write encrypted parquet + let props = WriterProperties::builder() + .with_file_encryption_properties(encrypt) + .build(); + + let tempdir = TempDir::new_in(Path::new(".")).unwrap(); + let tempfile = tempdir.path().join("data.parquet"); + let num_rows_written = write_batches(tempfile.clone(), props, batches).unwrap(); + + // Read encrypted parquet + let mut sc = SessionConfig::new(); + let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); + sc.options_mut().execution.parquet.file_decryption_properties = Some(fd); + + let state = SessionStateBuilder::new() + .with_config(sc) + .build(); + let ctx: SessionContext = SessionContext::new_with_state(state); + + let encrypted_batches = + read_parquet_test_data(tempfile.into_os_string().into_string().unwrap(), &ctx).await; + + let num_rows_read = encrypted_batches.iter().fold(0, |acc, x| acc + x.num_rows()); + + assert_eq!(num_rows_written, num_rows_read); +} \ No newline at end of file diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 87a5ed33f127d..c3c790e37574e 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -51,6 +51,7 @@ mod row_group_pruning; mod schema; mod schema_coercion; mod utils; +mod encryption; #[cfg(test)] #[ctor::ctor] diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 253bd8872dee2..e56cb81badd72 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -78,6 +78,7 @@ use parquet::arrow::arrow_writer::{ use parquet::arrow::async_reader::MetadataFetch; use parquet::arrow::{parquet_to_arrow_schema, ArrowSchemaConverter, AsyncArrowWriter}; use parquet::basic::Type; +use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::errors::ParquetError; use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder}; @@ -303,10 +304,11 @@ async fn fetch_schema_with_location( store: &dyn ObjectStore, file: &ObjectMeta, metadata_size_hint: Option, + file_decryption_properties: Option<&FileDecryptionProperties>, coerce_int96: Option, ) -> Result<(Path, Schema)> { let loc_path = file.location.clone(); - let schema = fetch_schema(store, file, metadata_size_hint, coerce_int96).await?; + let schema = fetch_schema(store, file, metadata_size_hint, file_decryption_properties, coerce_int96).await?; Ok((loc_path, schema)) } @@ -341,12 +343,23 @@ impl FileFormat for ParquetFormat { Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?), None => None, }; + let config_file_decryption_properties = &state.config().options() + .execution.parquet.file_decryption_properties.clone(); + let file_decryption_properties: Option = + match config_file_decryption_properties { + Some(cfd) => { + let fd: FileDecryptionProperties = cfd.clone().into(); + Some(fd) + }, + None => None, + }; let mut schemas: Vec<_> = futures::stream::iter(objects) .map(|object| { fetch_schema_with_location( store.as_ref(), object, self.metadata_size_hint(), + file_decryption_properties.as_ref(), coerce_int96, ) }) @@ -396,11 +409,21 @@ impl FileFormat for ParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { + let config_file_decryption_properties = &self.options.global.file_decryption_properties; + let file_decryption_properties: Option = + match config_file_decryption_properties { + Some(cfd) => { + let fd: FileDecryptionProperties = cfd.clone().into(); + Some(fd) + }, + None => None, + }; let stats = fetch_statistics( store.as_ref(), table_schema, object, self.metadata_size_hint(), + file_decryption_properties.as_ref(), ) .await?; Ok(stats) @@ -928,12 +951,14 @@ pub async fn fetch_parquet_metadata( store: &dyn ObjectStore, meta: &ObjectMeta, size_hint: Option, + decryption_properties: Option<&FileDecryptionProperties> ) -> Result { let file_size = meta.size; let fetch = ObjectStoreFetch::new(store, meta); ParquetMetaDataReader::new() .with_prefetch_hint(size_hint) + .with_decryption_properties(decryption_properties) .load_and_finish(fetch, file_size) .await .map_err(DataFusionError::from) @@ -944,9 +969,10 @@ async fn fetch_schema( store: &dyn ObjectStore, file: &ObjectMeta, metadata_size_hint: Option, + file_decryption_properties: Option<&FileDecryptionProperties>, coerce_int96: Option, ) -> Result { - let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?; + let metadata = fetch_parquet_metadata(store, file, metadata_size_hint, file_decryption_properties).await?; let file_metadata = metadata.file_metadata(); let schema = parquet_to_arrow_schema( file_metadata.schema_descr(), @@ -968,8 +994,9 @@ pub async fn fetch_statistics( table_schema: SchemaRef, file: &ObjectMeta, metadata_size_hint: Option, + decryption_properties: Option<&FileDecryptionProperties> ) -> Result { - let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?; + let metadata = fetch_parquet_metadata(store, file, metadata_size_hint, decryption_properties).await?; statistics_from_parquet_meta_calc(&metadata, table_schema) } From ec3f828d4003cf34c1f85d227a10311091bb8ef5 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 30 May 2025 16:43:36 -0700 Subject: [PATCH 05/58] cargo fmt --- .../core/tests/parquet/custom_reader.rs | 2 +- datafusion/core/tests/parquet/encryption.rs | 59 ++++++++++--------- datafusion/core/tests/parquet/mod.rs | 2 +- .../datasource-parquet/src/file_format.rs | 41 +++++++++---- 4 files changed, 64 insertions(+), 40 deletions(-) diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 3abaa61f2a830..5fc3513ff745b 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -241,7 +241,7 @@ impl AsyncFileReader for ParquetFileReader { self.store.as_ref(), &self.meta, self.metadata_size_hint, - None + None, ) .await .map_err(|e| { diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs index 4f90074470b19..44623c968eeec 100644 --- a/datafusion/core/tests/parquet/encryption.rs +++ b/datafusion/core/tests/parquet/encryption.rs @@ -26,25 +26,25 @@ //! select * from data limit 10; //! ``` +use arrow::record_batch::RecordBatch; +use datafusion::prelude::{ParquetReadOptions, SessionContext}; use std::fs::File; use std::path::{Path, PathBuf}; use std::sync::Arc; -use arrow::record_batch::RecordBatch; -use datafusion::prelude::{ - ParquetReadOptions, SessionContext, -}; +use datafusion::execution::SessionStateBuilder; +use datafusion_common::config::ConfigFileDecryptionProperties; +use datafusion_execution::config::SessionConfig; +use parquet::arrow::ArrowWriter; use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; -use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; use tempfile::TempDir; -use datafusion::execution::SessionStateBuilder; -use datafusion_common::config::ConfigFileDecryptionProperties; -use datafusion_execution::config::SessionConfig; - -async fn read_parquet_test_data>(path: T, ctx: &SessionContext) -> Vec { +async fn read_parquet_test_data>( + path: T, + ctx: &SessionContext, +) -> Vec { ctx.read_parquet(path.into(), ParquetReadOptions::default()) .await .unwrap() @@ -79,31 +79,30 @@ pub fn write_batches( #[tokio::test] async fn round_trip_encryption() { let ctx: SessionContext = SessionContext::new(); - + let batches = - read_parquet_test_data("tests/data/filter_pushdown/single_file.gz.parquet", &ctx).await; + read_parquet_test_data("tests/data/filter_pushdown/single_file.gz.parquet", &ctx) + .await; let schema = batches[0].schema(); let footer_key = b"0123456789012345".to_vec(); // 128bit/16 let column_key = b"1234567890123450".to_vec(); // 128bit/16 - let mut encrypt = - FileEncryptionProperties::builder(footer_key.clone()); - let mut decrypt = - FileDecryptionProperties::builder(footer_key.clone()); - + let mut encrypt = FileEncryptionProperties::builder(footer_key.clone()); + let mut decrypt = FileDecryptionProperties::builder(footer_key.clone()); + for field in schema.fields.iter() { encrypt = encrypt.with_column_key(field.name().as_str(), column_key.clone()); decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); } let encrypt = encrypt.build().unwrap(); let decrypt = decrypt.build().unwrap(); - + // Write encrypted parquet let props = WriterProperties::builder() .with_file_encryption_properties(encrypt) .build(); - + let tempdir = TempDir::new_in(Path::new(".")).unwrap(); let tempfile = tempdir.path().join("data.parquet"); let num_rows_written = write_batches(tempfile.clone(), props, batches).unwrap(); @@ -111,17 +110,21 @@ async fn round_trip_encryption() { // Read encrypted parquet let mut sc = SessionConfig::new(); let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); - sc.options_mut().execution.parquet.file_decryption_properties = Some(fd); - - let state = SessionStateBuilder::new() - .with_config(sc) - .build(); + sc.options_mut() + .execution + .parquet + .file_decryption_properties = Some(fd); + + let state = SessionStateBuilder::new().with_config(sc).build(); let ctx: SessionContext = SessionContext::new_with_state(state); let encrypted_batches = - read_parquet_test_data(tempfile.into_os_string().into_string().unwrap(), &ctx).await; + read_parquet_test_data(tempfile.into_os_string().into_string().unwrap(), &ctx) + .await; + + let num_rows_read = encrypted_batches + .iter() + .fold(0, |acc, x| acc + x.num_rows()); - let num_rows_read = encrypted_batches.iter().fold(0, |acc, x| acc + x.num_rows()); - assert_eq!(num_rows_written, num_rows_read); -} \ No newline at end of file +} diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index c3c790e37574e..835e8baedbb2f 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -43,6 +43,7 @@ use std::sync::Arc; use tempfile::NamedTempFile; mod custom_reader; +mod encryption; mod external_access_plan; mod file_statistics; mod filter_pushdown; @@ -51,7 +52,6 @@ mod row_group_pruning; mod schema; mod schema_coercion; mod utils; -mod encryption; #[cfg(test)] #[ctor::ctor] diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index e56cb81badd72..2aef81f6a0f88 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -308,7 +308,14 @@ async fn fetch_schema_with_location( coerce_int96: Option, ) -> Result<(Path, Schema)> { let loc_path = file.location.clone(); - let schema = fetch_schema(store, file, metadata_size_hint, file_decryption_properties, coerce_int96).await?; + let schema = fetch_schema( + store, + file, + metadata_size_hint, + file_decryption_properties, + coerce_int96, + ) + .await?; Ok((loc_path, schema)) } @@ -343,14 +350,19 @@ impl FileFormat for ParquetFormat { Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?), None => None, }; - let config_file_decryption_properties = &state.config().options() - .execution.parquet.file_decryption_properties.clone(); + let config_file_decryption_properties = &state + .config() + .options() + .execution + .parquet + .file_decryption_properties + .clone(); let file_decryption_properties: Option = match config_file_decryption_properties { Some(cfd) => { let fd: FileDecryptionProperties = cfd.clone().into(); Some(fd) - }, + } None => None, }; let mut schemas: Vec<_> = futures::stream::iter(objects) @@ -409,13 +421,14 @@ impl FileFormat for ParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { - let config_file_decryption_properties = &self.options.global.file_decryption_properties; + let config_file_decryption_properties = + &self.options.global.file_decryption_properties; let file_decryption_properties: Option = match config_file_decryption_properties { Some(cfd) => { let fd: FileDecryptionProperties = cfd.clone().into(); Some(fd) - }, + } None => None, }; let stats = fetch_statistics( @@ -951,7 +964,7 @@ pub async fn fetch_parquet_metadata( store: &dyn ObjectStore, meta: &ObjectMeta, size_hint: Option, - decryption_properties: Option<&FileDecryptionProperties> + decryption_properties: Option<&FileDecryptionProperties>, ) -> Result { let file_size = meta.size; let fetch = ObjectStoreFetch::new(store, meta); @@ -972,7 +985,13 @@ async fn fetch_schema( file_decryption_properties: Option<&FileDecryptionProperties>, coerce_int96: Option, ) -> Result { - let metadata = fetch_parquet_metadata(store, file, metadata_size_hint, file_decryption_properties).await?; + let metadata = fetch_parquet_metadata( + store, + file, + metadata_size_hint, + file_decryption_properties, + ) + .await?; let file_metadata = metadata.file_metadata(); let schema = parquet_to_arrow_schema( file_metadata.schema_descr(), @@ -994,9 +1013,11 @@ pub async fn fetch_statistics( table_schema: SchemaRef, file: &ObjectMeta, metadata_size_hint: Option, - decryption_properties: Option<&FileDecryptionProperties> + decryption_properties: Option<&FileDecryptionProperties>, ) -> Result { - let metadata = fetch_parquet_metadata(store, file, metadata_size_hint, decryption_properties).await?; + let metadata = + fetch_parquet_metadata(store, file, metadata_size_hint, decryption_properties) + .await?; statistics_from_parquet_meta_calc(&metadata, table_schema) } From 3538a27ab02ebecf6a48353aa1d2ec6d59dd507b Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 30 May 2025 17:08:05 -0700 Subject: [PATCH 06/58] Update test to add decryption parameter to called functions. --- .../src/datasource/file_format/parquet.rs | 42 +++++++++++++------ 1 file changed, 29 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 9705225c24c7b..a8042770b40a5 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -196,7 +196,8 @@ mod tests { let schema = format.infer_schema(&ctx, &store, &meta).await.unwrap(); let stats = - fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None).await?; + fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None, None) + .await?; assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; @@ -204,7 +205,8 @@ mod tests { assert_eq!(c1_stats.null_count, Precision::Exact(1)); assert_eq!(c2_stats.null_count, Precision::Exact(3)); - let stats = fetch_statistics(store.as_ref(), schema, &meta[1], None).await?; + let stats = + fetch_statistics(store.as_ref(), schema, &meta[1], None, None).await?; assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; @@ -376,9 +378,14 @@ mod tests { // Use a size hint larger than the parquet footer but smaller than the actual metadata, requiring a second fetch // for the remaining metadata - fetch_parquet_metadata(store.as_ref() as &dyn ObjectStore, &meta[0], Some(9)) - .await - .expect("error reading metadata with hint"); + fetch_parquet_metadata( + store.as_ref() as &dyn ObjectStore, + &meta[0], + Some(9), + None, + ) + .await + .expect("error reading metadata with hint"); assert_eq!(store.request_count(), 2); @@ -396,9 +403,14 @@ mod tests { .await .unwrap(); - let stats = - fetch_statistics(store.upcast().as_ref(), schema.clone(), &meta[0], Some(9)) - .await?; + let stats = fetch_statistics( + store.upcast().as_ref(), + schema.clone(), + &meta[0], + Some(9), + None, + ) + .await?; assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; @@ -413,7 +425,7 @@ mod tests { // Use the file size as the hint so we can get the full metadata from the first fetch let size_hint = meta[0].size as usize; - fetch_parquet_metadata(store.upcast().as_ref(), &meta[0], Some(size_hint)) + fetch_parquet_metadata(store.upcast().as_ref(), &meta[0], Some(size_hint), None) .await .expect("error reading metadata with hint"); @@ -432,6 +444,7 @@ mod tests { schema.clone(), &meta[0], Some(size_hint), + None, ) .await?; @@ -448,7 +461,7 @@ mod tests { // Use the a size hint larger than the file size to make sure we don't panic let size_hint = (meta[0].size + 100) as usize; - fetch_parquet_metadata(store.upcast().as_ref(), &meta[0], Some(size_hint)) + fetch_parquet_metadata(store.upcast().as_ref(), &meta[0], Some(size_hint), None) .await .expect("error reading metadata with hint"); @@ -487,7 +500,8 @@ mod tests { let schema = format.infer_schema(&state, &store, &files).await.unwrap(); // Fetch statistics for first file - let pq_meta = fetch_parquet_metadata(store.as_ref(), &files[0], None).await?; + let pq_meta = + fetch_parquet_metadata(store.as_ref(), &files[0], None, None).await?; let stats = statistics_from_parquet_meta_calc(&pq_meta, schema.clone())?; assert_eq!(stats.num_rows, Precision::Exact(4)); @@ -545,7 +559,8 @@ mod tests { }; // Fetch statistics for first file - let pq_meta = fetch_parquet_metadata(store.as_ref(), &files[0], None).await?; + let pq_meta = + fetch_parquet_metadata(store.as_ref(), &files[0], None, None).await?; let stats = statistics_from_parquet_meta_calc(&pq_meta, schema.clone())?; assert_eq!(stats.num_rows, Precision::Exact(3)); // column c1 @@ -571,7 +586,8 @@ mod tests { assert_eq!(c2_stats.min_value, Precision::Exact(null_i64.clone())); // Fetch statistics for second file - let pq_meta = fetch_parquet_metadata(store.as_ref(), &files[1], None).await?; + let pq_meta = + fetch_parquet_metadata(store.as_ref(), &files[1], None, None).await?; let stats = statistics_from_parquet_meta_calc(&pq_meta, schema.clone())?; assert_eq!(stats.num_rows, Precision::Exact(3)); // column c1: missing from the file so the table treats all 3 rows as null From a75499221bf1bbc14dcb847d7ef6645b1097e54b Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 2 Jun 2025 18:05:26 -0700 Subject: [PATCH 07/58] Try to get DataFrame.write_parquet to work with encryption. Doesn't quite, column encryption is broken. --- datafusion/core/src/dataframe/parquet.rs | 135 +++++++++++++++++++++++ 1 file changed, 135 insertions(+) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 1bb5444ca009f..18a7dbfe96d88 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -103,6 +103,7 @@ impl DataFrame { #[cfg(test)] mod tests { use std::collections::HashMap; + use std::path::PathBuf; use std::sync::Arc; use super::super::Result; @@ -117,6 +118,8 @@ mod tests { use datafusion_expr::{col, lit}; use object_store::local::LocalFileSystem; + use parquet::arrow::ArrowWriter; + use parquet::file::properties::WriterProperties; use parquet::file::reader::FileReader; use tempfile::TempDir; use url::Url; @@ -246,4 +249,136 @@ mod tests { Ok(()) } + + async fn read_parquet_test_data>( + path: T, + ctx: &SessionContext, + ) -> Vec { + ctx.read_parquet(path.into(), ParquetReadOptions::default()) + .await + .unwrap() + .collect() + .await + .unwrap() + } + + pub fn write_batches( + path: PathBuf, + props: WriterProperties, + batches: impl IntoIterator, + ) -> Result { + let mut batches = batches.into_iter(); + let first_batch = batches.next().expect("need at least one record batch"); + let schema = first_batch.schema(); + + let file = std::fs::File::create(&path)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props))?; + + writer.write(&first_batch)?; + let mut num_rows = first_batch.num_rows(); + + for batch in batches { + writer.write(&batch)?; + num_rows += batch.num_rows(); + } + writer.close()?; + Ok(num_rows) + } + + #[tokio::test] + async fn roundtrip_parquet_with_encryption() -> Result<()> { + use parquet::encryption::decrypt::FileDecryptionProperties; + use parquet::encryption::encrypt::FileEncryptionProperties; + use datafusion_common::config::ConfigFileDecryptionProperties; + use crate::execution::SessionStateBuilder; + + let test_df = test_util::test_table().await?; + + let schema = test_df.schema(); + let footer_key = b"0123456789012345".to_vec(); // 128bit/16 + let column_key = b"1234567890123450".to_vec(); // 128bit/16 + + let mut encrypt = FileEncryptionProperties::builder(footer_key.clone()); + let mut decrypt = FileDecryptionProperties::builder(footer_key.clone()); + + for field in schema.fields().iter() { + encrypt = encrypt.with_column_key(field.name().as_str(), column_key.clone()); + decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); + } + + let encrypt = encrypt.build().unwrap(); + let decrypt = decrypt.build().unwrap(); + + let df = test_df.clone(); + let tmp_dir = TempDir::new()?; + let tempfile = tmp_dir.path().join("roundtrip.parquet"); + let output_path = "file://local/roundtrip.parquet"; + + let num_rows_written: usize = if true { + // Write encrypted parquet using ArrowWriter + let props = WriterProperties::builder() + .with_file_encryption_properties(encrypt) + .build(); + + let batches = df.collect().await?; + write_batches(tempfile.clone(), props, batches).unwrap() + } else { + // Write encrypted parquet using write_parquet + let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); + let local_url = Url::parse("file://local").unwrap(); + let ctx = &test_df.session_state; + ctx.runtime_env().register_object_store(&local_url, local); + let mut options = TableParquetOptions::default(); + options.global.file_encryption_properties = Some(encrypt.clone().into()); + df.write_parquet( + output_path, + DataFrameWriteOptions::new().with_single_file_output(true), + Some(options), + ) + .await?; + test_df.count().await? + }; + + + // Check that file actually used encryption + // TODO, I guess arrow-rs does not have a serialized file reader that supports encryption? + /* + let file = std::fs::File::open(tmp_dir.path().join("test.parquet"))?; + + let reader = + parquet::file::serialized_reader::SerializedFileReader::new(file) + .unwrap(); + + let parquet_metadata = reader.metadata(); + + let written_compression = + parquet_metadata.row_group(0).column(0).compression(); + + assert_eq!(written_compression, parse_compression_string(compression)?); + */ + + // Read encrypted parquet + let mut sc = SessionConfig::new(); + let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); + sc.options_mut() + .execution + .parquet + .file_decryption_properties = Some(fd); + + let state = SessionStateBuilder::new().with_config(sc).build(); + let ctx: SessionContext = SessionContext::new_with_state(state); + + let encrypted_batches = + read_parquet_test_data(tempfile.into_os_string().into_string().unwrap(), &ctx) + .await; + + let num_rows_read = encrypted_batches + .iter() + .fold(0, |acc, x| acc + x.num_rows()); + + + assert_eq!(num_rows_read as usize, num_rows_written as usize); + + Ok(()) + } } From e4306723d86c9812695fbecc901206b60db8946a Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 3 Jun 2025 18:12:09 -0700 Subject: [PATCH 08/58] Update datafusion/datasource-parquet/src/opener.rs Co-authored-by: Adam Reeve --- datafusion/datasource-parquet/src/opener.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index f97bd193e83b4..5659bec0dcf54 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -135,8 +135,8 @@ impl FileOpener for ParquetOpener { // pruning predicates. Thus default to not requesting if from the // underlying reader. let mut options = ArrowReaderOptions::new().with_page_index(false); - if let Some(ref fd_val) = file_decryption_properties { - options = options.with_file_decryption_properties((**fd_val).clone()); + if let Some(fd_val) = file_decryption_properties { + options = options.with_file_decryption_properties((*fd_val).clone()); } let mut metadata_timer = file_metrics.metadata_load_time.timer(); From 7fcba70f38045e48ebccf434b08c6f335e175026 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 3 Jun 2025 18:13:35 -0700 Subject: [PATCH 09/58] Update datafusion/datasource-parquet/src/source.rs Co-authored-by: Adam Reeve --- datafusion/datasource-parquet/src/source.rs | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 17f53a4d8bd55..35bf5c8d82606 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -467,22 +467,12 @@ impl FileSource for ParquetSource { Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _ }); - let mut file_decryption_properties: Option> = None; - if self + let file_decryption_properties = self .table_parquet_options() .global .file_decryption_properties - .is_some() - { - let fdp: FileDecryptionProperties = self - .table_parquet_options() - .global - .file_decryption_properties - .clone() - .unwrap() - .into(); - file_decryption_properties = Some(Arc::new(fdp)); - } + .as_ref() + .map(|props| Arc::new(props.clone().into())); let coerce_int96 = self .table_parquet_options From d6b1fcad13d86e9de74517b14a76508bf02fddd7 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 11:55:25 -0700 Subject: [PATCH 10/58] Fix write test in parquet.rs --- datafusion/core/src/dataframe/parquet.rs | 62 ++++++++---------------- 1 file changed, 19 insertions(+), 43 deletions(-) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 18a7dbfe96d88..aa01f52488bbd 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -300,7 +300,7 @@ mod tests { let mut encrypt = FileEncryptionProperties::builder(footer_key.clone()); let mut decrypt = FileDecryptionProperties::builder(footer_key.clone()); - + for field in schema.fields().iter() { encrypt = encrypt.with_column_key(field.name().as_str(), column_key.clone()); decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); @@ -314,49 +314,26 @@ mod tests { let tempfile = tmp_dir.path().join("roundtrip.parquet"); let output_path = "file://local/roundtrip.parquet"; - let num_rows_written: usize = if true { - // Write encrypted parquet using ArrowWriter - let props = WriterProperties::builder() - .with_file_encryption_properties(encrypt) - .build(); - - let batches = df.collect().await?; - write_batches(tempfile.clone(), props, batches).unwrap() - } else { - // Write encrypted parquet using write_parquet - let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); - let local_url = Url::parse("file://local").unwrap(); - let ctx = &test_df.session_state; - ctx.runtime_env().register_object_store(&local_url, local); - let mut options = TableParquetOptions::default(); - options.global.file_encryption_properties = Some(encrypt.clone().into()); - df.write_parquet( - output_path, - DataFrameWriteOptions::new().with_single_file_output(true), - Some(options), - ) - .await?; - test_df.count().await? - }; + + // Write encrypted parquet using write_parquet + let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); + let local_url = Url::parse("file://local").unwrap(); + let ctx = &test_df.session_state; + ctx.runtime_env().register_object_store(&local_url, local); + let mut options = TableParquetOptions::default(); + options.global.file_encryption_properties = Some(encrypt.clone().into()); + // Parallel writing for encryption is broken right now. + // Rok is working on it. + options.global.allow_single_file_parallelism = false; + df.write_parquet( + output_path, + DataFrameWriteOptions::new().with_single_file_output(true), + Some(options), + ) + .await?; + let num_rows_written: usize = test_df.count().await?; - // Check that file actually used encryption - // TODO, I guess arrow-rs does not have a serialized file reader that supports encryption? - /* - let file = std::fs::File::open(tmp_dir.path().join("test.parquet"))?; - - let reader = - parquet::file::serialized_reader::SerializedFileReader::new(file) - .unwrap(); - - let parquet_metadata = reader.metadata(); - - let written_compression = - parquet_metadata.row_group(0).column(0).compression(); - - assert_eq!(written_compression, parse_compression_string(compression)?); - */ - // Read encrypted parquet let mut sc = SessionConfig::new(); let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); @@ -376,7 +353,6 @@ mod tests { .iter() .fold(0, |acc, x| acc + x.num_rows()); - assert_eq!(num_rows_read as usize, num_rows_written as usize); Ok(()) From 3353186eb4be767e80be99c91dfd03bf09a67046 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 12:19:53 -0700 Subject: [PATCH 11/58] Simplify encryption test. Remove unused imports. --- datafusion/core/src/dataframe/parquet.rs | 69 +++++---------------- datafusion/datasource-parquet/src/source.rs | 1 - 2 files changed, 15 insertions(+), 55 deletions(-) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index aa01f52488bbd..c345d4350909c 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -103,7 +103,6 @@ impl DataFrame { #[cfg(test)] mod tests { use std::collections::HashMap; - use std::path::PathBuf; use std::sync::Arc; use super::super::Result; @@ -118,8 +117,6 @@ mod tests { use datafusion_expr::{col, lit}; use object_store::local::LocalFileSystem; - use parquet::arrow::ArrowWriter; - use parquet::file::properties::WriterProperties; use parquet::file::reader::FileReader; use tempfile::TempDir; use url::Url; @@ -249,41 +246,7 @@ mod tests { Ok(()) } - - async fn read_parquet_test_data>( - path: T, - ctx: &SessionContext, - ) -> Vec { - ctx.read_parquet(path.into(), ParquetReadOptions::default()) - .await - .unwrap() - .collect() - .await - .unwrap() - } - - pub fn write_batches( - path: PathBuf, - props: WriterProperties, - batches: impl IntoIterator, - ) -> Result { - let mut batches = batches.into_iter(); - let first_batch = batches.next().expect("need at least one record batch"); - let schema = first_batch.schema(); - - let file = std::fs::File::create(&path)?; - let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props))?; - - writer.write(&first_batch)?; - let mut num_rows = first_batch.num_rows(); - - for batch in batches { - writer.write(&batch)?; - num_rows += batch.num_rows(); - } - writer.close()?; - Ok(num_rows) - } + #[tokio::test] async fn roundtrip_parquet_with_encryption() -> Result<()> { @@ -312,28 +275,22 @@ mod tests { let df = test_df.clone(); let tmp_dir = TempDir::new()?; let tempfile = tmp_dir.path().join("roundtrip.parquet"); - let output_path = "file://local/roundtrip.parquet"; - + let tempfile_str = tempfile.into_os_string().into_string().unwrap(); // Write encrypted parquet using write_parquet - let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); - let local_url = Url::parse("file://local").unwrap(); - let ctx = &test_df.session_state; - ctx.runtime_env().register_object_store(&local_url, local); let mut options = TableParquetOptions::default(); options.global.file_encryption_properties = Some(encrypt.clone().into()); // Parallel writing for encryption is broken right now. // Rok is working on it. options.global.allow_single_file_parallelism = false; df.write_parquet( - output_path, + tempfile_str.as_str(), DataFrameWriteOptions::new().with_single_file_output(true), Some(options), ) .await?; let num_rows_written: usize = test_df.count().await?; - // Read encrypted parquet let mut sc = SessionConfig::new(); let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); @@ -345,14 +302,18 @@ mod tests { let state = SessionStateBuilder::new().with_config(sc).build(); let ctx: SessionContext = SessionContext::new_with_state(state); - let encrypted_batches = - read_parquet_test_data(tempfile.into_os_string().into_string().unwrap(), &ctx) - .await; - - let num_rows_read = encrypted_batches - .iter() - .fold(0, |acc, x| acc + x.num_rows()); - + ctx.register_parquet( + "roundtrip_parquet", + &tempfile_str, + ParquetReadOptions::default(), + ) + .await?; + + let df_enc = ctx.sql( + "SELECT * FROM roundtrip_parquet", + ).await?; + let num_rows_read = df_enc.count().await?; + assert_eq!(num_rows_read as usize, num_rows_written as usize); Ok(()) diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 35bf5c8d82606..7e022aa979bff 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -49,7 +49,6 @@ use datafusion_physical_plan::DisplayFormatType; use itertools::Itertools; use object_store::ObjectStore; -use parquet::encryption::decrypt::FileDecryptionProperties; /// Execution plan for reading one or more Parquet files. /// From e4bc0e31bdbd227b8be5171e57f4a060935b97c1 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 12:21:48 -0700 Subject: [PATCH 12/58] Run cargo fmt. --- datafusion/core/src/dataframe/parquet.rs | 31 +++++++++++------------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index c345d4350909c..d9dface34f8db 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -246,15 +246,14 @@ mod tests { Ok(()) } - #[tokio::test] async fn roundtrip_parquet_with_encryption() -> Result<()> { + use crate::execution::SessionStateBuilder; + use datafusion_common::config::ConfigFileDecryptionProperties; use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; - use datafusion_common::config::ConfigFileDecryptionProperties; - use crate::execution::SessionStateBuilder; - + let test_df = test_util::test_table().await?; let schema = test_df.schema(); @@ -263,20 +262,20 @@ mod tests { let mut encrypt = FileEncryptionProperties::builder(footer_key.clone()); let mut decrypt = FileDecryptionProperties::builder(footer_key.clone()); - + for field in schema.fields().iter() { encrypt = encrypt.with_column_key(field.name().as_str(), column_key.clone()); decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); } - + let encrypt = encrypt.build().unwrap(); let decrypt = decrypt.build().unwrap(); - + let df = test_df.clone(); let tmp_dir = TempDir::new()?; let tempfile = tmp_dir.path().join("roundtrip.parquet"); let tempfile_str = tempfile.into_os_string().into_string().unwrap(); - + // Write encrypted parquet using write_parquet let mut options = TableParquetOptions::default(); options.global.file_encryption_properties = Some(encrypt.clone().into()); @@ -288,9 +287,9 @@ mod tests { DataFrameWriteOptions::new().with_single_file_output(true), Some(options), ) - .await?; + .await?; let num_rows_written: usize = test_df.count().await?; - + // Read encrypted parquet let mut sc = SessionConfig::new(); let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); @@ -301,19 +300,17 @@ mod tests { let state = SessionStateBuilder::new().with_config(sc).build(); let ctx: SessionContext = SessionContext::new_with_state(state); - + ctx.register_parquet( "roundtrip_parquet", &tempfile_str, ParquetReadOptions::default(), ) - .await?; - - let df_enc = ctx.sql( - "SELECT * FROM roundtrip_parquet", - ).await?; + .await?; + + let df_enc = ctx.sql("SELECT * FROM roundtrip_parquet").await?; let num_rows_read = df_enc.count().await?; - + assert_eq!(num_rows_read as usize, num_rows_written as usize); Ok(()) From f52e79c34a77c83f3258bd7437a097781cfdd6ab Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 12:25:18 -0700 Subject: [PATCH 13/58] Further streamline roundtrip test. --- datafusion/core/src/dataframe/parquet.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index d9dface34f8db..690c933598a5a 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -268,8 +268,8 @@ mod tests { decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); } - let encrypt = encrypt.build().unwrap(); - let decrypt = decrypt.build().unwrap(); + let encrypt = encrypt.build()?; + let decrypt = decrypt.build()?; let df = test_df.clone(); let tmp_dir = TempDir::new()?; @@ -288,7 +288,7 @@ mod tests { Some(options), ) .await?; - let num_rows_written: usize = test_df.count().await?; + let num_rows_written = test_df.count().await?; // Read encrypted parquet let mut sc = SessionConfig::new(); @@ -311,7 +311,7 @@ mod tests { let df_enc = ctx.sql("SELECT * FROM roundtrip_parquet").await?; let num_rows_read = df_enc.count().await?; - assert_eq!(num_rows_read as usize, num_rows_written as usize); + assert_eq!(num_rows_read, num_rows_written); Ok(()) } From 5615ac8846ca9ba948ad1487d15796625ad48757 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 14:39:09 -0700 Subject: [PATCH 14/58] Change From methods for FileEncryptionProperties and FileDecryptionProperties to use references. --- datafusion/common/src/config.rs | 19 ++++++++----------- .../common/src/file_options/parquet_writer.rs | 2 +- datafusion/core/src/dataframe/parquet.rs | 4 ++-- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ccec20e99deae..ec1ceafb26857 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -307,8 +307,8 @@ impl Into for ConfigFileDecryptionProperties { } #[cfg(feature = "parquet")] -impl From for ConfigFileDecryptionProperties { - fn from(f: FileDecryptionProperties) -> Self { +impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { + fn from(f: &FileDecryptionProperties) -> Self { let (column_names_vec, column_keys_vec) = f.column_keys(); let column_keys_meta = Vec::new(); let ck = EncryptionColumnKeys::new( @@ -374,8 +374,8 @@ impl Into for ConfigFileEncryptionProperties { } #[cfg(feature = "parquet")] -impl From for ConfigFileEncryptionProperties { - fn from(f: FileEncryptionProperties) -> Self { +impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { + fn from(f: &FileEncryptionProperties) -> Self { let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); let ck = EncryptionColumnKeys::new( &column_names_vec, @@ -2312,11 +2312,7 @@ impl Display for OutputFormat { #[cfg(test)] mod tests { - use crate::config::{ - ConfigEntry, ConfigExtension, ConfigField, ConfigFileDecryptionProperties, - ConfigFileEncryptionProperties, ConfigFileType, ExtensionOptions, Extensions, - TableOptions, - }; + use crate::config::{ConfigEntry, ConfigExtension, ConfigField, ConfigFileType, ExtensionOptions, Extensions, TableOptions}; use std::any::Any; use std::collections::HashMap; @@ -2452,6 +2448,7 @@ mod tests { fn parquet_table_encryption() { use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; + use crate::config::{ConfigFileDecryptionProperties, ConfigFileEncryptionProperties}; let footer_key = b"0123456789012345".to_vec(); // 128bit/16 let column_names = vec!["double_field", "float_field"]; @@ -2473,13 +2470,13 @@ mod tests { // Test round-trip let config_encrypt: ConfigFileEncryptionProperties = - file_encryption_properties.clone().into(); + (&file_encryption_properties).into(); let encryption_properties_built: FileEncryptionProperties = config_encrypt.clone().into(); assert_eq!(file_encryption_properties, encryption_properties_built); let config_decrypt: ConfigFileDecryptionProperties = - decryption_properties.clone().into(); + (&decryption_properties).into(); let decryption_properties_built: FileDecryptionProperties = config_decrypt.clone().into(); assert_eq!(decryption_properties, decryption_properties_built); diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index d7f1617f7a4d0..69576a5726ea5 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -599,7 +599,7 @@ mod tests { let fep: Option = match props.file_encryption_properties() { - Some(fe) => Some(fe.clone().into()), + Some(fe) => Some(fe.into()), None => None, }; diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 690c933598a5a..ab4a6e12887b9 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -278,7 +278,7 @@ mod tests { // Write encrypted parquet using write_parquet let mut options = TableParquetOptions::default(); - options.global.file_encryption_properties = Some(encrypt.clone().into()); + options.global.file_encryption_properties = Some((&encrypt).into()); // Parallel writing for encryption is broken right now. // Rok is working on it. options.global.allow_single_file_parallelism = false; @@ -292,7 +292,7 @@ mod tests { // Read encrypted parquet let mut sc = SessionConfig::new(); - let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); + let fd: ConfigFileDecryptionProperties = (&decrypt).into(); sc.options_mut() .execution .parquet From 61bc78e32f7e58ff5b4f6315a6b1c21cebc69c98 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 20:08:28 -0700 Subject: [PATCH 15/58] Change encryption config to directly hold column keys using custom config fields. --- datafusion/common/src/config.rs | 433 ++++++++++++++++++++------------ 1 file changed, 279 insertions(+), 154 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ec1ceafb26857..c03b7b898c52f 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -32,7 +32,6 @@ use hex; use parquet::encryption::decrypt::FileDecryptionProperties; #[cfg(feature = "parquet")] use parquet::encryption::encrypt::FileEncryptionProperties; -use serde::{Deserialize, Serialize}; /// A macro that wraps a configuration struct and automatically derives /// [`Default`] and [`ConfigField`] for it, allowing it to be used @@ -195,138 +194,109 @@ macro_rules! config_namespace { } } -#[derive(Serialize, Deserialize, Debug)] -pub struct EncryptionKeyAsHex { - pub key: String, - pub key_metadata: Option, -} - -#[derive(Debug)] -pub struct EncryptionKey { - pub key: Vec, - pub key_metadata: Option>, -} - -#[derive(Serialize, Deserialize, Debug)] -pub struct EncryptionColumnKeys { - pub column_keys_as_hex: HashMap, -} - -impl EncryptionColumnKeys { - pub fn new( - column_names: &Vec, - keys: &Vec>, - meta: &Vec>, - ) -> Self { - let mut column_keys_as_hex: HashMap = HashMap::new(); - - for (i, column_name) in column_names.iter().enumerate() { - let key_metadata: Option = meta.get(i).map(|x| hex::encode(x)); - let encryption_key = EncryptionKeyAsHex { - key: hex::encode(keys[i].clone()), - key_metadata, - }; - column_keys_as_hex.insert(column_name.clone(), encryption_key); - } - - EncryptionColumnKeys { column_keys_as_hex } - } - - pub fn to_json(&self) -> String { - serde_json::to_string(self).unwrap() - } - - pub fn from_json_to_column_keys(s: &String) -> HashMap { - let eck: EncryptionColumnKeys = - serde_json::from_str(s).expect("failed to decode column keys from JSON"); - let mut hm: HashMap = HashMap::new(); - for (key, val) in eck.column_keys_as_hex { - let col = key; - let key = hex::decode(val.key).expect("Invalid column key"); - let key_metadata = val - .key_metadata - .map(|x| hex::decode(x).expect("Invalid column metadata")); - let encryption_key = EncryptionKey { key, key_metadata }; - hm.insert(col, encryption_key); - } - hm - } -} - -config_namespace! { - pub struct ConfigFileEncryptionProperties { - pub encrypt_footer: bool, default = false - pub footer_key_as_hex: String, default = String::new() - pub footer_key_metadata_as_hex: String, default = String::new() - pub column_keys_as_json_hex: String, default = String::new() - pub aad_prefix_as_hex: String, default = String::new() - pub store_aad_prefix: bool, default = false - } +#[derive(Clone, Default, Debug, PartialEq)] +pub struct ConfigFileEncryptionProperties { + pub encrypt_footer: bool, // default = false + pub footer_key_as_hex: String, + pub footer_key_metadata_as_hex: String, + pub column_keys_as_hex: HashMap, + pub column_metadata_as_hex: HashMap, + pub aad_prefix_as_hex: String, + pub store_aad_prefix: bool, // default = false } -config_namespace! { - pub struct ConfigFileDecryptionProperties { - pub footer_key_as_hex: String, default = String::new() - pub column_keys_as_json_hex: String, default = String::new() - pub aad_prefix_as_hex: String, default = String::new() - pub footer_signature_verification: bool, default = true +impl ConfigFileEncryptionProperties { + /// Return new default TableParquetOptions + pub fn new() -> Self { + Self::default() } } -#[cfg(feature = "parquet")] -impl Into for ConfigFileDecryptionProperties { - fn into(self) -> FileDecryptionProperties { - let eck = - EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); - let mut column_names: Vec<&str> = Vec::new(); - let mut column_keys: Vec> = Vec::new(); - if !eck.is_empty() { - for (col_name, encryption_key) in eck.iter() { - column_names.push(col_name.as_str()); - column_keys.push(encryption_key.key.clone()); - } +impl ConfigField for ConfigFileEncryptionProperties { + fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { + let key = format!("{}.encrypt_footer", key_prefix); + let desc = "Encrypt the footer"; + self.encrypt_footer.visit(v, key.as_str(), desc); + + let key = format!("{}.footer_key_as_hex", key_prefix); + let desc = "Key to use for the parquet footer"; + self.footer_key_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{}.footer_key_metadata_as_hex", key_prefix); + let desc = "Metadata to use for the parquet footer"; + self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc); + + let desc = "Per column encryption keys"; + for (col_name, col_val) in self.column_keys_as_hex.iter() { + let key = format!("{}.column_keys_as_hex::{}", key_prefix, col_name); + col_val.visit(v, key.as_str(), desc); } - let mut fep = FileDecryptionProperties::builder( - hex::decode(self.footer_key_as_hex).unwrap(), - ) - .with_column_keys(column_names, column_keys) - .unwrap(); - - if !self.footer_signature_verification { - fep = fep.disable_footer_signature_verification(); + + let desc = "Per column metadata"; + for (col_name, col_val) in self.column_metadata_as_hex.iter() { + let key = format!("{}.column_metadata_as_hex::{}", key_prefix, col_name); + col_val.visit(v, key.as_str(), desc); } - if self.aad_prefix_as_hex.len() > 0 { - let aad_prefix = - hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); - fep = fep.with_aad_prefix(aad_prefix); - } + let key = format!("{}.aad_prefix_as_hex", key_prefix); + let desc = "AAD prefix to use"; + self.aad_prefix_as_hex.visit(v, key.as_str(), desc); - fep.build().unwrap() + let key = format!("{}.store_aad_prefix", key_prefix); + let desc = "If true, store the AAD prefix"; + self.store_aad_prefix.visit(v, key.as_str(), desc); } -} + + fn set(&mut self, key: &str, value: &str) -> Result<()> { + // Any hex encoded values must be pre-encoded using + // hex::encode() before calling set. + if key.starts_with("column_keys_as_hex.") { + let k = match key.split(".").collect::>()[..] { + [_meta] | [_meta, ""] => { + return _config_err!( + "Invalid column name provided, missing name in column_keys_as_hex." + ) + } + [_meta, k] => k.into(), + _ => { + return _config_err!( + "Invalid column name provided, found too many '.' in \"{key}\"" + ) + } + }; + self.column_keys_as_hex.insert(k, String::from(value)); + return Ok(()) + }; -#[cfg(feature = "parquet")] -impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { - fn from(f: &FileDecryptionProperties) -> Self { - let (column_names_vec, column_keys_vec) = f.column_keys(); - let column_keys_meta = Vec::new(); - let ck = EncryptionColumnKeys::new( - &column_names_vec, - &column_keys_vec, - &column_keys_meta, - ); - let mut aad_prefix: Vec = Vec::new(); - if let Some(prefix) = f.aad_prefix() { - aad_prefix = prefix.clone(); - } - ConfigFileDecryptionProperties { - footer_key_as_hex: hex::encode( - f.footer_key(None).unwrap_or_default().as_ref(), - ), - column_keys_as_json_hex: ck.to_json(), - aad_prefix_as_hex: hex::encode(aad_prefix), - footer_signature_verification: f.check_plaintext_footer_integrity(), + if key.starts_with("column_metadata_as_hex.") { + let k = match key.split(".").collect::>()[..] { + [_meta] | [_meta, ""] => { + return _config_err!( + "Invalid column name provided, missing name in column_metadata_as_hex." + ) + } + [_meta, k] => k.into(), + _ => { + return _config_err!( + "Invalid column name provided, found too many '.' in \"{key}\"" + ) + } + }; + self.column_metadata_as_hex.insert(k, String::from(value)); + return Ok(()) + }; + + let (key, rem) = key.split_once('.').unwrap_or((key, "")); + match key { + "encrypt_footer" => self.encrypt_footer.set(rem, value.as_ref()), + "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), + "footer_key_metadata_as_hex" => self.footer_key_metadata_as_hex.set(rem, value.as_ref()), + "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), + "store_aad_prefix" => self.store_aad_prefix.set(rem, value.as_ref()), + _ => _config_err!( + "Config value \"{}\" not found on ConfigFileEncryptionProperties", + key + ), } } } @@ -334,13 +304,11 @@ impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { #[cfg(feature = "parquet")] impl Into for ConfigFileEncryptionProperties { fn into(self) -> FileEncryptionProperties { - let eck = - EncryptionColumnKeys::from_json_to_column_keys(&self.column_keys_as_json_hex); let mut fep = FileEncryptionProperties::builder( hex::decode(self.footer_key_as_hex).unwrap(), ) - .with_plaintext_footer(!self.encrypt_footer) - .with_aad_prefix_storage(self.store_aad_prefix); + .with_plaintext_footer(!self.encrypt_footer) + .with_aad_prefix_storage(self.store_aad_prefix); if self.footer_key_metadata_as_hex.len() > 0 { fep = fep.with_footer_key_metadata( @@ -348,18 +316,21 @@ impl Into for ConfigFileEncryptionProperties { .expect("Invalid footer key metadata"), ); } - - for (column_name, encryption_key) in eck.iter() { - match &encryption_key.key_metadata { + + for (column_name, encryption_key) in self.column_keys_as_hex.iter() { + let encryption_key = hex::decode(encryption_key).expect("Invalid column encryption key"); + let key_metadata = self.column_metadata_as_hex.get(column_name).map(|x| + hex::decode(x).expect("Invalid column metadata")); + match key_metadata { Some(key_metadata) => { fep = fep.with_column_key_and_metadata( column_name, - encryption_key.key.clone(), - key_metadata.clone(), + encryption_key, + key_metadata, ); } None => { - fep = fep.with_column_key(column_name, encryption_key.key.clone()); + fep = fep.with_column_key(column_name, encryption_key); } } } @@ -377,11 +348,17 @@ impl Into for ConfigFileEncryptionProperties { impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { fn from(f: &FileEncryptionProperties) -> Self { let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); - let ck = EncryptionColumnKeys::new( - &column_names_vec, - &column_keys_vec, - &column_metas_vec, - ); + + let mut column_keys_as_hex: HashMap = HashMap::new(); + let mut column_metadata_as_hex: HashMap = HashMap::new(); + + for (i, column_name) in column_names_vec.iter().enumerate() { + column_keys_as_hex.insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); + let metadata_as_hex: Option = column_metas_vec.get(i).map(|x| hex::encode(x)); + if let Some(metadata_as_hex) = metadata_as_hex { + column_metadata_as_hex.insert(column_name.clone(), metadata_as_hex); + } + } let mut aad_prefix: Vec = Vec::new(); if let Some(prefix) = f.aad_prefix() { aad_prefix = prefix.clone(); @@ -393,13 +370,151 @@ impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { .footer_key_metadata() .map(|x| hex::encode(x)) .unwrap_or_default(), - column_keys_as_json_hex: ck.to_json(), + column_keys_as_hex, + column_metadata_as_hex, aad_prefix_as_hex: hex::encode(aad_prefix), store_aad_prefix: f.store_aad_prefix(), } } } +#[derive(Clone, Debug, PartialEq)] +pub struct ConfigFileDecryptionProperties { + pub footer_key_as_hex: String, + pub column_keys_as_hex: HashMap, + pub aad_prefix_as_hex: String, + pub footer_signature_verification: bool, // default = true +} + +impl Default for ConfigFileDecryptionProperties { + fn default() -> Self { + ConfigFileDecryptionProperties{ + footer_key_as_hex: String::new(), + column_keys_as_hex: HashMap::new(), + aad_prefix_as_hex: String::new(), + footer_signature_verification: true, + } + } +} +impl ConfigFileDecryptionProperties { + /// Return new default TableParquetOptions + pub fn new() -> Self { + Self::default() + } +} + +impl ConfigField for ConfigFileDecryptionProperties { + fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { + let key = format!("{}.footer_key_as_hex", key_prefix); + let desc = "Key to use for the parquet footer"; + self.footer_key_as_hex.visit(v, key.as_str(), desc); + + let desc = "Per column decryption keys"; + for (col_name, col_val) in self.column_keys_as_hex.iter() { + let key = format!("{}.column_keys_as_hex::{}", key_prefix, col_name); + col_val.visit(v, key.as_str(), desc); + } + + let key = format!("{}.aad_prefix_as_hex", key_prefix); + let desc = "AAD prefix to use"; + self.aad_prefix_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{}.footer_signature_verification", key_prefix); + let desc = "If true, verify the footer signature"; + self.footer_signature_verification.visit(v, key.as_str(), desc); + } + + fn set(&mut self, key: &str, value: &str) -> Result<()> { + // Any hex encoded values must be pre-encoded using + // hex::encode() before calling set. + if key.starts_with("column_keys_as_hex.") { + let k = match key.split(".").collect::>()[..] { + [_meta] | [_meta, ""] => { + return _config_err!( + "Invalid column name provided, missing name in column_keys_as_hex." + ) + } + [_meta, k] => k.into(), + _ => { + return _config_err!( + "Invalid column name provided, found too many '.' in \"{key}\"" + ) + } + }; + self.column_keys_as_hex.insert(k, String::from(value)); + return Ok(()) + }; + + let (key, rem) = key.split_once('.').unwrap_or((key, "")); + match key { + "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), + "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), + "footer_signature_verification" => self.footer_signature_verification.set(rem, value.as_ref()), + _ => _config_err!( + "Config value \"{}\" not found on ConfigFileEncryptionProperties", + key + ), + } + } +} + + +#[cfg(feature = "parquet")] +impl Into for ConfigFileDecryptionProperties { + fn into(self) -> FileDecryptionProperties { + let mut column_names: Vec<&str> = Vec::new(); + let mut column_keys: Vec> = Vec::new(); + + for (col_name, encryption_key) in self.column_keys_as_hex.iter() { + column_names.push(col_name.as_str()); + column_keys.push(hex::decode(encryption_key).expect("Invalid column decryption key")); + } + + let mut fep = FileDecryptionProperties::builder( + hex::decode(self.footer_key_as_hex).expect("Invalid footer key"), + ) + .with_column_keys(column_names, column_keys) + .unwrap(); + + if !self.footer_signature_verification { + fep = fep.disable_footer_signature_verification(); + } + + if self.aad_prefix_as_hex.len() > 0 { + let aad_prefix = + hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + fep = fep.with_aad_prefix(aad_prefix); + } + + fep.build().unwrap() + } +} + +#[cfg(feature = "parquet")] +impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { + fn from(f: &FileDecryptionProperties) -> Self { + let (column_names_vec, column_keys_vec) = f.column_keys(); + let mut column_keys_as_hex: HashMap = HashMap::new(); + for (i, column_name) in column_names_vec.iter().enumerate() { + column_keys_as_hex.insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); + } + + let mut aad_prefix: Vec = Vec::new(); + if let Some(prefix) = f.aad_prefix() { + aad_prefix = prefix.clone(); + } + ConfigFileDecryptionProperties { + footer_key_as_hex: hex::encode( + f.footer_key(None).unwrap_or_default().as_ref(), + ), + column_keys_as_hex, + aad_prefix_as_hex: hex::encode(aad_prefix), + footer_signature_verification: f.check_plaintext_footer_integrity(), + } + } +} + + config_namespace! { /// Options related to catalog and directory scanning /// @@ -804,8 +919,10 @@ config_namespace! { /// data frame. pub maximum_buffered_record_batches_per_stream: usize, default = 2 - // Optional encryption settings + /// Optional file decryption properties pub file_decryption_properties: Option, default = None + + /// Optional file encryption properties pub file_encryption_properties: Option, default = None } } @@ -2504,14 +2621,18 @@ mod tests { ) .unwrap(); - table_config - .parquet - .set( - "file_encryption_properties.column_keys_as_json_hex", - config_encrypt.column_keys_as_json_hex.as_str(), - ) - .unwrap(); - + for (i, col_name) in column_names.iter().enumerate() { + let key = format!("file_encryption_properties.column_keys_as_hex.{}", col_name); + let value = hex::encode(column_keys[i].clone()); + table_config + .parquet + .set( + key.as_str(), + value.as_str(), + ) + .unwrap(); + } + // Print matching final encryption config // println!("{:#?}", table_config.parquet.global.file_encryption_properties); @@ -2536,13 +2657,17 @@ mod tests { ) .unwrap(); - table_config - .parquet - .set( - "file_decryption_properties.column_keys_as_json_hex", - config_decrypt.column_keys_as_json_hex.as_str(), - ) - .unwrap(); + for (i, col_name) in column_names.iter().enumerate() { + let key = format!("file_decryption_properties.column_keys_as_hex.{}", col_name); + let value = hex::encode(column_keys[i].clone()); + table_config + .parquet + .set( + key.as_str(), + value.as_str(), + ) + .unwrap(); + } // Print matching final decryption config // println!("{:#?}", table_config.parquet.global.file_decryption_properties); From a81855fcbf3cfb63512c1ba124e1ebbfd5e6b15c Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 4 Jun 2025 20:24:26 -0700 Subject: [PATCH 16/58] Fix generated field names in visit for encryptor and decryptor to use "." instead of "::" --- datafusion/common/src/config.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index c03b7b898c52f..2355f26c0cbf0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -228,13 +228,13 @@ impl ConfigField for ConfigFileEncryptionProperties { let desc = "Per column encryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{}.column_keys_as_hex::{}", key_prefix, col_name); + let key = format!("{}.column_keys_as_hex.{}", key_prefix, col_name); col_val.visit(v, key.as_str(), desc); } let desc = "Per column metadata"; for (col_name, col_val) in self.column_metadata_as_hex.iter() { - let key = format!("{}.column_metadata_as_hex::{}", key_prefix, col_name); + let key = format!("{}.column_metadata_as_hex.{}", key_prefix, col_name); col_val.visit(v, key.as_str(), desc); } @@ -411,7 +411,7 @@ impl ConfigField for ConfigFileDecryptionProperties { let desc = "Per column decryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{}.column_keys_as_hex::{}", key_prefix, col_name); + let key = format!("{}.column_keys_as_hex.{}", key_prefix, col_name); col_val.visit(v, key.as_str(), desc); } From 4cf12b368f3221629f667bfbe63b17afe3d92292 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 5 Jun 2025 19:05:58 -0700 Subject: [PATCH 17/58] 1. Disable parallel writes with enccryption. 2. Fixed unused header warning in config.rs. 3. Fix test case in encryption.rs to call conversion to ConfigFileDecryption properties correctly. --- datafusion/common/src/config.rs | 1 + datafusion/core/src/dataframe/parquet.rs | 4 +--- datafusion/core/tests/parquet/encryption.rs | 2 +- datafusion/datasource-parquet/src/file_format.rs | 7 ++++++- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 2355f26c0cbf0..fc4c995ba53c9 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -27,6 +27,7 @@ use std::error::Error; use std::fmt::{self, Display}; use std::str::FromStr; +#[cfg(feature = "parquet")] use hex; #[cfg(feature = "parquet")] use parquet::encryption::decrypt::FileDecryptionProperties; diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index ab4a6e12887b9..ff45f4ddac51e 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -279,9 +279,7 @@ mod tests { // Write encrypted parquet using write_parquet let mut options = TableParquetOptions::default(); options.global.file_encryption_properties = Some((&encrypt).into()); - // Parallel writing for encryption is broken right now. - // Rok is working on it. - options.global.allow_single_file_parallelism = false; + df.write_parquet( tempfile_str.as_str(), DataFrameWriteOptions::new().with_single_file_output(true), diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs index 44623c968eeec..9c14f505f8eb4 100644 --- a/datafusion/core/tests/parquet/encryption.rs +++ b/datafusion/core/tests/parquet/encryption.rs @@ -109,7 +109,7 @@ async fn round_trip_encryption() { // Read encrypted parquet let mut sc = SessionConfig::new(); - let fd: ConfigFileDecryptionProperties = decrypt.clone().into(); + let fd: ConfigFileDecryptionProperties = (&decrypt).into(); sc.options_mut() .execution .parquet diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 2aef81f6a0f88..345317e190b98 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1307,8 +1307,13 @@ impl FileSink for ParquetSink { object_store: Arc, ) -> Result { let parquet_opts = &self.parquet_options; - let allow_single_file_parallelism = + let mut allow_single_file_parallelism = parquet_opts.global.allow_single_file_parallelism; + + if parquet_opts.global.file_encryption_properties.is_some() { + // For now, arrow-rs does not support parallel writes with encryption + allow_single_file_parallelism = false; + } let mut file_write_tasks: JoinSet< std::result::Result<(Path, FileMetaData), DataFusionError>, From f29bec371801425d3103d1a95dbcac334999578a Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 6 Jun 2025 13:55:36 -0700 Subject: [PATCH 18/58] cargo fmt --- datafusion/common/src/config.rs | 122 ++++++++++-------- datafusion/core/src/dataframe/parquet.rs | 2 +- .../datasource-parquet/src/file_format.rs | 2 +- 3 files changed, 69 insertions(+), 57 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index fc4c995ba53c9..630731211d839 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -199,7 +199,7 @@ macro_rules! config_namespace { pub struct ConfigFileEncryptionProperties { pub encrypt_footer: bool, // default = false pub footer_key_as_hex: String, - pub footer_key_metadata_as_hex: String, + pub footer_key_metadata_as_hex: String, pub column_keys_as_hex: HashMap, pub column_metadata_as_hex: HashMap, pub aad_prefix_as_hex: String, @@ -218,7 +218,7 @@ impl ConfigField for ConfigFileEncryptionProperties { let key = format!("{}.encrypt_footer", key_prefix); let desc = "Encrypt the footer"; self.encrypt_footer.visit(v, key.as_str(), desc); - + let key = format!("{}.footer_key_as_hex", key_prefix); let desc = "Key to use for the parquet footer"; self.footer_key_as_hex.visit(v, key.as_str(), desc); @@ -232,7 +232,7 @@ impl ConfigField for ConfigFileEncryptionProperties { let key = format!("{}.column_keys_as_hex.{}", key_prefix, col_name); col_val.visit(v, key.as_str(), desc); } - + let desc = "Per column metadata"; for (col_name, col_val) in self.column_metadata_as_hex.iter() { let key = format!("{}.column_metadata_as_hex.{}", key_prefix, col_name); @@ -247,7 +247,7 @@ impl ConfigField for ConfigFileEncryptionProperties { let desc = "If true, store the AAD prefix"; self.store_aad_prefix.visit(v, key.as_str(), desc); } - + fn set(&mut self, key: &str, value: &str) -> Result<()> { // Any hex encoded values must be pre-encoded using // hex::encode() before calling set. @@ -266,7 +266,7 @@ impl ConfigField for ConfigFileEncryptionProperties { } }; self.column_keys_as_hex.insert(k, String::from(value)); - return Ok(()) + return Ok(()); }; if key.starts_with("column_metadata_as_hex.") { @@ -284,20 +284,22 @@ impl ConfigField for ConfigFileEncryptionProperties { } }; self.column_metadata_as_hex.insert(k, String::from(value)); - return Ok(()) + return Ok(()); }; - + let (key, rem) = key.split_once('.').unwrap_or((key, "")); match key { - "encrypt_footer" => self.encrypt_footer.set(rem, value.as_ref()), - "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), - "footer_key_metadata_as_hex" => self.footer_key_metadata_as_hex.set(rem, value.as_ref()), - "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), - "store_aad_prefix" => self.store_aad_prefix.set(rem, value.as_ref()), - _ => _config_err!( - "Config value \"{}\" not found on ConfigFileEncryptionProperties", - key - ), + "encrypt_footer" => self.encrypt_footer.set(rem, value.as_ref()), + "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), + "footer_key_metadata_as_hex" => { + self.footer_key_metadata_as_hex.set(rem, value.as_ref()) + } + "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), + "store_aad_prefix" => self.store_aad_prefix.set(rem, value.as_ref()), + _ => _config_err!( + "Config value \"{}\" not found on ConfigFileEncryptionProperties", + key + ), } } } @@ -308,8 +310,8 @@ impl Into for ConfigFileEncryptionProperties { let mut fep = FileEncryptionProperties::builder( hex::decode(self.footer_key_as_hex).unwrap(), ) - .with_plaintext_footer(!self.encrypt_footer) - .with_aad_prefix_storage(self.store_aad_prefix); + .with_plaintext_footer(!self.encrypt_footer) + .with_aad_prefix_storage(self.store_aad_prefix); if self.footer_key_metadata_as_hex.len() > 0 { fep = fep.with_footer_key_metadata( @@ -317,11 +319,14 @@ impl Into for ConfigFileEncryptionProperties { .expect("Invalid footer key metadata"), ); } - + for (column_name, encryption_key) in self.column_keys_as_hex.iter() { - let encryption_key = hex::decode(encryption_key).expect("Invalid column encryption key"); - let key_metadata = self.column_metadata_as_hex.get(column_name).map(|x| - hex::decode(x).expect("Invalid column metadata")); + let encryption_key = + hex::decode(encryption_key).expect("Invalid column encryption key"); + let key_metadata = self + .column_metadata_as_hex + .get(column_name) + .map(|x| hex::decode(x).expect("Invalid column metadata")); match key_metadata { Some(key_metadata) => { fep = fep.with_column_key_and_metadata( @@ -354,8 +359,10 @@ impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { let mut column_metadata_as_hex: HashMap = HashMap::new(); for (i, column_name) in column_names_vec.iter().enumerate() { - column_keys_as_hex.insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); - let metadata_as_hex: Option = column_metas_vec.get(i).map(|x| hex::encode(x)); + column_keys_as_hex + .insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); + let metadata_as_hex: Option = + column_metas_vec.get(i).map(|x| hex::encode(x)); if let Some(metadata_as_hex) = metadata_as_hex { column_metadata_as_hex.insert(column_name.clone(), metadata_as_hex); } @@ -383,13 +390,13 @@ impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { pub struct ConfigFileDecryptionProperties { pub footer_key_as_hex: String, pub column_keys_as_hex: HashMap, - pub aad_prefix_as_hex: String, + pub aad_prefix_as_hex: String, pub footer_signature_verification: bool, // default = true } impl Default for ConfigFileDecryptionProperties { fn default() -> Self { - ConfigFileDecryptionProperties{ + ConfigFileDecryptionProperties { footer_key_as_hex: String::new(), column_keys_as_hex: HashMap::new(), aad_prefix_as_hex: String::new(), @@ -409,7 +416,7 @@ impl ConfigField for ConfigFileDecryptionProperties { let key = format!("{}.footer_key_as_hex", key_prefix); let desc = "Key to use for the parquet footer"; self.footer_key_as_hex.visit(v, key.as_str(), desc); - + let desc = "Per column decryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { let key = format!("{}.column_keys_as_hex.{}", key_prefix, col_name); @@ -422,9 +429,10 @@ impl ConfigField for ConfigFileDecryptionProperties { let key = format!("{}.footer_signature_verification", key_prefix); let desc = "If true, verify the footer signature"; - self.footer_signature_verification.visit(v, key.as_str(), desc); + self.footer_signature_verification + .visit(v, key.as_str(), desc); } - + fn set(&mut self, key: &str, value: &str) -> Result<()> { // Any hex encoded values must be pre-encoded using // hex::encode() before calling set. @@ -443,23 +451,24 @@ impl ConfigField for ConfigFileDecryptionProperties { } }; self.column_keys_as_hex.insert(k, String::from(value)); - return Ok(()) + return Ok(()); }; - + let (key, rem) = key.split_once('.').unwrap_or((key, "")); match key { "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), - "footer_signature_verification" => self.footer_signature_verification.set(rem, value.as_ref()), + "footer_signature_verification" => { + self.footer_signature_verification.set(rem, value.as_ref()) + } _ => _config_err!( - "Config value \"{}\" not found on ConfigFileEncryptionProperties", - key - ), + "Config value \"{}\" not found on ConfigFileEncryptionProperties", + key + ), } } } - #[cfg(feature = "parquet")] impl Into for ConfigFileDecryptionProperties { fn into(self) -> FileDecryptionProperties { @@ -468,9 +477,11 @@ impl Into for ConfigFileDecryptionProperties { for (col_name, encryption_key) in self.column_keys_as_hex.iter() { column_names.push(col_name.as_str()); - column_keys.push(hex::decode(encryption_key).expect("Invalid column decryption key")); + column_keys.push( + hex::decode(encryption_key).expect("Invalid column decryption key"), + ); } - + let mut fep = FileDecryptionProperties::builder( hex::decode(self.footer_key_as_hex).expect("Invalid footer key"), ) @@ -497,9 +508,10 @@ impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { let (column_names_vec, column_keys_vec) = f.column_keys(); let mut column_keys_as_hex: HashMap = HashMap::new(); for (i, column_name) in column_names_vec.iter().enumerate() { - column_keys_as_hex.insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); + column_keys_as_hex + .insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); } - + let mut aad_prefix: Vec = Vec::new(); if let Some(prefix) = f.aad_prefix() { aad_prefix = prefix.clone(); @@ -515,7 +527,6 @@ impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { } } - config_namespace! { /// Options related to catalog and directory scanning /// @@ -922,7 +933,7 @@ config_namespace! { /// Optional file decryption properties pub file_decryption_properties: Option, default = None - + /// Optional file encryption properties pub file_encryption_properties: Option, default = None } @@ -2430,7 +2441,10 @@ impl Display for OutputFormat { #[cfg(test)] mod tests { - use crate::config::{ConfigEntry, ConfigExtension, ConfigField, ConfigFileType, ExtensionOptions, Extensions, TableOptions}; + use crate::config::{ + ConfigEntry, ConfigExtension, ConfigField, ConfigFileType, ExtensionOptions, + Extensions, TableOptions, + }; use std::any::Any; use std::collections::HashMap; @@ -2564,9 +2578,11 @@ mod tests { #[cfg(feature = "parquet")] #[test] fn parquet_table_encryption() { + use crate::config::{ + ConfigFileDecryptionProperties, ConfigFileEncryptionProperties, + }; use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; - use crate::config::{ConfigFileDecryptionProperties, ConfigFileEncryptionProperties}; let footer_key = b"0123456789012345".to_vec(); // 128bit/16 let column_names = vec!["double_field", "float_field"]; @@ -2623,17 +2639,15 @@ mod tests { .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = format!("file_encryption_properties.column_keys_as_hex.{}", col_name); + let key = + format!("file_encryption_properties.column_keys_as_hex.{}", col_name); let value = hex::encode(column_keys[i].clone()); table_config .parquet - .set( - key.as_str(), - value.as_str(), - ) + .set(key.as_str(), value.as_str()) .unwrap(); } - + // Print matching final encryption config // println!("{:#?}", table_config.parquet.global.file_encryption_properties); @@ -2659,14 +2673,12 @@ mod tests { .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = format!("file_decryption_properties.column_keys_as_hex.{}", col_name); + let key = + format!("file_decryption_properties.column_keys_as_hex.{}", col_name); let value = hex::encode(column_keys[i].clone()); table_config .parquet - .set( - key.as_str(), - value.as_str(), - ) + .set(key.as_str(), value.as_str()) .unwrap(); } diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index ff45f4ddac51e..2feb67b0923a5 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -279,7 +279,7 @@ mod tests { // Write encrypted parquet using write_parquet let mut options = TableParquetOptions::default(); options.global.file_encryption_properties = Some((&encrypt).into()); - + df.write_parquet( tempfile_str.as_str(), DataFrameWriteOptions::new().with_single_file_output(true), diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 345317e190b98..1ddb5c3cf6934 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1309,7 +1309,7 @@ impl FileSink for ParquetSink { let parquet_opts = &self.parquet_options; let mut allow_single_file_parallelism = parquet_opts.global.allow_single_file_parallelism; - + if parquet_opts.global.file_encryption_properties.is_some() { // For now, arrow-rs does not support parallel writes with encryption allow_single_file_parallelism = false; From 86fe04b7ee9e7040f6b5044c8d0ba8dc48eebce5 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 6 Jun 2025 14:08:06 -0700 Subject: [PATCH 19/58] Update datafusion/common/src/file_options/parquet_writer.rs Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- datafusion/common/src/file_options/parquet_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 69576a5726ea5..9fb3e748b0ee6 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -227,7 +227,7 @@ impl ParquetOptions { bloom_filter_on_write, bloom_filter_fpp, bloom_filter_ndv, - file_decryption_properties: _, + file_decryption_properties: None, file_encryption_properties, // not in WriterProperties From d4ea63ff2bd0b9617a5ea769be43f5fe16d4b3b5 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 6 Jun 2025 14:33:01 -0700 Subject: [PATCH 20/58] fix variables shown in information schema test. --- datafusion/sqllogictest/test_files/information_schema.slt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 841b289e754a7..35961f782e6a7 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -241,6 +241,8 @@ datafusion.execution.parquet.dictionary_enabled true datafusion.execution.parquet.dictionary_page_size_limit 1048576 datafusion.execution.parquet.enable_page_index true datafusion.execution.parquet.encoding NULL +datafusion.execution.parquet.file_decryption_properties NULL +datafusion.execution.parquet.file_encryption_properties NULL datafusion.execution.parquet.max_row_group_size 1048576 datafusion.execution.parquet.max_statistics_size 4096 datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 @@ -351,6 +353,8 @@ datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionar datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting +datafusion.execution.parquet.file_decryption_properties NULL Optional file decryption properties +datafusion.execution.parquet.file_encryption_properties NULL Optional file encryption properties datafusion.execution.parquet.max_row_group_size 1048576 (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. datafusion.execution.parquet.max_statistics_size 4096 (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. From 86db3a5a22087c7036d90dc9561a45bb9261d01e Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 6 Jun 2025 14:37:57 -0700 Subject: [PATCH 21/58] Backout bad suggestion from copilot --- datafusion/common/src/file_options/parquet_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 9fb3e748b0ee6..69576a5726ea5 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -227,7 +227,7 @@ impl ParquetOptions { bloom_filter_on_write, bloom_filter_fpp, bloom_filter_ndv, - file_decryption_properties: None, + file_decryption_properties: _, file_encryption_properties, // not in WriterProperties From b34441adfbd59608904252b64656fbcf2998cc32 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 6 Jun 2025 15:58:57 -0700 Subject: [PATCH 22/58] Remove unused serde reference Add an example to read and write encrypted parquet files. --- Cargo.lock | 2 - datafusion-examples/README.md | 1 + .../examples/parquet_encrypted.rs | 110 ++++++++++++++++++ datafusion/common/Cargo.toml | 2 - 4 files changed, 111 insertions(+), 4 deletions(-) create mode 100644 datafusion-examples/examples/parquet_encrypted.rs diff --git a/Cargo.lock b/Cargo.lock index d9e86f3980a1a..485737437ad2a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1997,8 +1997,6 @@ dependencies = [ "pyo3", "rand 0.9.1", "recursive", - "serde", - "serde_json", "sqlparser", "tokio", "web-time", diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 3ba4c77cd84c3..d4ec507eb0e32 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -65,6 +65,7 @@ cargo run --example dataframe - [`flight_sql_server.rs`](examples/flight/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients - [`function_factory.rs`](examples/function_factory.rs): Register `CREATE FUNCTION` handler to implement SQL macros - [`optimizer_rule.rs`](examples/optimizer_rule.rs): Use a custom OptimizerRule to replace certain predicates +- [`parquet_encrypted.rs`](examples/parquet_index.rs): Read and write encrypted Parquet files using DataFusion - [`parquet_index.rs`](examples/parquet_index.rs): Create an secondary index over several parquet files and use it to speed up queries - [`parquet_exec_visitor.rs`](examples/parquet_exec_visitor.rs): Extract statistics by visiting an ExecutionPlan after execution - [`parse_sql_expr.rs`](examples/parse_sql_expr.rs): Parse SQL text into DataFusion `Expr`. diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs new file mode 100644 index 0000000000000..50dca326cd81f --- /dev/null +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -0,0 +1,110 @@ +use tempfile::TempDir; +use datafusion::common::DataFusionError; +use datafusion::config::{ConfigFileDecryptionProperties, TableParquetOptions}; +use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; +use datafusion::execution::SessionStateBuilder; +use datafusion::logical_expr::{col, lit}; +use datafusion::parquet::encryption::decrypt::FileDecryptionProperties; +use datafusion::parquet::encryption::encrypt::FileEncryptionProperties; +use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; + + +#[tokio::main] +async fn main() -> datafusion::common::Result<()> { + // The SessionContext is the main high level API for interacting with DataFusion + let ctx = SessionContext::new(); + + // Find the local path of "alltypes_plain.parquet" + let testdata = datafusion::test_util::parquet_test_data(); + let filename = &format!("{testdata}/alltypes_plain.parquet"); + + // Read the sample parquet file + let parquet_df = ctx + .read_parquet(filename, ParquetReadOptions::default()) + .await?; + + // Show information from the dataframe + println!("Original Parquet DataFrame:"); + query_dataframe(&parquet_df, false).await?; + + // Setup encryption and decryption properties + let (encrypt, decrypt) = setup_encryption(&parquet_df)?; + + // Create a temporary file location for the encrypted parquet file + let tmp_dir = TempDir::new()?; + let tempfile = tmp_dir.path().join("alltypes_plain-encrypted.parquet"); + let tempfile_str = tempfile.into_os_string().into_string().unwrap(); + + // Write encrypted parquet + let mut options = TableParquetOptions::default(); + options.global.file_encryption_properties = Some((&encrypt).into()); + parquet_df.write_parquet( + tempfile_str.as_str(), + DataFrameWriteOptions::new().with_single_file_output(true), + Some(options), + ) + .await?; + + + // Read encrypted parquet + let mut sc = SessionConfig::new(); + let fd: ConfigFileDecryptionProperties = (&decrypt).into(); + sc.options_mut() + .execution + .parquet + .file_decryption_properties = Some(fd); + + let state = SessionStateBuilder::new().with_config(sc).build(); + let ctx: SessionContext = SessionContext::new_with_state(state); + + let encrypted_parquet_df = ctx + .read_parquet(tempfile_str, ParquetReadOptions::default()) + .await?; + + // Show information from the dataframe + println!("\n\nEncrypted Parquet DataFrame:"); + query_dataframe(&encrypted_parquet_df, true).await?; + + + Ok(()) +} + +// Show information from the dataframe +async fn query_dataframe(df: &DataFrame, filter: bool) -> Result<(), DataFusionError> { + // show its schema using 'describe' + df.clone().describe().await?.show().await?; + + if filter { + // Select three columns and filter the results + // so that only rows where id > 1 are returned + df + .clone() + .select_columns(&["id", "bool_col", "timestamp_col"])? + .filter(col("id").gt(lit(5)))? + .show() + .await?; + } + + + Ok(()) +} + +// Setup encryption and decryption properties +fn setup_encryption(parquet_df: &DataFrame) -> Result<(FileEncryptionProperties, FileDecryptionProperties), DataFusionError> { + + let schema = parquet_df.schema(); + let footer_key = b"0123456789012345".to_vec(); // 128bit/16 + let column_key = b"1234567890123450".to_vec(); // 128bit/16 + + let mut encrypt = FileEncryptionProperties::builder(footer_key.clone()); + let mut decrypt = FileDecryptionProperties::builder(footer_key.clone()); + + for field in schema.fields().iter() { + encrypt = encrypt.with_column_key(field.name().as_str(), column_key.clone()); + decrypt = decrypt.with_column_key(field.name().as_str(), column_key.clone()); + } + + let encrypt = encrypt.build()?; + let decrypt = decrypt.build()?; + Ok((encrypt, decrypt)) +} \ No newline at end of file diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 1a26d8db7ff1e..ea6ac0b323f81 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -67,8 +67,6 @@ paste = "1.0.15" pyo3 = { version = "0.24.2", optional = true } recursive = { workspace = true, optional = true } sqlparser = { workspace = true } -serde = { version = "1.0.217", features = ["derive"] } -serde_json = {workspace = true} tokio = { workspace = true } From 668d728f2942b623df62b7216202222520e35e8d Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Fri, 6 Jun 2025 16:05:57 -0700 Subject: [PATCH 23/58] cargo fmt --- .../examples/parquet_encrypted.rs | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index 50dca326cd81f..535e917aeab4c 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -1,4 +1,3 @@ -use tempfile::TempDir; use datafusion::common::DataFusionError; use datafusion::config::{ConfigFileDecryptionProperties, TableParquetOptions}; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; @@ -7,7 +6,7 @@ use datafusion::logical_expr::{col, lit}; use datafusion::parquet::encryption::decrypt::FileDecryptionProperties; use datafusion::parquet::encryption::encrypt::FileEncryptionProperties; use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; - +use tempfile::TempDir; #[tokio::main] async fn main() -> datafusion::common::Result<()> { @@ -38,14 +37,14 @@ async fn main() -> datafusion::common::Result<()> { // Write encrypted parquet let mut options = TableParquetOptions::default(); options.global.file_encryption_properties = Some((&encrypt).into()); - parquet_df.write_parquet( - tempfile_str.as_str(), - DataFrameWriteOptions::new().with_single_file_output(true), - Some(options), - ) + parquet_df + .write_parquet( + tempfile_str.as_str(), + DataFrameWriteOptions::new().with_single_file_output(true), + Some(options), + ) .await?; - // Read encrypted parquet let mut sc = SessionConfig::new(); let fd: ConfigFileDecryptionProperties = (&decrypt).into(); @@ -65,7 +64,6 @@ async fn main() -> datafusion::common::Result<()> { println!("\n\nEncrypted Parquet DataFrame:"); query_dataframe(&encrypted_parquet_df, true).await?; - Ok(()) } @@ -77,21 +75,20 @@ async fn query_dataframe(df: &DataFrame, filter: bool) -> Result<(), DataFusionE if filter { // Select three columns and filter the results // so that only rows where id > 1 are returned - df - .clone() + df.clone() .select_columns(&["id", "bool_col", "timestamp_col"])? .filter(col("id").gt(lit(5)))? .show() .await?; } - - + Ok(()) } // Setup encryption and decryption properties -fn setup_encryption(parquet_df: &DataFrame) -> Result<(FileEncryptionProperties, FileDecryptionProperties), DataFusionError> { - +fn setup_encryption( + parquet_df: &DataFrame, +) -> Result<(FileEncryptionProperties, FileDecryptionProperties), DataFusionError> { let schema = parquet_df.schema(); let footer_key = b"0123456789012345".to_vec(); // 128bit/16 let column_key = b"1234567890123450".to_vec(); // 128bit/16 @@ -107,4 +104,4 @@ fn setup_encryption(parquet_df: &DataFrame) -> Result<(FileEncryptionProperties, let encrypt = encrypt.build()?; let decrypt = decrypt.build()?; Ok((encrypt, decrypt)) -} \ No newline at end of file +} From ec1e8da542f1fd21a77bc7fcf758117c9f81cad3 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 14:54:02 -0700 Subject: [PATCH 24/58] change file_format.rs to use global encryption options in struct. --- datafusion/datasource-parquet/src/file_format.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 1ddb5c3cf6934..16b78d3358504 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -350,13 +350,8 @@ impl FileFormat for ParquetFormat { Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?), None => None, }; - let config_file_decryption_properties = &state - .config() - .options() - .execution - .parquet - .file_decryption_properties - .clone(); + let config_file_decryption_properties = + &self.options.global.file_decryption_properties; let file_decryption_properties: Option = match config_file_decryption_properties { Some(cfd) => { From e23340877e8ff15a9cad134eb00f58865fa4ba61 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 15:10:44 -0700 Subject: [PATCH 25/58] Turn off page_index for encrypted example. Get encrypted example working with filter. --- .../examples/parquet_encrypted.rs | 26 +++++++++---------- datafusion/datasource-parquet/src/opener.rs | 8 +++++- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index 535e917aeab4c..6c7deb9821dc4 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -24,7 +24,7 @@ async fn main() -> datafusion::common::Result<()> { // Show information from the dataframe println!("Original Parquet DataFrame:"); - query_dataframe(&parquet_df, false).await?; + query_dataframe(&parquet_df).await?; // Setup encryption and decryption properties let (encrypt, decrypt) = setup_encryption(&parquet_df)?; @@ -52,7 +52,8 @@ async fn main() -> datafusion::common::Result<()> { .execution .parquet .file_decryption_properties = Some(fd); - + + let state = SessionStateBuilder::new().with_config(sc).build(); let ctx: SessionContext = SessionContext::new_with_state(state); @@ -62,25 +63,24 @@ async fn main() -> datafusion::common::Result<()> { // Show information from the dataframe println!("\n\nEncrypted Parquet DataFrame:"); - query_dataframe(&encrypted_parquet_df, true).await?; + query_dataframe(&encrypted_parquet_df).await?; Ok(()) } // Show information from the dataframe -async fn query_dataframe(df: &DataFrame, filter: bool) -> Result<(), DataFusionError> { +async fn query_dataframe(df: &DataFrame) -> Result<(), DataFusionError> { // show its schema using 'describe' df.clone().describe().await?.show().await?; + + // Select three columns and filter the results + // so that only rows where id > 1 are returned + df.clone() + .select_columns(&["id", "bool_col", "timestamp_col"])? + .filter(col("id").gt(lit(5)))? + .show() + .await?; - if filter { - // Select three columns and filter the results - // so that only rows where id > 1 are returned - df.clone() - .select_columns(&["id", "bool_col", "timestamp_col"])? - .filter(col("id").gt(lit(5)))? - .show() - .await?; - } Ok(()) } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5659bec0dcf54..f8a3889f558ce 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -125,8 +125,14 @@ impl FileOpener for ParquetOpener { let predicate_creation_errors = MetricBuilder::new(&self.metrics) .global_counter("num_predicate_creation_errors"); - let enable_page_index = self.enable_page_index; + let mut enable_page_index = self.enable_page_index; let file_decryption_properties = self.file_decryption_properties.clone(); + + // For now, page index does not work with encrypted files. See: + // https://github.com/apache/arrow-rs/issues/7629 + if file_decryption_properties.is_some() { + enable_page_index = false; + } Ok(Box::pin(async move { // Don't load the page index yet. Since it is not stored inline in From 9ffaae403c6933b2b891b0bb4cc2b0e02f992272 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 15:14:30 -0700 Subject: [PATCH 26/58] Tidy up example output. --- datafusion-examples/examples/parquet_encrypted.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index 6c7deb9821dc4..7ef31d19202cc 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -23,6 +23,7 @@ async fn main() -> datafusion::common::Result<()> { .await?; // Show information from the dataframe + println!("==============================================================================="); println!("Original Parquet DataFrame:"); query_dataframe(&parquet_df).await?; @@ -62,7 +63,8 @@ async fn main() -> datafusion::common::Result<()> { .await?; // Show information from the dataframe - println!("\n\nEncrypted Parquet DataFrame:"); + println!("\n\n==============================================================================="); + println!("Encrypted Parquet DataFrame:"); query_dataframe(&encrypted_parquet_df).await?; Ok(()) @@ -71,10 +73,12 @@ async fn main() -> datafusion::common::Result<()> { // Show information from the dataframe async fn query_dataframe(df: &DataFrame) -> Result<(), DataFusionError> { // show its schema using 'describe' + println!("Schema:"); df.clone().describe().await?.show().await?; // Select three columns and filter the results // so that only rows where id > 1 are returned + println!("\nSelected rows and columns:"); df.clone() .select_columns(&["id", "bool_col", "timestamp_col"])? .filter(col("id").gt(lit(5)))? From 8e244e96940bbb365d19f02dc0e614909f55a26d Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 15:36:04 -0700 Subject: [PATCH 27/58] Add missing license. Run taplo format --- Cargo.toml | 2 +- .../examples/parquet_encrypted.rs | 23 ++++++++++++++++--- datafusion/common/Cargo.toml | 3 +-- 3 files changed, 22 insertions(+), 6 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 9e4ac7651b999..03857ecb89e1d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -155,7 +155,7 @@ parquet = { version = "55.1.0", default-features = false, features = [ "arrow", "async", "object_store", - "encryption" + "encryption", ] } pbjson = { version = "0.7.0" } pbjson-types = "0.7" diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index 7ef31d19202cc..41be43a31986f 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -1,3 +1,20 @@ +// 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. + use datafusion::common::DataFusionError; use datafusion::config::{ConfigFileDecryptionProperties, TableParquetOptions}; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; @@ -53,8 +70,8 @@ async fn main() -> datafusion::common::Result<()> { .execution .parquet .file_decryption_properties = Some(fd); - - + + let state = SessionStateBuilder::new().with_config(sc).build(); let ctx: SessionContext = SessionContext::new_with_state(state); @@ -75,7 +92,7 @@ async fn query_dataframe(df: &DataFrame) -> Result<(), DataFusionError> { // show its schema using 'describe' println!("Schema:"); df.clone().describe().await?.show().await?; - + // Select three columns and filter the results // so that only rows where id > 1 are returned println!("\nSelected rows and columns:"); diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index ea6ac0b323f81..da1ebcc5fa5ac 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -56,8 +56,8 @@ arrow = { workspace = true } arrow-ipc = { workspace = true } base64 = "0.22.1" half = { workspace = true } -hex = "0.4.3" hashbrown = { workspace = true } +hex = "0.4.3" indexmap = { workspace = true } libc = "0.2.172" log = { workspace = true } @@ -69,7 +69,6 @@ recursive = { workspace = true, optional = true } sqlparser = { workspace = true } tokio = { workspace = true } - [target.'cfg(target_family = "wasm")'.dependencies] web-time = "1.1.0" From 2871d515ea0e3a1e0cc69f9afd0ebd5bc24e4f8f Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 15:42:21 -0700 Subject: [PATCH 28/58] Update configs.md by running dev/update_config_docs.sh --- docs/source/user-guide/configs.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4129ddc392ab6..b8487094fc781 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -81,6 +81,8 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | | datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.file_decryption_properties | NULL | Optional file decryption properties | +| datafusion.execution.parquet.file_encryption_properties | NULL | Optional file encryption properties | | datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | | datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | From c405167fce90c85355e70fa145cf3b5e09732c52 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 16:31:27 -0700 Subject: [PATCH 29/58] Cargo fmt + clippy changes. --- .../examples/parquet_encrypted.rs | 6 +- datafusion/common/src/config.rs | 68 +++++++++---------- .../common/src/file_options/parquet_writer.rs | 12 ++-- datafusion/datasource-parquet/src/opener.rs | 4 +- 4 files changed, 42 insertions(+), 48 deletions(-) diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index 41be43a31986f..f57330311e1f1 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -40,7 +40,9 @@ async fn main() -> datafusion::common::Result<()> { .await?; // Show information from the dataframe - println!("==============================================================================="); + println!( + "===============================================================================" + ); println!("Original Parquet DataFrame:"); query_dataframe(&parquet_df).await?; @@ -71,7 +73,6 @@ async fn main() -> datafusion::common::Result<()> { .parquet .file_decryption_properties = Some(fd); - let state = SessionStateBuilder::new().with_config(sc).build(); let ctx: SessionContext = SessionContext::new_with_state(state); @@ -102,7 +103,6 @@ async fn query_dataframe(df: &DataFrame) -> Result<(), DataFusionError> { .show() .await?; - Ok(()) } diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 630731211d839..a6b5fffd50dcd 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -215,35 +215,35 @@ impl ConfigFileEncryptionProperties { impl ConfigField for ConfigFileEncryptionProperties { fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { - let key = format!("{}.encrypt_footer", key_prefix); + let key = format!("{key_prefix}.encrypt_footer"); let desc = "Encrypt the footer"; self.encrypt_footer.visit(v, key.as_str(), desc); - let key = format!("{}.footer_key_as_hex", key_prefix); + let key = format!("{key_prefix}.footer_key_as_hex"); let desc = "Key to use for the parquet footer"; self.footer_key_as_hex.visit(v, key.as_str(), desc); - let key = format!("{}.footer_key_metadata_as_hex", key_prefix); + let key = format!("{key_prefix}.footer_key_metadata_as_hex"); let desc = "Metadata to use for the parquet footer"; self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc); let desc = "Per column encryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{}.column_keys_as_hex.{}", key_prefix, col_name); + let key = format!("{key_prefix}.column_keys_as_hex.{col_name}"); col_val.visit(v, key.as_str(), desc); } let desc = "Per column metadata"; for (col_name, col_val) in self.column_metadata_as_hex.iter() { - let key = format!("{}.column_metadata_as_hex.{}", key_prefix, col_name); + let key = format!("{key_prefix}.column_metadata_as_hex.{col_name}"); col_val.visit(v, key.as_str(), desc); } - let key = format!("{}.aad_prefix_as_hex", key_prefix); + let key = format!("{key_prefix}.aad_prefix_as_hex"); let desc = "AAD prefix to use"; self.aad_prefix_as_hex.visit(v, key.as_str(), desc); - let key = format!("{}.store_aad_prefix", key_prefix); + let key = format!("{key_prefix}.store_aad_prefix"); let desc = "If true, store the AAD prefix"; self.store_aad_prefix.visit(v, key.as_str(), desc); } @@ -305,25 +305,25 @@ impl ConfigField for ConfigFileEncryptionProperties { } #[cfg(feature = "parquet")] -impl Into for ConfigFileEncryptionProperties { - fn into(self) -> FileEncryptionProperties { +impl From for FileEncryptionProperties { + fn from(val: ConfigFileEncryptionProperties) -> Self { let mut fep = FileEncryptionProperties::builder( - hex::decode(self.footer_key_as_hex).unwrap(), + hex::decode(val.footer_key_as_hex).unwrap(), ) - .with_plaintext_footer(!self.encrypt_footer) - .with_aad_prefix_storage(self.store_aad_prefix); + .with_plaintext_footer(!val.encrypt_footer) + .with_aad_prefix_storage(val.store_aad_prefix); - if self.footer_key_metadata_as_hex.len() > 0 { + if !val.footer_key_metadata_as_hex.is_empty() { fep = fep.with_footer_key_metadata( - hex::decode(&self.footer_key_metadata_as_hex) + hex::decode(&val.footer_key_metadata_as_hex) .expect("Invalid footer key metadata"), ); } - for (column_name, encryption_key) in self.column_keys_as_hex.iter() { + for (column_name, encryption_key) in val.column_keys_as_hex.iter() { let encryption_key = hex::decode(encryption_key).expect("Invalid column encryption key"); - let key_metadata = self + let key_metadata = val .column_metadata_as_hex .get(column_name) .map(|x| hex::decode(x).expect("Invalid column metadata")); @@ -341,9 +341,9 @@ impl Into for ConfigFileEncryptionProperties { } } - if self.aad_prefix_as_hex.len() > 0 { + if !val.aad_prefix_as_hex.is_empty() { let aad_prefix: Vec = - hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + hex::decode(&val.aad_prefix_as_hex).expect("Invalid AAD prefix"); fep = fep.with_aad_prefix(aad_prefix); } fep.build().unwrap() @@ -362,7 +362,7 @@ impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { column_keys_as_hex .insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); let metadata_as_hex: Option = - column_metas_vec.get(i).map(|x| hex::encode(x)); + column_metas_vec.get(i).map(hex::encode); if let Some(metadata_as_hex) = metadata_as_hex { column_metadata_as_hex.insert(column_name.clone(), metadata_as_hex); } @@ -376,7 +376,7 @@ impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { footer_key_as_hex: hex::encode(f.footer_key()), footer_key_metadata_as_hex: f .footer_key_metadata() - .map(|x| hex::encode(x)) + .map(hex::encode) .unwrap_or_default(), column_keys_as_hex, column_metadata_as_hex, @@ -413,21 +413,21 @@ impl ConfigFileDecryptionProperties { impl ConfigField for ConfigFileDecryptionProperties { fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { - let key = format!("{}.footer_key_as_hex", key_prefix); + let key = format!("{key_prefix}.footer_key_as_hex"); let desc = "Key to use for the parquet footer"; self.footer_key_as_hex.visit(v, key.as_str(), desc); let desc = "Per column decryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{}.column_keys_as_hex.{}", key_prefix, col_name); + let key = format!("{key_prefix}.column_keys_as_hex.{col_name}"); col_val.visit(v, key.as_str(), desc); } - let key = format!("{}.aad_prefix_as_hex", key_prefix); + let key = format!("{key_prefix}.aad_prefix_as_hex"); let desc = "AAD prefix to use"; self.aad_prefix_as_hex.visit(v, key.as_str(), desc); - let key = format!("{}.footer_signature_verification", key_prefix); + let key = format!("{key_prefix}.footer_signature_verification"); let desc = "If true, verify the footer signature"; self.footer_signature_verification .visit(v, key.as_str(), desc); @@ -470,12 +470,12 @@ impl ConfigField for ConfigFileDecryptionProperties { } #[cfg(feature = "parquet")] -impl Into for ConfigFileDecryptionProperties { - fn into(self) -> FileDecryptionProperties { +impl From for FileDecryptionProperties { + fn from(val: ConfigFileDecryptionProperties) -> Self { let mut column_names: Vec<&str> = Vec::new(); let mut column_keys: Vec> = Vec::new(); - for (col_name, encryption_key) in self.column_keys_as_hex.iter() { + for (col_name, encryption_key) in val.column_keys_as_hex.iter() { column_names.push(col_name.as_str()); column_keys.push( hex::decode(encryption_key).expect("Invalid column decryption key"), @@ -483,18 +483,18 @@ impl Into for ConfigFileDecryptionProperties { } let mut fep = FileDecryptionProperties::builder( - hex::decode(self.footer_key_as_hex).expect("Invalid footer key"), + hex::decode(val.footer_key_as_hex).expect("Invalid footer key"), ) .with_column_keys(column_names, column_keys) .unwrap(); - if !self.footer_signature_verification { + if !val.footer_signature_verification { fep = fep.disable_footer_signature_verification(); } - if self.aad_prefix_as_hex.len() > 0 { + if !val.aad_prefix_as_hex.is_empty() { let aad_prefix = - hex::decode(&self.aad_prefix_as_hex).expect("Invalid AAD prefix"); + hex::decode(&val.aad_prefix_as_hex).expect("Invalid AAD prefix"); fep = fep.with_aad_prefix(aad_prefix); } @@ -2639,8 +2639,7 @@ mod tests { .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = - format!("file_encryption_properties.column_keys_as_hex.{}", col_name); + let key = format!("file_encryption_properties.column_keys_as_hex.{col_name}"); let value = hex::encode(column_keys[i].clone()); table_config .parquet @@ -2673,8 +2672,7 @@ mod tests { .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = - format!("file_decryption_properties.column_keys_as_hex.{}", col_name); + let key = format!("file_decryption_properties.column_keys_as_hex.{col_name}"); let value = hex::encode(column_keys[i].clone()); table_config .parquet diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 69576a5726ea5..5efb10ace2f29 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -265,10 +265,9 @@ impl ParquetOptions { .set_data_page_row_count_limit(*data_page_row_count_limit) .set_bloom_filter_enabled(*bloom_filter_on_write); - let fep: Option = match file_encryption_properties { - Some(fe) => Some(fe.clone().into()), - None => None, - }; + let fep: Option = file_encryption_properties + .as_ref() + .map(|fe| fe.clone().into()); if fep.is_some() { builder = builder.with_file_encryption_properties(fep.unwrap()); @@ -598,10 +597,7 @@ mod tests { }; let fep: Option = - match props.file_encryption_properties() { - Some(fe) => Some(fe.into()), - None => None, - }; + props.file_encryption_properties().map(|fe| fe.into()); #[allow(deprecated)] // max_statistics_size TableParquetOptions { diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index f8a3889f558ce..61790d5768e5c 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -127,10 +127,10 @@ impl FileOpener for ParquetOpener { let mut enable_page_index = self.enable_page_index; let file_decryption_properties = self.file_decryption_properties.clone(); - + // For now, page index does not work with encrypted files. See: // https://github.com/apache/arrow-rs/issues/7629 - if file_decryption_properties.is_some() { + if file_decryption_properties.is_some() { enable_page_index = false; } From 506801e26e225d2e04da464e95bc354d8f1364f9 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 16:48:06 -0700 Subject: [PATCH 30/58] Add filter test for encrypted files. --- datafusion/core/src/dataframe/parquet.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 2feb67b0923a5..1d53263910116 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -311,6 +311,21 @@ mod tests { assert_eq!(num_rows_read, num_rows_written); + // Read encrypted parquet and subset rows + columns + let encrypted_parquet_df = ctx + .read_parquet(tempfile_str, ParquetReadOptions::default()) + .await?; + + // Select three columns and filter the results + // Test that the filter works as expected + let selected = encrypted_parquet_df + .clone() + .select_columns(&["c1", "c2", "c3"])? + .filter(col("c2").gt(lit(4)))?; + + let num_rows_selected = selected.count().await?; + assert_eq!(num_rows_selected, 14); + Ok(()) } } From 3058a903386c183e4c3741eddbe0faf447fe3d41 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 17:13:43 -0700 Subject: [PATCH 31/58] Cargo clippy changes. --- benchmarks/src/bin/dfbench.rs | 4 ++-- benchmarks/src/bin/imdb.rs | 2 +- benchmarks/src/bin/tpch.rs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/benchmarks/src/bin/dfbench.rs b/benchmarks/src/bin/dfbench.rs index 06337cb758885..41b64063c099c 100644 --- a/benchmarks/src/bin/dfbench.rs +++ b/benchmarks/src/bin/dfbench.rs @@ -60,11 +60,11 @@ pub async fn main() -> Result<()> { Options::Cancellation(opt) => opt.run().await, Options::Clickbench(opt) => opt.run().await, Options::H2o(opt) => opt.run().await, - Options::Imdb(opt) => opt.run().await, + Options::Imdb(opt) => Box::pin(opt.run()).await, Options::ParquetFilter(opt) => opt.run().await, Options::Sort(opt) => opt.run().await, Options::SortTpch(opt) => opt.run().await, - Options::Tpch(opt) => opt.run().await, + Options::Tpch(opt) => Box::pin(opt.run()).await, Options::TpchConvert(opt) => opt.run().await, } } diff --git a/benchmarks/src/bin/imdb.rs b/benchmarks/src/bin/imdb.rs index 13421f8a89a9b..5ce99928df662 100644 --- a/benchmarks/src/bin/imdb.rs +++ b/benchmarks/src/bin/imdb.rs @@ -53,7 +53,7 @@ pub async fn main() -> Result<()> { env_logger::init(); match ImdbOpt::from_args() { ImdbOpt::Benchmark(BenchmarkSubCommandOpt::DataFusionBenchmark(opt)) => { - opt.run().await + Box::pin(opt.run()).await } ImdbOpt::Convert(opt) => opt.run().await, } diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index 3270b082cfb43..ca2bb8e57c0ec 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -58,7 +58,7 @@ async fn main() -> Result<()> { env_logger::init(); match TpchOpt::from_args() { TpchOpt::Benchmark(BenchmarkSubCommandOpt::DataFusionBenchmark(opt)) => { - opt.run().await + Box::pin(opt.run()).await } TpchOpt::Convert(opt) => opt.run().await, } From bbeecfefe8d14ae193cdfc10a5f1e8883e07f8b0 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 18:02:06 -0700 Subject: [PATCH 32/58] Fix link in README.md --- datafusion-examples/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index d4ec507eb0e32..285762bb57e74 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -65,7 +65,7 @@ cargo run --example dataframe - [`flight_sql_server.rs`](examples/flight/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients - [`function_factory.rs`](examples/function_factory.rs): Register `CREATE FUNCTION` handler to implement SQL macros - [`optimizer_rule.rs`](examples/optimizer_rule.rs): Use a custom OptimizerRule to replace certain predicates -- [`parquet_encrypted.rs`](examples/parquet_index.rs): Read and write encrypted Parquet files using DataFusion +- [`parquet_encrypted.rs`](examples/parquet_encrypted.rs): Read and write encrypted Parquet files using DataFusion - [`parquet_index.rs`](examples/parquet_index.rs): Create an secondary index over several parquet files and use it to speed up queries - [`parquet_exec_visitor.rs`](examples/parquet_exec_visitor.rs): Extract statistics by visiting an ExecutionPlan after execution - [`parse_sql_expr.rs`](examples/parse_sql_expr.rs): Parse SQL text into DataFusion `Expr`. From 4ceb072d41449dbce66ae9cd0c8eeaba1d601d27 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 9 Jun 2025 18:06:08 -0700 Subject: [PATCH 33/58] Add issue tag for parallel writes. --- datafusion/datasource-parquet/src/file_format.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index e61dbd4d7e60a..0070de5f00f04 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1309,6 +1309,7 @@ impl FileSink for ParquetSink { if parquet_opts.global.file_encryption_properties.is_some() { // For now, arrow-rs does not support parallel writes with encryption + // See https://github.com/apache/arrow-rs/issues/7359 allow_single_file_parallelism = false; } From c998378a118280146aa559b1902e437de8b5ef8d Mon Sep 17 00:00:00 2001 From: Adam Reeve Date: Mon, 16 Jun 2025 16:16:18 +1200 Subject: [PATCH 34/58] Move file encryption and decryption properties out of global options --- .../examples/parquet_encrypted.rs | 24 ++----- datafusion/common/src/config.rs | 68 +++++++++++-------- .../common/src/file_options/parquet_writer.rs | 31 ++++----- datafusion/core/src/dataframe/parquet.rs | 28 ++------ .../src/datasource/file_format/options.rs | 19 +++++- datafusion/core/tests/parquet/encryption.rs | 40 ++++++----- .../datasource-parquet/src/file_format.rs | 8 +-- datafusion/datasource-parquet/src/source.rs | 4 +- datafusion/proto-common/src/from_proto/mod.rs | 3 +- .../proto/src/logical_plan/file_formats.rs | 3 +- 10 files changed, 111 insertions(+), 117 deletions(-) diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index f57330311e1f1..823c5bebc9e4f 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -16,13 +16,12 @@ // under the License. use datafusion::common::DataFusionError; -use datafusion::config::{ConfigFileDecryptionProperties, TableParquetOptions}; +use datafusion::config::TableParquetOptions; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; -use datafusion::execution::SessionStateBuilder; use datafusion::logical_expr::{col, lit}; use datafusion::parquet::encryption::decrypt::FileDecryptionProperties; use datafusion::parquet::encryption::encrypt::FileEncryptionProperties; -use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; +use datafusion::prelude::{ParquetReadOptions, SessionContext}; use tempfile::TempDir; #[tokio::main] @@ -56,7 +55,7 @@ async fn main() -> datafusion::common::Result<()> { // Write encrypted parquet let mut options = TableParquetOptions::default(); - options.global.file_encryption_properties = Some((&encrypt).into()); + options.crypto.file_encryption = Some((&encrypt).into()); parquet_df .write_parquet( tempfile_str.as_str(), @@ -66,19 +65,10 @@ async fn main() -> datafusion::common::Result<()> { .await?; // Read encrypted parquet - let mut sc = SessionConfig::new(); - let fd: ConfigFileDecryptionProperties = (&decrypt).into(); - sc.options_mut() - .execution - .parquet - .file_decryption_properties = Some(fd); - - let state = SessionStateBuilder::new().with_config(sc).build(); - let ctx: SessionContext = SessionContext::new_with_state(state); - - let encrypted_parquet_df = ctx - .read_parquet(tempfile_str, ParquetReadOptions::default()) - .await?; + let ctx: SessionContext = SessionContext::new(); + let read_options = ParquetReadOptions::default().file_decryption_properties(decrypt); + + let encrypted_parquet_df = ctx.read_parquet(tempfile_str, read_options).await?; // Show information from the dataframe println!("\n\n==============================================================================="); diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index f0caf0450c7a9..b33f3c0eba78a 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -229,13 +229,13 @@ impl ConfigField for ConfigFileEncryptionProperties { let desc = "Per column encryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{key_prefix}.column_keys_as_hex.{col_name}"); + let key = format!("{key_prefix}.column_keys_as_hex::{col_name}"); col_val.visit(v, key.as_str(), desc); } let desc = "Per column metadata"; for (col_name, col_val) in self.column_metadata_as_hex.iter() { - let key = format!("{key_prefix}.column_metadata_as_hex.{col_name}"); + let key = format!("{key_prefix}.column_metadata_as_hex::{col_name}"); col_val.visit(v, key.as_str(), desc); } @@ -251,17 +251,17 @@ impl ConfigField for ConfigFileEncryptionProperties { fn set(&mut self, key: &str, value: &str) -> Result<()> { // Any hex encoded values must be pre-encoded using // hex::encode() before calling set. - if key.starts_with("column_keys_as_hex.") { - let k = match key.split(".").collect::>()[..] { + if key.starts_with("column_keys_as_hex::") { + let k = match key.split("::").collect::>()[..] { [_meta] | [_meta, ""] => { return _config_err!( - "Invalid column name provided, missing name in column_keys_as_hex." + "Invalid column name provided, missing name in column_keys_as_hex::" ) } [_meta, k] => k.into(), _ => { return _config_err!( - "Invalid column name provided, found too many '.' in \"{key}\"" + "Invalid column name provided, found too many '::' in \"{key}\"" ) } }; @@ -269,17 +269,17 @@ impl ConfigField for ConfigFileEncryptionProperties { return Ok(()); }; - if key.starts_with("column_metadata_as_hex.") { - let k = match key.split(".").collect::>()[..] { + if key.starts_with("column_metadata_as_hex::") { + let k = match key.split("::").collect::>()[..] { [_meta] | [_meta, ""] => { return _config_err!( - "Invalid column name provided, missing name in column_metadata_as_hex." + "Invalid column name provided, missing name in column_metadata_as_hex::" ) } [_meta, k] => k.into(), _ => { return _config_err!( - "Invalid column name provided, found too many '.' in \"{key}\"" + "Invalid column name provided, found too many '::' in \"{key}\"" ) } }; @@ -419,7 +419,7 @@ impl ConfigField for ConfigFileDecryptionProperties { let desc = "Per column decryption keys"; for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{key_prefix}.column_keys_as_hex.{col_name}"); + let key = format!("{key_prefix}.column_keys_as_hex::{col_name}"); col_val.visit(v, key.as_str(), desc); } @@ -436,8 +436,8 @@ impl ConfigField for ConfigFileDecryptionProperties { fn set(&mut self, key: &str, value: &str) -> Result<()> { // Any hex encoded values must be pre-encoded using // hex::encode() before calling set. - if key.starts_with("column_keys_as_hex.") { - let k = match key.split(".").collect::>()[..] { + if key.starts_with("column_keys_as_hex::") { + let k = match key.split("::").collect::>()[..] { [_meta] | [_meta, ""] => { return _config_err!( "Invalid column name provided, missing name in column_keys_as_hex." @@ -446,7 +446,7 @@ impl ConfigField for ConfigFileDecryptionProperties { [_meta, k] => k.into(), _ => { return _config_err!( - "Invalid column name provided, found too many '.' in \"{key}\"" + "Invalid column name provided, found too many '::' in \"{key}\"" ) } }; @@ -930,12 +930,17 @@ config_namespace! { /// writing out already in-memory data, such as from a cached /// data frame. pub maximum_buffered_record_batches_per_stream: usize, default = 2 + } +} +config_namespace! { + /// Options for configuring Parquet Modular Encryption + pub struct ParquetEncryptionOptions { /// Optional file decryption properties - pub file_decryption_properties: Option, default = None + pub file_decryption: Option, default = None /// Optional file encryption properties - pub file_encryption_properties: Option, default = None + pub file_encryption: Option, default = None } } @@ -2070,6 +2075,8 @@ pub struct TableParquetOptions { /// ) /// ``` pub key_value_metadata: HashMap>, + /// Options for configuring Parquet modular encryption + pub crypto: ParquetEncryptionOptions, } impl TableParquetOptions { @@ -2097,7 +2104,9 @@ impl ConfigField for TableParquetOptions { fn visit(&self, v: &mut V, key_prefix: &str, description: &'static str) { self.global.visit(v, key_prefix, description); self.column_specific_options - .visit(v, key_prefix, description) + .visit(v, key_prefix, description); + self.crypto + .visit(v, &format!("{key_prefix}.crypto"), description); } fn set(&mut self, key: &str, value: &str) -> Result<()> { @@ -2118,6 +2127,8 @@ impl ConfigField for TableParquetOptions { }; self.key_value_metadata.insert(k, Some(value.into())); Ok(()) + } else if key.starts_with("crypto.") { + self.crypto.set(&key[7..], value) } else if key.contains("::") { self.column_specific_options.set(key, value) } else { @@ -2635,20 +2646,20 @@ mod tests { table_config .parquet .set( - "file_encryption_properties.encrypt_footer", + "crypto.file_encryption.encrypt_footer", config_encrypt.encrypt_footer.to_string().as_str(), ) .unwrap(); table_config .parquet .set( - "file_encryption_properties.footer_key_as_hex", + "crypto.file_encryption.footer_key_as_hex", config_encrypt.footer_key_as_hex.as_str(), ) .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = format!("file_encryption_properties.column_keys_as_hex.{col_name}"); + let key = format!("crypto.file_encryption.column_keys_as_hex::{col_name}"); let value = hex::encode(column_keys[i].clone()); table_config .parquet @@ -2657,10 +2668,10 @@ mod tests { } // Print matching final encryption config - // println!("{:#?}", table_config.parquet.global.file_encryption_properties); + // println!("{:#?}", table_config.parquet.crypto.file_encryption); assert_eq!( - table_config.parquet.global.file_encryption_properties, + table_config.parquet.crypto.file_encryption, Some(config_encrypt) ); @@ -2675,13 +2686,13 @@ mod tests { table_config .parquet .set( - "file_decryption_properties.footer_key_as_hex", + "crypto.file_decryption.footer_key_as_hex", config_decrypt.footer_key_as_hex.as_str(), ) .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = format!("file_decryption_properties.column_keys_as_hex.{col_name}"); + let key = format!("crypto.file_decryption.column_keys_as_hex::{col_name}"); let value = hex::encode(column_keys[i].clone()); table_config .parquet @@ -2690,20 +2701,19 @@ mod tests { } // Print matching final decryption config - // println!("{:#?}", table_config.parquet.global.file_decryption_properties); + // println!("{:#?}", table_config.parquet.crypto.file_decryption); assert_eq!( - table_config.parquet.global.file_decryption_properties, + table_config.parquet.crypto.file_decryption, Some(config_decrypt.clone()) ); // Set config directly let mut table_config = TableOptions::new(); table_config.set_config_format(ConfigFileType::PARQUET); - table_config.parquet.global.file_decryption_properties = - Some(config_decrypt.clone()); + table_config.parquet.crypto.file_decryption = Some(config_decrypt.clone()); assert_eq!( - table_config.parquet.global.file_decryption_properties, + table_config.parquet.crypto.file_decryption, Some(config_decrypt.clone()) ); } diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 5efb10ace2f29..60f0f4abb0c05 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -41,8 +41,6 @@ use parquet::{ schema::types::ColumnPath, }; -use parquet::encryption::encrypt::FileEncryptionProperties; - /// Options for writing parquet files #[derive(Clone, Debug)] pub struct ParquetWriterOptions { @@ -97,10 +95,17 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { global, column_specific_options, key_value_metadata, + crypto, } = table_parquet_options; let mut builder = global.into_writer_properties_builder()?; + if let Some(file_encryption_properties) = &crypto.file_encryption { + builder = builder.with_file_encryption_properties( + file_encryption_properties.clone().into(), + ); + } + // check that the arrow schema is present in the kv_metadata, if configured to do so if !global.skip_arrow_metadata && !key_value_metadata.contains_key(ARROW_SCHEMA_META_KEY) @@ -227,8 +232,6 @@ impl ParquetOptions { bloom_filter_on_write, bloom_filter_fpp, bloom_filter_ndv, - file_decryption_properties: _, - file_encryption_properties, // not in WriterProperties enable_page_index: _, @@ -265,14 +268,6 @@ impl ParquetOptions { .set_data_page_row_count_limit(*data_page_row_count_limit) .set_bloom_filter_enabled(*bloom_filter_on_write); - let fep: Option = file_encryption_properties - .as_ref() - .map(|fe| fe.clone().into()); - - if fep.is_some() { - builder = builder.with_file_encryption_properties(fep.unwrap()); - } - builder = { #[allow(deprecated)] builder.set_max_statistics_size( @@ -462,7 +457,8 @@ mod tests { use std::collections::HashMap; use crate::config::{ - ConfigFileEncryptionProperties, ParquetColumnOptions, ParquetOptions, + ConfigFileEncryptionProperties, ParquetColumnOptions, ParquetEncryptionOptions, + ParquetOptions, }; use super::*; @@ -513,8 +509,6 @@ mod tests { bloom_filter_on_write: !defaults.bloom_filter_on_write, bloom_filter_fpp: Some(0.42), bloom_filter_ndv: Some(42), - file_decryption_properties: None, - file_encryption_properties: None, // not in WriterProperties, but itemizing here to not skip newly added props enable_page_index: defaults.enable_page_index, @@ -624,8 +618,6 @@ mod tests { .unwrap_or_default(), bloom_filter_fpp: default_col_props.bloom_filter_fpp, bloom_filter_ndv: default_col_props.bloom_filter_ndv, - file_encryption_properties: fep, - file_decryption_properties: None, // not in WriterProperties enable_page_index: global_options_defaults.enable_page_index, @@ -648,6 +640,10 @@ mod tests { }, column_specific_options, key_value_metadata, + crypto: ParquetEncryptionOptions { + file_encryption: fep, + file_decryption: None, + }, } } @@ -702,6 +698,7 @@ mod tests { )] .into(), key_value_metadata: [(key, value)].into(), + crypto: Default::default(), }; let writer_props = WriterPropertiesBuilder::try_from(&table_parquet_opts) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 1d53263910116..fa0b34e413b76 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -249,8 +249,6 @@ mod tests { #[tokio::test] async fn roundtrip_parquet_with_encryption() -> Result<()> { - use crate::execution::SessionStateBuilder; - use datafusion_common::config::ConfigFileDecryptionProperties; use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; @@ -278,7 +276,7 @@ mod tests { // Write encrypted parquet using write_parquet let mut options = TableParquetOptions::default(); - options.global.file_encryption_properties = Some((&encrypt).into()); + options.crypto.file_encryption = Some((&encrypt).into()); df.write_parquet( tempfile_str.as_str(), @@ -289,22 +287,12 @@ mod tests { let num_rows_written = test_df.count().await?; // Read encrypted parquet - let mut sc = SessionConfig::new(); - let fd: ConfigFileDecryptionProperties = (&decrypt).into(); - sc.options_mut() - .execution - .parquet - .file_decryption_properties = Some(fd); + let ctx: SessionContext = SessionContext::new(); + let read_options = + ParquetReadOptions::default().file_decryption_properties(decrypt); - let state = SessionStateBuilder::new().with_config(sc).build(); - let ctx: SessionContext = SessionContext::new_with_state(state); - - ctx.register_parquet( - "roundtrip_parquet", - &tempfile_str, - ParquetReadOptions::default(), - ) - .await?; + ctx.register_parquet("roundtrip_parquet", &tempfile_str, read_options.clone()) + .await?; let df_enc = ctx.sql("SELECT * FROM roundtrip_parquet").await?; let num_rows_read = df_enc.count().await?; @@ -312,9 +300,7 @@ mod tests { assert_eq!(num_rows_read, num_rows_written); // Read encrypted parquet and subset rows + columns - let encrypted_parquet_df = ctx - .read_parquet(tempfile_str, ParquetReadOptions::default()) - .await?; + let encrypted_parquet_df = ctx.read_parquet(tempfile_str, read_options).await?; // Select three columns and filter the results // Test that the filter works as expected diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 9aaf1cf598113..9ef956ef7995e 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -43,6 +43,7 @@ use datafusion_common::{ use async_trait::async_trait; use datafusion_datasource_json::file_format::JsonFormat; use datafusion_expr::SortExpr; +use parquet::encryption::decrypt::FileDecryptionProperties; /// Options that control the reading of CSV files. /// @@ -252,6 +253,8 @@ pub struct ParquetReadOptions<'a> { pub schema: Option<&'a Schema>, /// Indicates how the file is sorted pub file_sort_order: Vec>, + /// Properties for decryption of Parquet files that use modular encryption + pub file_decryption_properties: Option, } impl Default for ParquetReadOptions<'_> { @@ -263,6 +266,7 @@ impl Default for ParquetReadOptions<'_> { skip_metadata: None, schema: None, file_sort_order: vec![], + file_decryption_properties: None, } } } @@ -313,6 +317,15 @@ impl<'a> ParquetReadOptions<'a> { self.file_sort_order = file_sort_order; self } + + /// Configure file decryption properties for reading encrypted Parquet files + pub fn file_decryption_properties( + mut self, + file_decryption_properties: FileDecryptionProperties, + ) -> Self { + self.file_decryption_properties = Some(file_decryption_properties); + self + } } /// Options that control the reading of ARROW files. @@ -574,7 +587,11 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> { config: &SessionConfig, table_options: TableOptions, ) -> ListingOptions { - let mut file_format = ParquetFormat::new().with_options(table_options.parquet); + let mut options = table_options.parquet; + if let Some(file_decryption_properties) = &self.file_decryption_properties { + options.crypto.file_decryption = Some(file_decryption_properties.into()); + } + let mut file_format = ParquetFormat::new().with_options(options); if let Some(parquet_pruning) = self.parquet_pruning { file_format = file_format.with_enable_pruning(parquet_pruning) diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs index 9c14f505f8eb4..3f180aab3017f 100644 --- a/datafusion/core/tests/parquet/encryption.rs +++ b/datafusion/core/tests/parquet/encryption.rs @@ -32,20 +32,18 @@ use std::fs::File; use std::path::{Path, PathBuf}; use std::sync::Arc; -use datafusion::execution::SessionStateBuilder; -use datafusion_common::config::ConfigFileDecryptionProperties; -use datafusion_execution::config::SessionConfig; use parquet::arrow::ArrowWriter; use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::encryption::encrypt::FileEncryptionProperties; use parquet::file::properties::WriterProperties; use tempfile::TempDir; -async fn read_parquet_test_data>( +async fn read_parquet_test_data<'a, T: Into>( path: T, ctx: &SessionContext, + options: ParquetReadOptions<'a>, ) -> Vec { - ctx.read_parquet(path.into(), ParquetReadOptions::default()) + ctx.read_parquet(path.into(), options) .await .unwrap() .collect() @@ -80,9 +78,13 @@ pub fn write_batches( async fn round_trip_encryption() { let ctx: SessionContext = SessionContext::new(); - let batches = - read_parquet_test_data("tests/data/filter_pushdown/single_file.gz.parquet", &ctx) - .await; + let options = ParquetReadOptions::default(); + let batches = read_parquet_test_data( + "tests/data/filter_pushdown/single_file.gz.parquet", + &ctx, + options, + ) + .await; let schema = batches[0].schema(); let footer_key = b"0123456789012345".to_vec(); // 128bit/16 @@ -108,19 +110,15 @@ async fn round_trip_encryption() { let num_rows_written = write_batches(tempfile.clone(), props, batches).unwrap(); // Read encrypted parquet - let mut sc = SessionConfig::new(); - let fd: ConfigFileDecryptionProperties = (&decrypt).into(); - sc.options_mut() - .execution - .parquet - .file_decryption_properties = Some(fd); - - let state = SessionStateBuilder::new().with_config(sc).build(); - let ctx: SessionContext = SessionContext::new_with_state(state); - - let encrypted_batches = - read_parquet_test_data(tempfile.into_os_string().into_string().unwrap(), &ctx) - .await; + let ctx: SessionContext = SessionContext::new(); + let options = ParquetReadOptions::default().file_decryption_properties(decrypt); + + let encrypted_batches = read_parquet_test_data( + tempfile.into_os_string().into_string().unwrap(), + &ctx, + options, + ) + .await; let num_rows_read = encrypted_batches .iter() diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 0070de5f00f04..59663fe5100a2 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -350,8 +350,7 @@ impl FileFormat for ParquetFormat { Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?), None => None, }; - let config_file_decryption_properties = - &self.options.global.file_decryption_properties; + let config_file_decryption_properties = &self.options.crypto.file_decryption; let file_decryption_properties: Option = match config_file_decryption_properties { Some(cfd) => { @@ -416,8 +415,7 @@ impl FileFormat for ParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { - let config_file_decryption_properties = - &self.options.global.file_decryption_properties; + let config_file_decryption_properties = &self.options.crypto.file_decryption; let file_decryption_properties: Option = match config_file_decryption_properties { Some(cfd) => { @@ -1307,7 +1305,7 @@ impl FileSink for ParquetSink { let mut allow_single_file_parallelism = parquet_opts.global.allow_single_file_parallelism; - if parquet_opts.global.file_encryption_properties.is_some() { + if parquet_opts.crypto.file_encryption.is_some() { // For now, arrow-rs does not support parallel writes with encryption // See https://github.com/apache/arrow-rs/issues/7359 allow_single_file_parallelism = false; diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 56181b636b2fa..90aa3267e6ea5 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -477,8 +477,8 @@ impl FileSource for ParquetSource { let file_decryption_properties = self .table_parquet_options() - .global - .file_decryption_properties + .crypto + .file_decryption .as_ref() .map(|props| Arc::new(props.clone().into())); diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 83c4592921940..0823e150268de 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -988,8 +988,6 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { protobuf::parquet_options::CoerceInt96Opt::CoerceInt96(v) => Some(v), }).unwrap_or(None), skip_arrow_metadata: value.skip_arrow_metadata, - file_decryption_properties: None, - file_encryption_properties: None, }) } } @@ -1068,6 +1066,7 @@ impl TryFrom<&protobuf::TableParquetOptions> for TableParquetOptions { .unwrap(), column_specific_options, key_value_metadata: Default::default(), + crypto: Default::default(), }) } } diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 31b525ce01e46..620442c79e72c 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -513,8 +513,6 @@ impl From<&ParquetOptionsProto> for ParquetOptions { coerce_int96: proto.coerce_int96_opt.as_ref().map(|opt| match opt { parquet_options::CoerceInt96Opt::CoerceInt96(coerce_int96) => coerce_int96.clone(), }), - file_decryption_properties: None, - file_encryption_properties: None, } } } @@ -578,6 +576,7 @@ impl From<&TableParquetOptionsProto> for TableParquetOptions { .iter() .map(|(k, v)| (k.clone(), Some(v.clone()))) .collect(), + crypto: Default::default(), } } } From 7780b335aff74c1f59eb8ea1723c4bd4ddb236a4 Mon Sep 17 00:00:00 2001 From: Adam Reeve Date: Mon, 16 Jun 2025 17:25:05 +1200 Subject: [PATCH 35/58] Use config_namespace_with_hashmap for column encryption/decryption props --- datafusion/common/src/config.rs | 629 +++++++++++++++----------------- 1 file changed, 294 insertions(+), 335 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index b33f3c0eba78a..9b9125a03982d 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -194,339 +194,6 @@ macro_rules! config_namespace { } } } - -#[derive(Clone, Default, Debug, PartialEq)] -pub struct ConfigFileEncryptionProperties { - pub encrypt_footer: bool, // default = false - pub footer_key_as_hex: String, - pub footer_key_metadata_as_hex: String, - pub column_keys_as_hex: HashMap, - pub column_metadata_as_hex: HashMap, - pub aad_prefix_as_hex: String, - pub store_aad_prefix: bool, // default = false -} - -impl ConfigFileEncryptionProperties { - /// Return new default TableParquetOptions - pub fn new() -> Self { - Self::default() - } -} - -impl ConfigField for ConfigFileEncryptionProperties { - fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { - let key = format!("{key_prefix}.encrypt_footer"); - let desc = "Encrypt the footer"; - self.encrypt_footer.visit(v, key.as_str(), desc); - - let key = format!("{key_prefix}.footer_key_as_hex"); - let desc = "Key to use for the parquet footer"; - self.footer_key_as_hex.visit(v, key.as_str(), desc); - - let key = format!("{key_prefix}.footer_key_metadata_as_hex"); - let desc = "Metadata to use for the parquet footer"; - self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc); - - let desc = "Per column encryption keys"; - for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{key_prefix}.column_keys_as_hex::{col_name}"); - col_val.visit(v, key.as_str(), desc); - } - - let desc = "Per column metadata"; - for (col_name, col_val) in self.column_metadata_as_hex.iter() { - let key = format!("{key_prefix}.column_metadata_as_hex::{col_name}"); - col_val.visit(v, key.as_str(), desc); - } - - let key = format!("{key_prefix}.aad_prefix_as_hex"); - let desc = "AAD prefix to use"; - self.aad_prefix_as_hex.visit(v, key.as_str(), desc); - - let key = format!("{key_prefix}.store_aad_prefix"); - let desc = "If true, store the AAD prefix"; - self.store_aad_prefix.visit(v, key.as_str(), desc); - } - - fn set(&mut self, key: &str, value: &str) -> Result<()> { - // Any hex encoded values must be pre-encoded using - // hex::encode() before calling set. - if key.starts_with("column_keys_as_hex::") { - let k = match key.split("::").collect::>()[..] { - [_meta] | [_meta, ""] => { - return _config_err!( - "Invalid column name provided, missing name in column_keys_as_hex::" - ) - } - [_meta, k] => k.into(), - _ => { - return _config_err!( - "Invalid column name provided, found too many '::' in \"{key}\"" - ) - } - }; - self.column_keys_as_hex.insert(k, String::from(value)); - return Ok(()); - }; - - if key.starts_with("column_metadata_as_hex::") { - let k = match key.split("::").collect::>()[..] { - [_meta] | [_meta, ""] => { - return _config_err!( - "Invalid column name provided, missing name in column_metadata_as_hex::" - ) - } - [_meta, k] => k.into(), - _ => { - return _config_err!( - "Invalid column name provided, found too many '::' in \"{key}\"" - ) - } - }; - self.column_metadata_as_hex.insert(k, String::from(value)); - return Ok(()); - }; - - let (key, rem) = key.split_once('.').unwrap_or((key, "")); - match key { - "encrypt_footer" => self.encrypt_footer.set(rem, value.as_ref()), - "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), - "footer_key_metadata_as_hex" => { - self.footer_key_metadata_as_hex.set(rem, value.as_ref()) - } - "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), - "store_aad_prefix" => self.store_aad_prefix.set(rem, value.as_ref()), - _ => _config_err!( - "Config value \"{}\" not found on ConfigFileEncryptionProperties", - key - ), - } - } -} - -#[cfg(feature = "parquet")] -impl From for FileEncryptionProperties { - fn from(val: ConfigFileEncryptionProperties) -> Self { - let mut fep = FileEncryptionProperties::builder( - hex::decode(val.footer_key_as_hex).unwrap(), - ) - .with_plaintext_footer(!val.encrypt_footer) - .with_aad_prefix_storage(val.store_aad_prefix); - - if !val.footer_key_metadata_as_hex.is_empty() { - fep = fep.with_footer_key_metadata( - hex::decode(&val.footer_key_metadata_as_hex) - .expect("Invalid footer key metadata"), - ); - } - - for (column_name, encryption_key) in val.column_keys_as_hex.iter() { - let encryption_key = - hex::decode(encryption_key).expect("Invalid column encryption key"); - let key_metadata = val - .column_metadata_as_hex - .get(column_name) - .map(|x| hex::decode(x).expect("Invalid column metadata")); - match key_metadata { - Some(key_metadata) => { - fep = fep.with_column_key_and_metadata( - column_name, - encryption_key, - key_metadata, - ); - } - None => { - fep = fep.with_column_key(column_name, encryption_key); - } - } - } - - if !val.aad_prefix_as_hex.is_empty() { - let aad_prefix: Vec = - hex::decode(&val.aad_prefix_as_hex).expect("Invalid AAD prefix"); - fep = fep.with_aad_prefix(aad_prefix); - } - fep.build().unwrap() - } -} - -#[cfg(feature = "parquet")] -impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { - fn from(f: &FileEncryptionProperties) -> Self { - let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); - - let mut column_keys_as_hex: HashMap = HashMap::new(); - let mut column_metadata_as_hex: HashMap = HashMap::new(); - - for (i, column_name) in column_names_vec.iter().enumerate() { - column_keys_as_hex - .insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); - let metadata_as_hex: Option = - column_metas_vec.get(i).map(hex::encode); - if let Some(metadata_as_hex) = metadata_as_hex { - column_metadata_as_hex.insert(column_name.clone(), metadata_as_hex); - } - } - let mut aad_prefix: Vec = Vec::new(); - if let Some(prefix) = f.aad_prefix() { - aad_prefix = prefix.clone(); - } - ConfigFileEncryptionProperties { - encrypt_footer: f.encrypt_footer(), - footer_key_as_hex: hex::encode(f.footer_key()), - footer_key_metadata_as_hex: f - .footer_key_metadata() - .map(hex::encode) - .unwrap_or_default(), - column_keys_as_hex, - column_metadata_as_hex, - aad_prefix_as_hex: hex::encode(aad_prefix), - store_aad_prefix: f.store_aad_prefix(), - } - } -} - -#[derive(Clone, Debug, PartialEq)] -pub struct ConfigFileDecryptionProperties { - pub footer_key_as_hex: String, - pub column_keys_as_hex: HashMap, - pub aad_prefix_as_hex: String, - pub footer_signature_verification: bool, // default = true -} - -impl Default for ConfigFileDecryptionProperties { - fn default() -> Self { - ConfigFileDecryptionProperties { - footer_key_as_hex: String::new(), - column_keys_as_hex: HashMap::new(), - aad_prefix_as_hex: String::new(), - footer_signature_verification: true, - } - } -} -impl ConfigFileDecryptionProperties { - /// Return new default TableParquetOptions - pub fn new() -> Self { - Self::default() - } -} - -impl ConfigField for ConfigFileDecryptionProperties { - fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { - let key = format!("{key_prefix}.footer_key_as_hex"); - let desc = "Key to use for the parquet footer"; - self.footer_key_as_hex.visit(v, key.as_str(), desc); - - let desc = "Per column decryption keys"; - for (col_name, col_val) in self.column_keys_as_hex.iter() { - let key = format!("{key_prefix}.column_keys_as_hex::{col_name}"); - col_val.visit(v, key.as_str(), desc); - } - - let key = format!("{key_prefix}.aad_prefix_as_hex"); - let desc = "AAD prefix to use"; - self.aad_prefix_as_hex.visit(v, key.as_str(), desc); - - let key = format!("{key_prefix}.footer_signature_verification"); - let desc = "If true, verify the footer signature"; - self.footer_signature_verification - .visit(v, key.as_str(), desc); - } - - fn set(&mut self, key: &str, value: &str) -> Result<()> { - // Any hex encoded values must be pre-encoded using - // hex::encode() before calling set. - if key.starts_with("column_keys_as_hex::") { - let k = match key.split("::").collect::>()[..] { - [_meta] | [_meta, ""] => { - return _config_err!( - "Invalid column name provided, missing name in column_keys_as_hex." - ) - } - [_meta, k] => k.into(), - _ => { - return _config_err!( - "Invalid column name provided, found too many '::' in \"{key}\"" - ) - } - }; - self.column_keys_as_hex.insert(k, String::from(value)); - return Ok(()); - }; - - let (key, rem) = key.split_once('.').unwrap_or((key, "")); - match key { - "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), - "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), - "footer_signature_verification" => { - self.footer_signature_verification.set(rem, value.as_ref()) - } - _ => _config_err!( - "Config value \"{}\" not found on ConfigFileEncryptionProperties", - key - ), - } - } -} - -#[cfg(feature = "parquet")] -impl From for FileDecryptionProperties { - fn from(val: ConfigFileDecryptionProperties) -> Self { - let mut column_names: Vec<&str> = Vec::new(); - let mut column_keys: Vec> = Vec::new(); - - for (col_name, encryption_key) in val.column_keys_as_hex.iter() { - column_names.push(col_name.as_str()); - column_keys.push( - hex::decode(encryption_key).expect("Invalid column decryption key"), - ); - } - - let mut fep = FileDecryptionProperties::builder( - hex::decode(val.footer_key_as_hex).expect("Invalid footer key"), - ) - .with_column_keys(column_names, column_keys) - .unwrap(); - - if !val.footer_signature_verification { - fep = fep.disable_footer_signature_verification(); - } - - if !val.aad_prefix_as_hex.is_empty() { - let aad_prefix = - hex::decode(&val.aad_prefix_as_hex).expect("Invalid AAD prefix"); - fep = fep.with_aad_prefix(aad_prefix); - } - - fep.build().unwrap() - } -} - -#[cfg(feature = "parquet")] -impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { - fn from(f: &FileDecryptionProperties) -> Self { - let (column_names_vec, column_keys_vec) = f.column_keys(); - let mut column_keys_as_hex: HashMap = HashMap::new(); - for (i, column_name) in column_names_vec.iter().enumerate() { - column_keys_as_hex - .insert(column_name.clone(), hex::encode(column_keys_vec[i].clone())); - } - - let mut aad_prefix: Vec = Vec::new(); - if let Some(prefix) = f.aad_prefix() { - aad_prefix = prefix.clone(); - } - ConfigFileDecryptionProperties { - footer_key_as_hex: hex::encode( - f.footer_key(None).unwrap_or_default().as_ref(), - ), - column_keys_as_hex, - aad_prefix_as_hex: hex::encode(aad_prefix), - footer_signature_verification: f.check_plaintext_footer_integrity(), - } - } -} - config_namespace! { /// Options related to catalog and directory scanning /// @@ -2279,6 +1946,298 @@ config_namespace_with_hashmap! { } } +#[derive(Clone, Debug, Default, PartialEq)] +pub struct ConfigFileEncryptionProperties { + pub encrypt_footer: bool, + pub footer_key_as_hex: String, + pub footer_key_metadata_as_hex: String, + pub column_encryption_properties: HashMap, + pub aad_prefix_as_hex: String, + pub store_aad_prefix: bool, +} + +config_namespace_with_hashmap! { + pub struct ColumnEncryptionProperties { + /// Per column encryption key + pub column_key_as_hex: String, default = "".to_string() + /// Per column encryption key metadata + pub column_metadata_as_hex: Option, default = None + } +} + +impl ConfigField for ConfigFileEncryptionProperties { + fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { + let key = format!("{key_prefix}.encrypt_footer"); + let desc = "Encrypt the footer"; + self.encrypt_footer.visit(v, key.as_str(), desc); + + let key = format!("{key_prefix}.footer_key_as_hex"); + let desc = "Key to use for the parquet footer"; + self.footer_key_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{key_prefix}.footer_key_metadata_as_hex"); + let desc = "Metadata to use for the parquet footer"; + self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{key_prefix}.aad_prefix_as_hex"); + let desc = "AAD prefix to use"; + self.aad_prefix_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{key_prefix}.store_aad_prefix"); + let desc = "If true, store the AAD prefix"; + self.store_aad_prefix.visit(v, key.as_str(), desc); + + self.aad_prefix_as_hex.visit(v, key.as_str(), desc); + } + + fn set(&mut self, key: &str, value: &str) -> Result<()> { + // Any hex encoded values must be pre-encoded using + // hex::encode() before calling set. + + if key.contains("::") { + // Handle any column specific properties + return self + .column_encryption_properties + .set(&key, &value.to_string()); + }; + + let (key, rem) = key.split_once('.').unwrap_or((key, "")); + match key { + "encrypt_footer" => self.encrypt_footer.set(rem, value.as_ref()), + "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), + "footer_key_metadata_as_hex" => { + self.footer_key_metadata_as_hex.set(rem, value.as_ref()) + } + "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), + "store_aad_prefix" => self.store_aad_prefix.set(rem, value.as_ref()), + _ => _config_err!( + "Config value \"{}\" not found on ConfigFileEncryptionProperties", + key + ), + } + } +} + +#[cfg(feature = "parquet")] +impl From for FileEncryptionProperties { + fn from(val: ConfigFileEncryptionProperties) -> Self { + let mut fep = FileEncryptionProperties::builder( + hex::decode(val.footer_key_as_hex).unwrap(), + ) + .with_plaintext_footer(!val.encrypt_footer) + .with_aad_prefix_storage(val.store_aad_prefix); + + if !val.footer_key_metadata_as_hex.is_empty() { + fep = fep.with_footer_key_metadata( + hex::decode(&val.footer_key_metadata_as_hex) + .expect("Invalid footer key metadata"), + ); + } + + for (column_name, encryption_props) in val.column_encryption_properties.iter() { + let encryption_key = hex::decode(&encryption_props.column_key_as_hex) + .expect("Invalid column encryption key"); + let key_metadata = encryption_props + .column_metadata_as_hex + .as_ref() + .map(|x| hex::decode(x).expect("Invalid column metadata")); + match key_metadata { + Some(key_metadata) => { + fep = fep.with_column_key_and_metadata( + column_name, + encryption_key, + key_metadata, + ); + } + None => { + fep = fep.with_column_key(column_name, encryption_key); + } + } + } + + if !val.aad_prefix_as_hex.is_empty() { + let aad_prefix: Vec = + hex::decode(&val.aad_prefix_as_hex).expect("Invalid AAD prefix"); + fep = fep.with_aad_prefix(aad_prefix); + } + fep.build().unwrap() + } +} + +#[cfg(feature = "parquet")] +impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { + fn from(f: &FileEncryptionProperties) -> Self { + let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); + + let mut column_encryption_properties: HashMap< + String, + ColumnEncryptionProperties, + > = HashMap::new(); + + for (i, column_name) in column_names_vec.iter().enumerate() { + let column_key_as_hex = hex::encode(&column_keys_vec[i]); + let column_metadata_as_hex: Option = + column_metas_vec.get(i).map(hex::encode); + column_encryption_properties.insert( + column_name.clone(), + ColumnEncryptionProperties { + column_key_as_hex, + column_metadata_as_hex, + }, + ); + } + let mut aad_prefix: Vec = Vec::new(); + if let Some(prefix) = f.aad_prefix() { + aad_prefix = prefix.clone(); + } + ConfigFileEncryptionProperties { + encrypt_footer: f.encrypt_footer(), + footer_key_as_hex: hex::encode(f.footer_key()), + footer_key_metadata_as_hex: f + .footer_key_metadata() + .map(hex::encode) + .unwrap_or_default(), + column_encryption_properties, + aad_prefix_as_hex: hex::encode(aad_prefix), + store_aad_prefix: f.store_aad_prefix(), + } + } +} + +#[derive(Clone, Debug, PartialEq)] +pub struct ConfigFileDecryptionProperties { + pub footer_key_as_hex: String, + pub column_decryption_properties: HashMap, + pub aad_prefix_as_hex: String, + pub footer_signature_verification: bool, // default = true +} + +config_namespace_with_hashmap! { + pub struct ColumnDecryptionProperties { + /// Per column encryption key + pub column_key_as_hex: String, default = "".to_string() + } +} + +impl Default for ConfigFileDecryptionProperties { + fn default() -> Self { + ConfigFileDecryptionProperties { + footer_key_as_hex: String::new(), + column_decryption_properties: Default::default(), + aad_prefix_as_hex: String::new(), + footer_signature_verification: true, + } + } +} + +impl ConfigField for ConfigFileDecryptionProperties { + fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { + let key = format!("{key_prefix}.footer_key_as_hex"); + let desc = "Key to use for the parquet footer"; + self.footer_key_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{key_prefix}.aad_prefix_as_hex"); + let desc = "AAD prefix to use"; + self.aad_prefix_as_hex.visit(v, key.as_str(), desc); + + let key = format!("{key_prefix}.footer_signature_verification"); + let desc = "If true, verify the footer signature"; + self.footer_signature_verification + .visit(v, key.as_str(), desc); + + self.column_decryption_properties.visit(v, key_prefix, desc); + } + + fn set(&mut self, key: &str, value: &str) -> Result<()> { + // Any hex encoded values must be pre-encoded using + // hex::encode() before calling set. + + if key.contains("::") { + // Handle any column specific properties + return self + .column_decryption_properties + .set(&key, &value.to_string()); + }; + + let (key, rem) = key.split_once('.').unwrap_or((key, "")); + match key { + "footer_key_as_hex" => self.footer_key_as_hex.set(rem, value.as_ref()), + "aad_prefix_as_hex" => self.aad_prefix_as_hex.set(rem, value.as_ref()), + "footer_signature_verification" => { + self.footer_signature_verification.set(rem, value.as_ref()) + } + _ => _config_err!( + "Config value \"{}\" not found on ConfigFileEncryptionProperties", + key + ), + } + } +} + +#[cfg(feature = "parquet")] +impl From for FileDecryptionProperties { + fn from(val: ConfigFileDecryptionProperties) -> Self { + let mut column_names: Vec<&str> = Vec::new(); + let mut column_keys: Vec> = Vec::new(); + + for (col_name, decryption_properties) in val.column_decryption_properties.iter() { + column_names.push(col_name.as_str()); + column_keys.push( + hex::decode(&decryption_properties.column_key_as_hex) + .expect("Invalid column decryption key"), + ); + } + + let mut fep = FileDecryptionProperties::builder( + hex::decode(val.footer_key_as_hex).expect("Invalid footer key"), + ) + .with_column_keys(column_names, column_keys) + .unwrap(); + + if !val.footer_signature_verification { + fep = fep.disable_footer_signature_verification(); + } + + if !val.aad_prefix_as_hex.is_empty() { + let aad_prefix = + hex::decode(&val.aad_prefix_as_hex).expect("Invalid AAD prefix"); + fep = fep.with_aad_prefix(aad_prefix); + } + + fep.build().unwrap() + } +} + +#[cfg(feature = "parquet")] +impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { + fn from(f: &FileDecryptionProperties) -> Self { + let (column_names_vec, column_keys_vec) = f.column_keys(); + let mut column_decryption_properties: HashMap< + String, + ColumnDecryptionProperties, + > = HashMap::new(); + for (i, column_name) in column_names_vec.iter().enumerate() { + let props = ColumnDecryptionProperties { + column_key_as_hex: hex::encode(column_keys_vec[i].clone()), + }; + column_decryption_properties.insert(column_name.clone(), props); + } + + let mut aad_prefix: Vec = Vec::new(); + if let Some(prefix) = f.aad_prefix() { + aad_prefix = prefix.clone(); + } + ConfigFileDecryptionProperties { + footer_key_as_hex: hex::encode( + f.footer_key(None).unwrap_or_default().as_ref(), + ), + column_decryption_properties, + aad_prefix_as_hex: hex::encode(aad_prefix), + footer_signature_verification: f.check_plaintext_footer_integrity(), + } + } +} + config_namespace! { /// Options controlling CSV format pub struct CsvOptions { @@ -2659,7 +2618,7 @@ mod tests { .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = format!("crypto.file_encryption.column_keys_as_hex::{col_name}"); + let key = format!("crypto.file_encryption.column_key_as_hex::{col_name}"); let value = hex::encode(column_keys[i].clone()); table_config .parquet @@ -2692,7 +2651,7 @@ mod tests { .unwrap(); for (i, col_name) in column_names.iter().enumerate() { - let key = format!("crypto.file_decryption.column_keys_as_hex::{col_name}"); + let key = format!("crypto.file_decryption.column_key_as_hex::{col_name}"); let value = hex::encode(column_keys[i].clone()); table_config .parquet From 3af85bcfeaf34081c047809fc2644834f5a6c6f4 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 24 Jun 2025 18:16:46 -0700 Subject: [PATCH 36/58] Remove outdated docs on crypto settings. Signed-off-by: Corwin Joy --- docs/source/user-guide/configs.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4a044ebf30442..42282e39e41f5 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -81,8 +81,6 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | | datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.file_decryption_properties | NULL | Optional file decryption properties | -| datafusion.execution.parquet.file_encryption_properties | NULL | Optional file encryption properties | | datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | | datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | From 325529393e02458cc87ac1e65d39d3be9f236be2 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 24 Jun 2025 21:50:05 -0700 Subject: [PATCH 37/58] 1. Add docs for using encryption configuration. 2. Add example SQL for using encryption from CLI. 3. Fix removed variables in test for configuration information. 4. Clippy and cargo fmt. Signed-off-by: Corwin Joy --- .../tests/sql/encrypted_parquet.sql | 75 +++++++++++++++++++ datafusion/common/src/config.rs | 41 +++++++--- .../test_files/information_schema.slt | 4 - 3 files changed, 107 insertions(+), 13 deletions(-) create mode 100644 datafusion-cli/tests/sql/encrypted_parquet.sql diff --git a/datafusion-cli/tests/sql/encrypted_parquet.sql b/datafusion-cli/tests/sql/encrypted_parquet.sql new file mode 100644 index 0000000000000..ad866a8fdd7e2 --- /dev/null +++ b/datafusion-cli/tests/sql/encrypted_parquet.sql @@ -0,0 +1,75 @@ +/* + Test parquet encryption and decryption in DataFusion SQL. + See datafusion/common/src/config.rs for equivalent rust code +*/ + +-- Keys are hex encoded, you can generate these via encode, e.g. +select encode('0123456789012345', 'hex'); +/* +Expected output: ++----------------------------------------------+ +| encode(Utf8("0123456789012345"),Utf8("hex")) | ++----------------------------------------------+ +| 30313233343536373839303132333435 | ++----------------------------------------------+ +*/ + +CREATE EXTERNAL TABLE encrypted_parquet_table +( +double_field double, +float_field float +) +STORED AS PARQUET LOCATION 'pq/' OPTIONS ( + 'format.crypto.file_encryption.encrypt_footer' 'true', + 'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" + -- Same for decryption + 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" +); + +CREATE TABLE temp_table ( + double_field double, + float_field float +); + +INSERT INTO temp_table VALUES(-1.0, -1.0); +INSERT INTO temp_table VALUES(1.0, 2.0); +INSERT INTO temp_table VALUES(3.0, 4.0); +INSERT INTO temp_table VALUES(5.0, 6.0); + +INSERT INTO TABLE encrypted_parquet_table(double_field, float_field) SELECT * FROM temp_table; + +SELECT * FROM encrypted_parquet_table +WHERE double_field > 0.0 AND float_field > 0.0; + +/* +Expected output: ++--------------+-------------+ +| double_field | float_field | ++--------------+-------------+ +| 1.0 | 2.0 | +| 5.0 | 6.0 | +| 3.0 | 4.0 | ++--------------+-------------+ +*/ + +CREATE EXTERNAL TABLE parquet_table +( +double_field double, +float_field float +) +STORED AS PARQUET LOCATION 'pq/'; + +SELECT * FROM parquet_table; +/* +Expected output: +Parquet error: Parquet error: Parquet file has an encrypted footer but decryption properties were not provided +*/ + + + + + diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 9b9125a03982d..c9e18a0557ea8 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -1743,6 +1743,22 @@ pub struct TableParquetOptions { /// ``` pub key_value_metadata: HashMap>, /// Options for configuring Parquet modular encryption + /// See ConfigFileEncryptionProperties and ConfigFileDecryptionProperties in datafusion/common/src/config.rs + /// These can be set via 'format.crypto', for example: + /// ```sql + /// OPTIONS ( + /// 'format.crypto.file_encryption.encrypt_footer' 'true', + /// 'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" */ + /// 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + /// 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" + /// -- Same for decryption + /// 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + /// 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + /// 'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" + /// ) + /// ``` + /// See datafusion-cli/tests/sql/encrypted_parquet.sql for a more complete example. + /// Note that keys must be provided as in hex format since these are binary strings. pub crypto: ParquetEncryptionOptions, } @@ -1794,8 +1810,8 @@ impl ConfigField for TableParquetOptions { }; self.key_value_metadata.insert(k, Some(value.into())); Ok(()) - } else if key.starts_with("crypto.") { - self.crypto.set(&key[7..], value) + } else if let Some(crypto_feature) = key.strip_prefix("crypto.") { + self.crypto.set(crypto_feature, value) } else if key.contains("::") { self.column_specific_options.set(key, value) } else { @@ -1948,11 +1964,17 @@ config_namespace_with_hashmap! { #[derive(Clone, Debug, Default, PartialEq)] pub struct ConfigFileEncryptionProperties { + /// Should the parquet footer be encrypted pub encrypt_footer: bool, + /// Key to use for the parquet footer encoded in hex format pub footer_key_as_hex: String, + /// Metadata information for footer key pub footer_key_metadata_as_hex: String, + /// HashMap of column names --> (key in hex format, metadata) pub column_encryption_properties: HashMap, + /// AAD prefix string uniquely identifies the file and prevents file swapping pub aad_prefix_as_hex: String, + /// If true, store the AAD prefix in the file pub store_aad_prefix: bool, } @@ -1996,9 +2018,7 @@ impl ConfigField for ConfigFileEncryptionProperties { if key.contains("::") { // Handle any column specific properties - return self - .column_encryption_properties - .set(&key, &value.to_string()); + return self.column_encryption_properties.set(key, value); }; let (key, rem) = key.split_once('.').unwrap_or((key, "")); @@ -2106,10 +2126,15 @@ impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { #[derive(Clone, Debug, PartialEq)] pub struct ConfigFileDecryptionProperties { + /// Binary string to use for the parquet footer encoded in hex format pub footer_key_as_hex: String, + /// HashMap of column names --> key in hex format pub column_decryption_properties: HashMap, + /// AAD prefix string uniquely identifies the file and prevents file swapping pub aad_prefix_as_hex: String, - pub footer_signature_verification: bool, // default = true + /// If true, then verify signature for files with plaintext footers. + /// default = true + pub footer_signature_verification: bool, } config_namespace_with_hashmap! { @@ -2154,9 +2179,7 @@ impl ConfigField for ConfigFileDecryptionProperties { if key.contains("::") { // Handle any column specific properties - return self - .column_decryption_properties - .set(&key, &value.to_string()); + return self.column_decryption_properties.set(key, value); }; let (key, rem) = key.split_once('.').unwrap_or((key, "")); diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 4c91c82730682..dc8b7680d83e1 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -241,8 +241,6 @@ datafusion.execution.parquet.dictionary_enabled true datafusion.execution.parquet.dictionary_page_size_limit 1048576 datafusion.execution.parquet.enable_page_index true datafusion.execution.parquet.encoding NULL -datafusion.execution.parquet.file_decryption_properties NULL -datafusion.execution.parquet.file_encryption_properties NULL datafusion.execution.parquet.max_row_group_size 1048576 datafusion.execution.parquet.max_statistics_size 4096 datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 @@ -354,8 +352,6 @@ datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionar datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting -datafusion.execution.parquet.file_decryption_properties NULL Optional file decryption properties -datafusion.execution.parquet.file_encryption_properties NULL Optional file encryption properties datafusion.execution.parquet.max_row_group_size 1048576 (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. datafusion.execution.parquet.max_statistics_size 4096 (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. From 9972ecbf9c381987270028732963e832494de4cc Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 25 Jun 2025 15:44:02 -0700 Subject: [PATCH 38/58] Update code to add missing ParquetOpener parameter due to merge from main Signed-off-by: Corwin Joy --- datafusion/datasource-parquet/src/opener.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5c774b23a26f1..2b561deef94fe 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -97,9 +97,6 @@ pub(super) struct ParquetOpener { impl FileOpener for ParquetOpener { fn open(&self, file_meta: FileMeta, file: PartitionedFile) -> Result { - let predicate_creation_errors = MetricBuilder::new(&self.metrics) - .global_counter("num_predicate_creation_errors"); - let file_range = file_meta.range.clone(); let extensions = file_meta.extensions.clone(); let file_name = file_meta.location().to_string(); @@ -652,6 +649,7 @@ mod test { schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory), enable_row_group_stats_pruning: true, coerce_int96: None, + file_decryption_properties: None, } }; @@ -736,6 +734,7 @@ mod test { schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory), enable_row_group_stats_pruning: true, coerce_int96: None, + file_decryption_properties: None, } }; @@ -832,6 +831,7 @@ mod test { schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory), enable_row_group_stats_pruning: true, coerce_int96: None, + file_decryption_properties: None, } }; let make_meta = || FileMeta { From 068e65d528ef0b92cff2e17c8d95cac26c728b5e Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 25 Jun 2025 17:18:56 -0700 Subject: [PATCH 39/58] Add CLI documentation for Parquet options and provide an encryption example Signed-off-by: Corwin Joy --- docs/source/user-guide/cli/datasources.md | 39 +++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/docs/source/user-guide/cli/datasources.md b/docs/source/user-guide/cli/datasources.md index afc4f6c0c50f5..a1e967d8634d3 100644 --- a/docs/source/user-guide/cli/datasources.md +++ b/docs/source/user-guide/cli/datasources.md @@ -190,6 +190,45 @@ STORED AS PARQUET LOCATION '/mnt/nyctaxi/'; ``` +### Parquet Specific Options +You can specify additional options for parquet files using the `OPTIONS` clause. +For example, to read and write a parquet directory with encryption settings you could use: + +```sql +CREATE EXTERNAL TABLE encrypted_parquet_table +( +double_field double, +float_field float +) +STORED AS PARQUET LOCATION 'pq/' OPTIONS ( + -- encryption + 'format.crypto.file_encryption.encrypt_footer' 'true', + 'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" + -- decryption + 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" +); +``` + +Here the keys are specified in hexadecimal format because they are binary data. These can be encoded in SQL using: + +```sql +select encode('0123456789012345', 'hex'); +/* ++----------------------------------------------+ +| encode(Utf8("0123456789012345"),Utf8("hex")) | ++----------------------------------------------+ +| 30313233343536373839303132333435 | ++----------------------------------------------+ +*/ +``` +For more details on the available options, refer to the Rust +[TableParquetOptions](https://docs.rs/datafusion/latest/datafusion/common/config/struct.TableParquetOptions.html) +documentation in DataFusion. + ## CSV DataFusion will infer the CSV schema automatically or you can provide it explicitly. From ecd5f93a5c612b4a3e65809fb85eb335cf50e56a Mon Sep 17 00:00:00 2001 From: Adam Reeve Date: Thu, 26 Jun 2025 12:38:34 +1200 Subject: [PATCH 40/58] Use ConfigFileDecryptionProperties in ParquetReadOptions Signed-off-by: Adam Reeve --- datafusion-examples/examples/parquet_encrypted.rs | 3 ++- datafusion/core/src/dataframe/parquet.rs | 2 +- datafusion/core/src/datasource/file_format/options.rs | 9 ++++----- datafusion/core/tests/parquet/encryption.rs | 3 ++- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/datafusion-examples/examples/parquet_encrypted.rs b/datafusion-examples/examples/parquet_encrypted.rs index 823c5bebc9e4f..e9e239b7a1c32 100644 --- a/datafusion-examples/examples/parquet_encrypted.rs +++ b/datafusion-examples/examples/parquet_encrypted.rs @@ -66,7 +66,8 @@ async fn main() -> datafusion::common::Result<()> { // Read encrypted parquet let ctx: SessionContext = SessionContext::new(); - let read_options = ParquetReadOptions::default().file_decryption_properties(decrypt); + let read_options = + ParquetReadOptions::default().file_decryption_properties((&decrypt).into()); let encrypted_parquet_df = ctx.read_parquet(tempfile_str, read_options).await?; diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index fa0b34e413b76..a2bec74ee1408 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -289,7 +289,7 @@ mod tests { // Read encrypted parquet let ctx: SessionContext = SessionContext::new(); let read_options = - ParquetReadOptions::default().file_decryption_properties(decrypt); + ParquetReadOptions::default().file_decryption_properties((&decrypt).into()); ctx.register_parquet("roundtrip_parquet", &tempfile_str, read_options.clone()) .await?; diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 9ef956ef7995e..02b792823a827 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -34,7 +34,7 @@ use crate::error::Result; use crate::execution::context::{SessionConfig, SessionState}; use arrow::datatypes::{DataType, Schema, SchemaRef}; -use datafusion_common::config::TableOptions; +use datafusion_common::config::{ConfigFileDecryptionProperties, TableOptions}; use datafusion_common::{ DEFAULT_ARROW_EXTENSION, DEFAULT_AVRO_EXTENSION, DEFAULT_CSV_EXTENSION, DEFAULT_JSON_EXTENSION, DEFAULT_PARQUET_EXTENSION, @@ -43,7 +43,6 @@ use datafusion_common::{ use async_trait::async_trait; use datafusion_datasource_json::file_format::JsonFormat; use datafusion_expr::SortExpr; -use parquet::encryption::decrypt::FileDecryptionProperties; /// Options that control the reading of CSV files. /// @@ -254,7 +253,7 @@ pub struct ParquetReadOptions<'a> { /// Indicates how the file is sorted pub file_sort_order: Vec>, /// Properties for decryption of Parquet files that use modular encryption - pub file_decryption_properties: Option, + pub file_decryption_properties: Option, } impl Default for ParquetReadOptions<'_> { @@ -321,7 +320,7 @@ impl<'a> ParquetReadOptions<'a> { /// Configure file decryption properties for reading encrypted Parquet files pub fn file_decryption_properties( mut self, - file_decryption_properties: FileDecryptionProperties, + file_decryption_properties: ConfigFileDecryptionProperties, ) -> Self { self.file_decryption_properties = Some(file_decryption_properties); self @@ -589,7 +588,7 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> { ) -> ListingOptions { let mut options = table_options.parquet; if let Some(file_decryption_properties) = &self.file_decryption_properties { - options.crypto.file_decryption = Some(file_decryption_properties.into()); + options.crypto.file_decryption = Some(file_decryption_properties.clone()); } let mut file_format = ParquetFormat::new().with_options(options); diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs index 3f180aab3017f..203c985428bc0 100644 --- a/datafusion/core/tests/parquet/encryption.rs +++ b/datafusion/core/tests/parquet/encryption.rs @@ -111,7 +111,8 @@ async fn round_trip_encryption() { // Read encrypted parquet let ctx: SessionContext = SessionContext::new(); - let options = ParquetReadOptions::default().file_decryption_properties(decrypt); + let options = + ParquetReadOptions::default().file_decryption_properties((&decrypt).into()); let encrypted_batches = read_parquet_test_data( tempfile.into_os_string().into_string().unwrap(), From f3e694500ec737587f4e6bfd0a43a6dfa20415ec Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 25 Jun 2025 21:38:30 -0700 Subject: [PATCH 41/58] Implement default for ConfigFileEncryptionProperties Signed-off-by: Corwin Joy --- datafusion/common/src/config.rs | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index bf4e36583479a..43814162a1609 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -2056,9 +2056,10 @@ config_namespace_with_hashmap! { } } -#[derive(Clone, Debug, Default, PartialEq)] +#[derive(Clone, Debug, PartialEq)] pub struct ConfigFileEncryptionProperties { /// Should the parquet footer be encrypted + /// default is true pub encrypt_footer: bool, /// Key to use for the parquet footer encoded in hex format pub footer_key_as_hex: String, @@ -2069,9 +2070,24 @@ pub struct ConfigFileEncryptionProperties { /// AAD prefix string uniquely identifies the file and prevents file swapping pub aad_prefix_as_hex: String, /// If true, store the AAD prefix in the file + /// default is false pub store_aad_prefix: bool, } +// Setup to match EncryptionPropertiesBuilder::new() +impl Default for ConfigFileEncryptionProperties { + fn default() -> Self { + ConfigFileEncryptionProperties { + encrypt_footer: true, + footer_key_as_hex: String::new(), + footer_key_metadata_as_hex: String::new(), + column_encryption_properties: Default::default(), + aad_prefix_as_hex: String::new(), + store_aad_prefix: false, + } + } +} + config_namespace_with_hashmap! { pub struct ColumnEncryptionProperties { /// Per column encryption key @@ -2238,6 +2254,7 @@ config_namespace_with_hashmap! { } } +// Setup to match DecryptionPropertiesBuilder::new() impl Default for ConfigFileDecryptionProperties { fn default() -> Self { ConfigFileDecryptionProperties { From a7005a300de521f83f246461cd9ba7b6f5ce21a4 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 25 Jun 2025 22:17:11 -0700 Subject: [PATCH 42/58] Add sqllogictest for parquet with encryption Signed-off-by: Corwin Joy --- .../tests/sql/encrypted_parquet.sql | 75 ---------------- .../test_files/encrypted_parquet.slt | 89 +++++++++++++++++++ 2 files changed, 89 insertions(+), 75 deletions(-) delete mode 100644 datafusion-cli/tests/sql/encrypted_parquet.sql create mode 100644 datafusion/sqllogictest/test_files/encrypted_parquet.slt diff --git a/datafusion-cli/tests/sql/encrypted_parquet.sql b/datafusion-cli/tests/sql/encrypted_parquet.sql deleted file mode 100644 index ad866a8fdd7e2..0000000000000 --- a/datafusion-cli/tests/sql/encrypted_parquet.sql +++ /dev/null @@ -1,75 +0,0 @@ -/* - Test parquet encryption and decryption in DataFusion SQL. - See datafusion/common/src/config.rs for equivalent rust code -*/ - --- Keys are hex encoded, you can generate these via encode, e.g. -select encode('0123456789012345', 'hex'); -/* -Expected output: -+----------------------------------------------+ -| encode(Utf8("0123456789012345"),Utf8("hex")) | -+----------------------------------------------+ -| 30313233343536373839303132333435 | -+----------------------------------------------+ -*/ - -CREATE EXTERNAL TABLE encrypted_parquet_table -( -double_field double, -float_field float -) -STORED AS PARQUET LOCATION 'pq/' OPTIONS ( - 'format.crypto.file_encryption.encrypt_footer' 'true', - 'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" - 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" - 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" - -- Same for decryption - 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" - 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" - 'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" -); - -CREATE TABLE temp_table ( - double_field double, - float_field float -); - -INSERT INTO temp_table VALUES(-1.0, -1.0); -INSERT INTO temp_table VALUES(1.0, 2.0); -INSERT INTO temp_table VALUES(3.0, 4.0); -INSERT INTO temp_table VALUES(5.0, 6.0); - -INSERT INTO TABLE encrypted_parquet_table(double_field, float_field) SELECT * FROM temp_table; - -SELECT * FROM encrypted_parquet_table -WHERE double_field > 0.0 AND float_field > 0.0; - -/* -Expected output: -+--------------+-------------+ -| double_field | float_field | -+--------------+-------------+ -| 1.0 | 2.0 | -| 5.0 | 6.0 | -| 3.0 | 4.0 | -+--------------+-------------+ -*/ - -CREATE EXTERNAL TABLE parquet_table -( -double_field double, -float_field float -) -STORED AS PARQUET LOCATION 'pq/'; - -SELECT * FROM parquet_table; -/* -Expected output: -Parquet error: Parquet error: Parquet file has an encrypted footer but decryption properties were not provided -*/ - - - - - diff --git a/datafusion/sqllogictest/test_files/encrypted_parquet.slt b/datafusion/sqllogictest/test_files/encrypted_parquet.slt new file mode 100644 index 0000000000000..d580b7d1ad2b8 --- /dev/null +++ b/datafusion/sqllogictest/test_files/encrypted_parquet.slt @@ -0,0 +1,89 @@ +# 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. + +# Test parquet encryption and decryption in DataFusion SQL. +# See datafusion/common/src/config.rs for equivalent rust code + +statement count 0 +CREATE EXTERNAL TABLE encrypted_parquet_table +( +double_field double, +float_field float +) +STORED AS PARQUET LOCATION 'test_files/scratch/encrypted_parquet/' OPTIONS ( + -- Configure encryption for reading and writing Parquet files + -- Encryption properties + 'format.crypto.file_encryption.encrypt_footer' 'true', + 'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" + -- Decryption properties + 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" +) + +statement count 0 +CREATE TABLE temp_table ( + double_field double, + float_field float +) + +query I +INSERT INTO temp_table VALUES(-1.0, -1.0) +---- +1 + +query I +INSERT INTO temp_table VALUES(1.0, 2.0) +---- +1 + +query I +INSERT INTO temp_table VALUES(3.0, 4.0) +---- +1 + +query I +INSERT INTO temp_table VALUES(5.0, 6.0) +---- +1 + +query I +INSERT INTO TABLE encrypted_parquet_table(double_field, float_field) SELECT * FROM temp_table +---- +4 + +query RR +SELECT * FROM encrypted_parquet_table +WHERE double_field > 0.0 AND float_field > 0.0 +ORDER BY double_field +---- +1 2 +3 4 +5 6 + +statement count 0 +CREATE EXTERNAL TABLE parquet_table +( +double_field double, +float_field float +) +STORED AS PARQUET LOCATION 'test_files/scratch/encrypted_parquet/' + +query error DataFusion error: Parquet error: Parquet error: Parquet file has an encrypted footer but decryption properties were not provided +SELECT * FROM parquet_table From 48166f44cf441744c4e06bc21faf6fdea8ee9808 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 26 Jun 2025 15:52:13 -0700 Subject: [PATCH 43/58] Apply prettier changes from CI Signed-off-by: Corwin Joy --- docs/source/user-guide/cli/datasources.md | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/docs/source/user-guide/cli/datasources.md b/docs/source/user-guide/cli/datasources.md index a1e967d8634d3..c15b8a5e46c99 100644 --- a/docs/source/user-guide/cli/datasources.md +++ b/docs/source/user-guide/cli/datasources.md @@ -191,6 +191,7 @@ LOCATION '/mnt/nyctaxi/'; ``` ### Parquet Specific Options + You can specify additional options for parquet files using the `OPTIONS` clause. For example, to read and write a parquet directory with encryption settings you could use: @@ -204,11 +205,11 @@ STORED AS PARQUET LOCATION 'pq/' OPTIONS ( -- encryption 'format.crypto.file_encryption.encrypt_footer' 'true', 'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" - 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" - 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" - -- decryption - 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" - 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" + 'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" + -- decryption + 'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345" + 'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450" 'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451" ); ``` @@ -225,8 +226,9 @@ select encode('0123456789012345', 'hex'); +----------------------------------------------+ */ ``` -For more details on the available options, refer to the Rust -[TableParquetOptions](https://docs.rs/datafusion/latest/datafusion/common/config/struct.TableParquetOptions.html) + +For more details on the available options, refer to the Rust +[TableParquetOptions](https://docs.rs/datafusion/latest/datafusion/common/config/struct.TableParquetOptions.html) documentation in DataFusion. ## CSV From 29821d4db14ecee632f0c844ea3514402021d45d Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 26 Jun 2025 17:57:55 -0700 Subject: [PATCH 44/58] Begin adding config for encryption. --- Cargo.lock | 2 ++ Cargo.toml | 2 +- datafusion/common/Cargo.toml | 5 +++- datafusion/common/src/config.rs | 24 ++++++++------- .../common/src/file_options/parquet_writer.rs | 7 ++++- datafusion/core/Cargo.toml | 5 ++++ datafusion/core/src/dataframe/parquet.rs | 1 + datafusion/core/tests/parquet/encryption.rs | 1 + datafusion/datasource-parquet/Cargo.toml | 6 ++++ .../datasource-parquet/src/file_format.rs | 29 +++++++++++++------ datafusion/datasource-parquet/src/opener.rs | 4 ++- datafusion/datasource-parquet/src/source.rs | 3 ++ 12 files changed, 66 insertions(+), 23 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 418b13e7ac535..fbd9d940fbb8e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1861,6 +1861,7 @@ dependencies = [ "env_logger", "flate2", "futures", + "hex", "insta", "itertools 0.14.0", "log", @@ -2149,6 +2150,7 @@ dependencies = [ "datafusion-physical-plan", "datafusion-session", "futures", + "hex", "itertools 0.14.0", "log", "object_store", diff --git a/Cargo.toml b/Cargo.toml index 9c52913770e6c..06b6fd76d74fa 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -150,6 +150,7 @@ env_logger = "0.11" futures = "0.3" half = { version = "2.6.0", default-features = false } hashbrown = { version = "0.14.5", features = ["raw"] } +hex = { version = "0.4.3" } indexmap = "2.9.0" itertools = "0.14" log = "^0.4" @@ -159,7 +160,6 @@ parquet = { version = "55.1.0", default-features = false, features = [ "arrow", "async", "object_store", - "encryption", ] } pbjson = { version = "0.7.0" } pbjson-types = "0.7" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index b356f249b79bb..e2311523ca70a 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -40,6 +40,9 @@ name = "datafusion_common" [features] avro = ["apache-avro"] backtrace = [] +parquet_encryption = [ + "parquet", "parquet/encryption", "dep:hex", +] pyarrow = ["pyo3", "arrow/pyarrow", "parquet"] force_hash_collisions = [] recursive_protection = ["dep:recursive"] @@ -58,7 +61,7 @@ base64 = "0.22.1" chrono = { workspace = true } half = { workspace = true } hashbrown = { workspace = true } -hex = "0.4.3" +hex = { workspace = true, optional = true } indexmap = { workspace = true } libc = "0.2.174" log = { workspace = true } diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 43814162a1609..7392ce9a260c9 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -29,12 +29,16 @@ use std::error::Error; use std::fmt::{self, Display}; use std::str::FromStr; -#[cfg(feature = "parquet")] +#[cfg(feature = "parquet_encryption")] use hex; -#[cfg(feature = "parquet")] -use parquet::encryption::decrypt::FileDecryptionProperties; -#[cfg(feature = "parquet")] -use parquet::encryption::encrypt::FileEncryptionProperties; +#[cfg(feature = "parquet_encryption")] +pub use parquet::encryption::decrypt::FileDecryptionProperties; +#[cfg(feature = "parquet_encryption")] +pub use parquet::encryption::encrypt::FileEncryptionProperties; +#[cfg(not(feature = "parquet_encryption"))] +pub struct FileDecryptionProperties; +#[cfg(not(feature = "parquet_encryption"))] +pub struct FileEncryptionProperties; /// A macro that wraps a configuration struct and automatically derives /// [`Default`] and [`ConfigField`] for it, allowing it to be used @@ -2148,7 +2152,7 @@ impl ConfigField for ConfigFileEncryptionProperties { } } -#[cfg(feature = "parquet")] +#[cfg(feature = "parquet_encryption")] impl From for FileEncryptionProperties { fn from(val: ConfigFileEncryptionProperties) -> Self { let mut fep = FileEncryptionProperties::builder( @@ -2194,7 +2198,7 @@ impl From for FileEncryptionProperties { } } -#[cfg(feature = "parquet")] +#[cfg(feature = "parquet_encryption")] impl From<&FileEncryptionProperties> for ConfigFileEncryptionProperties { fn from(f: &FileEncryptionProperties) -> Self { let (column_names_vec, column_keys_vec, column_metas_vec) = f.column_keys(); @@ -2308,7 +2312,7 @@ impl ConfigField for ConfigFileDecryptionProperties { } } -#[cfg(feature = "parquet")] +#[cfg(feature = "parquet_encryption")] impl From for FileDecryptionProperties { fn from(val: ConfigFileDecryptionProperties) -> Self { let mut column_names: Vec<&str> = Vec::new(); @@ -2342,7 +2346,7 @@ impl From for FileDecryptionProperties { } } -#[cfg(feature = "parquet")] +#[cfg(feature = "parquet_encryption")] impl From<&FileDecryptionProperties> for ConfigFileDecryptionProperties { fn from(f: &FileDecryptionProperties) -> Self { let (column_names_vec, column_keys_vec) = f.column_keys(); @@ -2688,7 +2692,7 @@ mod tests { ); } - #[cfg(feature = "parquet")] + #[cfg(feature = "parquet_encryption")] #[test] fn parquet_table_encryption() { use crate::config::{ diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 60f0f4abb0c05..8b22de0cadb00 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -95,11 +95,13 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { global, column_specific_options, key_value_metadata, + #[allow(unused)] crypto, } = table_parquet_options; let mut builder = global.into_writer_properties_builder()?; - + + #[cfg(feature = "parquet_encryption")] if let Some(file_encryption_properties) = &crypto.file_encryption { builder = builder.with_file_encryption_properties( file_encryption_properties.clone().into(), @@ -590,8 +592,11 @@ mod tests { HashMap::from([(COL_NAME.into(), configured_col_props)]) }; + #[cfg(feature = "parquet_encryption")] let fep: Option = props.file_encryption_properties().map(|fe| fe.into()); + #[cfg(not(feature = "parquet_encryption"))] + let fep: Option = None; #[allow(deprecated)] // max_statistics_size TableParquetOptions { diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 9747f44240604..0b09791b945e6 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -61,6 +61,7 @@ default = [ "unicode_expressions", "compression", "parquet", + "parquet_encryption", "recursive_protection", ] encoding_expressions = ["datafusion-functions/encoding_expressions"] @@ -68,6 +69,9 @@ encoding_expressions = ["datafusion-functions/encoding_expressions"] force_hash_collisions = ["datafusion-physical-plan/force_hash_collisions", "datafusion-common/force_hash_collisions"] math_expressions = ["datafusion-functions/math_expressions"] parquet = ["datafusion-common/parquet", "dep:parquet", "datafusion-datasource-parquet"] +parquet_encryption = [ + "dep:parquet", "parquet/encryption", "dep:hex", +] pyarrow = ["datafusion-common/pyarrow", "parquet"] regex_expressions = [ "datafusion-functions/regex_expressions", @@ -127,6 +131,7 @@ datafusion-session = { workspace = true } datafusion-sql = { workspace = true } flate2 = { version = "1.1.2", optional = true } futures = { workspace = true } +hex = { workspace = true, optional = true } itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index a2bec74ee1408..83bb60184fb92 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -247,6 +247,7 @@ mod tests { Ok(()) } + #[cfg(feature = "parquet_encryption")] #[tokio::test] async fn roundtrip_parquet_with_encryption() -> Result<()> { use parquet::encryption::decrypt::FileDecryptionProperties; diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs index 203c985428bc0..8e90b9aaa9551 100644 --- a/datafusion/core/tests/parquet/encryption.rs +++ b/datafusion/core/tests/parquet/encryption.rs @@ -74,6 +74,7 @@ pub fn write_batches( Ok(num_rows) } +#[cfg(feature = "parquet_encryption")] #[tokio::test] async fn round_trip_encryption() { let ctx: SessionContext = SessionContext::new(); diff --git a/datafusion/datasource-parquet/Cargo.toml b/datafusion/datasource-parquet/Cargo.toml index b6a548c998dc2..98c4ef987dbfa 100644 --- a/datafusion/datasource-parquet/Cargo.toml +++ b/datafusion/datasource-parquet/Cargo.toml @@ -47,6 +47,7 @@ datafusion-physical-optimizer = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-session = { workspace = true } futures = { workspace = true } +hex = { workspace = true, optional = true } itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } @@ -64,3 +65,8 @@ workspace = true [lib] name = "datafusion_datasource_parquet" path = "src/mod.rs" + +[features] +parquet_encryption = [ + "parquet/encryption", "dep:hex", +] diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 59663fe5100a2..b7b9cc8d98363 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -38,7 +38,7 @@ use datafusion_datasource::write::demux::DemuxedStreamReceiver; use arrow::compute::sum; use arrow::datatypes::{DataType, Field, FieldRef}; -use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; +use datafusion_common::config::{ConfigField, FileDecryptionProperties, ConfigFileType, TableParquetOptions}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{ @@ -78,7 +78,7 @@ use parquet::arrow::arrow_writer::{ use parquet::arrow::async_reader::MetadataFetch; use parquet::arrow::{parquet_to_arrow_schema, ArrowSchemaConverter, AsyncArrowWriter}; use parquet::basic::Type; -use parquet::encryption::decrypt::FileDecryptionProperties; + use parquet::errors::ParquetError; use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder}; @@ -88,6 +88,7 @@ use parquet::schema::types::SchemaDescriptor; use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver, Sender}; + /// Initial writing buffer size. Note this is just a size hint for efficiency. It /// will grow beyond the set value if needed. const INITIAL_BUFFER_BYTES: usize = 1048576; @@ -350,15 +351,17 @@ impl FileFormat for ParquetFormat { Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?), None => None, }; - let config_file_decryption_properties = &self.options.crypto.file_decryption; + #[cfg(feature = "parquet_encryption")] let file_decryption_properties: Option = - match config_file_decryption_properties { + match &self.options.crypto.file_decryption { Some(cfd) => { let fd: FileDecryptionProperties = cfd.clone().into(); Some(fd) } None => None, }; + #[cfg(not(feature = "parquet_encryption"))] + let file_decryption_properties: Option = None; let mut schemas: Vec<_> = futures::stream::iter(objects) .map(|object| { fetch_schema_with_location( @@ -415,15 +418,17 @@ impl FileFormat for ParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { - let config_file_decryption_properties = &self.options.crypto.file_decryption; + #[cfg(feature = "parquet_encryption")] let file_decryption_properties: Option = - match config_file_decryption_properties { + match &self.options.crypto.file_decryption { Some(cfd) => { let fd: FileDecryptionProperties = cfd.clone().into(); Some(fd) } None => None, }; + #[cfg(not(feature = "parquet_encryption"))] + let file_decryption_properties: Option = None; let stats = fetch_statistics( store.as_ref(), table_schema, @@ -959,14 +964,20 @@ pub async fn fetch_parquet_metadata( store: &dyn ObjectStore, meta: &ObjectMeta, size_hint: Option, + #[allow(unused)] decryption_properties: Option<&FileDecryptionProperties>, ) -> Result { let file_size = meta.size; let fetch = ObjectStoreFetch::new(store, meta); - ParquetMetaDataReader::new() - .with_prefetch_hint(size_hint) - .with_decryption_properties(decryption_properties) + #[allow(unused_mut)] + let mut reader = ParquetMetaDataReader::new() + .with_prefetch_hint(size_hint); + + #[cfg(feature = "parquet_encryption")] + let reader = reader.with_decryption_properties(decryption_properties); + + reader .load_and_finish(fetch, file_size) .await .map_err(DataFusionError::from) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 2b561deef94fe..d77b44cb94f8e 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -36,6 +36,7 @@ use datafusion_common::pruning::{ PruningStatistics, }; use datafusion_common::{exec_err, Result}; +use datafusion_common::config::FileDecryptionProperties; use datafusion_datasource::PartitionedFile; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_optimizer::pruning::PruningPredicate; @@ -47,9 +48,9 @@ use log::debug; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; -use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::file::metadata::ParquetMetaDataReader; + /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { /// Execution partition index @@ -210,6 +211,7 @@ impl FileOpener for ParquetOpener { // pruning predicates. Thus default to not requesting if from the // underlying reader. let mut options = ArrowReaderOptions::new().with_page_index(false); + #[cfg(feature = "parquet_encryption")] if let Some(fd_val) = file_decryption_properties { options = options.with_file_decryption_properties((*fd_val).clone()); } diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index f2e7823151002..5fc91fa13ce74 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -475,12 +475,15 @@ impl FileSource for ParquetSource { Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _ }); + #[cfg(feature = "parquet_encryption")] let file_decryption_properties = self .table_parquet_options() .crypto .file_decryption .as_ref() .map(|props| Arc::new(props.clone().into())); + #[cfg(not(feature = "parquet_encryption"))] + let file_decryption_properties = None; let coerce_int96 = self .table_parquet_options From 01291305eb5b4aa9b333ab4e55407456b1a62a8e Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 30 Jun 2025 16:23:18 -0700 Subject: [PATCH 45/58] Fix merge errors. --- datafusion/common/src/file_options/parquet_writer.rs | 7 +------ datafusion/datasource-parquet/src/opener.rs | 1 - 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 9fce8c643a5fb..378acf2822102 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -107,12 +107,7 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { file_encryption_properties.clone().into(), ); } - - if let Some(file_encryption_properties) = &crypto.file_encryption { - builder = builder.with_file_encryption_properties( - file_encryption_properties.clone().into(), - ); - } + // check that the arrow schema is present in the kv_metadata, if configured to do so if !global.skip_arrow_metadata diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 01d504247852b..3517bcd6fc82f 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -46,7 +46,6 @@ use log::debug; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; -use parquet::encryption::decrypt::FileDecryptionProperties; use parquet::file::metadata::ParquetMetaDataReader; From 511c94bc9ee3b17e6c57584b034d9b13dd61d633 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 30 Jun 2025 20:20:12 -0700 Subject: [PATCH 46/58] Fix config dependency in datafusion/core/Cargo.toml --- datafusion/core/Cargo.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 0b09791b945e6..26d7be6a4372c 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -70,7 +70,8 @@ force_hash_collisions = ["datafusion-physical-plan/force_hash_collisions", "data math_expressions = ["datafusion-functions/math_expressions"] parquet = ["datafusion-common/parquet", "dep:parquet", "datafusion-datasource-parquet"] parquet_encryption = [ - "dep:parquet", "parquet/encryption", "dep:hex", + "dep:parquet", "parquet/encryption", "datafusion-common/parquet_encryption", + "datafusion-datasource-parquet/parquet_encryption", "dep:hex", ] pyarrow = ["datafusion-common/pyarrow", "parquet"] regex_expressions = [ From efd38d1598e6b7d09200abc36967b127f8b15ef4 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 30 Jun 2025 20:29:05 -0700 Subject: [PATCH 47/58] ci reformat --- Cargo.toml | 2 +- datafusion/common/Cargo.toml | 4 +++- datafusion/common/src/file_options/parquet_writer.rs | 1 - datafusion/core/Cargo.toml | 7 +++++-- datafusion/datasource-parquet/Cargo.toml | 3 ++- datafusion/datasource-parquet/src/file_format.rs | 11 +++++------ datafusion/datasource-parquet/src/opener.rs | 3 +-- 7 files changed, 17 insertions(+), 14 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 29cddada6b27d..aca4a2e24bd09 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -150,8 +150,8 @@ env_logger = "0.11" futures = "0.3" half = { version = "2.6.0", default-features = false } hashbrown = { version = "0.14.5", features = ["raw"] } -indexmap = "2.10.0" hex = { version = "0.4.3" } +indexmap = "2.10.0" itertools = "0.14" log = "^0.4" object_store = { version = "0.12.2", default-features = false } diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index e2311523ca70a..8d331db339007 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -41,7 +41,9 @@ name = "datafusion_common" avro = ["apache-avro"] backtrace = [] parquet_encryption = [ - "parquet", "parquet/encryption", "dep:hex", + "parquet", + "parquet/encryption", + "dep:hex", ] pyarrow = ["pyo3", "arrow/pyarrow", "parquet"] force_hash_collisions = [] diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 378acf2822102..de496b4f0019c 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -107,7 +107,6 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { file_encryption_properties.clone().into(), ); } - // check that the arrow schema is present in the kv_metadata, if configured to do so if !global.skip_arrow_metadata diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 26d7be6a4372c..69cfeece00db5 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -70,8 +70,11 @@ force_hash_collisions = ["datafusion-physical-plan/force_hash_collisions", "data math_expressions = ["datafusion-functions/math_expressions"] parquet = ["datafusion-common/parquet", "dep:parquet", "datafusion-datasource-parquet"] parquet_encryption = [ - "dep:parquet", "parquet/encryption", "datafusion-common/parquet_encryption", - "datafusion-datasource-parquet/parquet_encryption", "dep:hex", + "dep:parquet", + "parquet/encryption", + "datafusion-common/parquet_encryption", + "datafusion-datasource-parquet/parquet_encryption", + "dep:hex", ] pyarrow = ["datafusion-common/pyarrow", "parquet"] regex_expressions = [ diff --git a/datafusion/datasource-parquet/Cargo.toml b/datafusion/datasource-parquet/Cargo.toml index 03ca6641f7e14..43643f992ba6e 100644 --- a/datafusion/datasource-parquet/Cargo.toml +++ b/datafusion/datasource-parquet/Cargo.toml @@ -69,5 +69,6 @@ path = "src/mod.rs" [features] parquet_encryption = [ - "parquet/encryption", "dep:hex", + "parquet/encryption", + "dep:hex", ] diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 71a3d36868e92..dd6d91d1f8494 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -38,7 +38,9 @@ use datafusion_datasource::write::demux::DemuxedStreamReceiver; use arrow::compute::sum; use arrow::datatypes::{DataType, Field, FieldRef}; -use datafusion_common::config::{ConfigField, FileDecryptionProperties, ConfigFileType, TableParquetOptions}; +use datafusion_common::config::{ + ConfigField, ConfigFileType, FileDecryptionProperties, TableParquetOptions, +}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{ @@ -88,7 +90,6 @@ use parquet::schema::types::SchemaDescriptor; use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver, Sender}; - /// Initial writing buffer size. Note this is just a size hint for efficiency. It /// will grow beyond the set value if needed. const INITIAL_BUFFER_BYTES: usize = 1048576; @@ -964,15 +965,13 @@ pub async fn fetch_parquet_metadata( store: &dyn ObjectStore, meta: &ObjectMeta, size_hint: Option, - #[allow(unused)] - decryption_properties: Option<&FileDecryptionProperties>, + #[allow(unused)] decryption_properties: Option<&FileDecryptionProperties>, ) -> Result { let file_size = meta.size; let fetch = ObjectStoreFetch::new(store, meta); #[allow(unused_mut)] - let mut reader = ParquetMetaDataReader::new() - .with_prefetch_hint(size_hint); + let mut reader = ParquetMetaDataReader::new().with_prefetch_hint(size_hint); #[cfg(feature = "parquet_encryption")] let reader = reader.with_decryption_properties(decryption_properties); diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 3517bcd6fc82f..1fcab6f3c230b 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -31,8 +31,8 @@ use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::{FieldRef, SchemaRef, TimeUnit}; use arrow::error::ArrowError; -use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_common::config::FileDecryptionProperties; +use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_datasource::PartitionedFile; use datafusion_physical_expr::PhysicalExprSchemaRewriter; use datafusion_physical_expr_common::physical_expr::{ @@ -48,7 +48,6 @@ use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::metadata::ParquetMetaDataReader; - /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { /// Execution partition index From 69f7c5776bd2d3ab739ef1d10bfe4dc7d8ec83b6 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Mon, 30 Jun 2025 20:48:39 -0700 Subject: [PATCH 48/58] Fix missing Cargo dependency. --- datafusion/datasource-parquet/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/datasource-parquet/Cargo.toml b/datafusion/datasource-parquet/Cargo.toml index 43643f992ba6e..8a75a445c8ffb 100644 --- a/datafusion/datasource-parquet/Cargo.toml +++ b/datafusion/datasource-parquet/Cargo.toml @@ -70,5 +70,6 @@ path = "src/mod.rs" [features] parquet_encryption = [ "parquet/encryption", + "datafusion-common/parquet_encryption", "dep:hex", ] From 81375dcfbce6cce2e29d7b0c61319decfa59bbda Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 1 Jul 2025 19:47:56 -0700 Subject: [PATCH 49/58] Update Cargo.toml Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index aca4a2e24bd09..70c8fa4bb5ba9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -150,7 +150,7 @@ env_logger = "0.11" futures = "0.3" half = { version = "2.6.0", default-features = false } hashbrown = { version = "0.14.5", features = ["raw"] } -hex = { version = "0.4.3" } +hex = { version = "0.4.3", optional = true } indexmap = "2.10.0" itertools = "0.14" log = "^0.4" From 5f0f1b0574e78d3122aae9aa2c23527da180a767 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 1 Jul 2025 19:54:25 -0700 Subject: [PATCH 50/58] Revert bad suggestion from copilot Signed-off-by: Corwin Joy --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 70c8fa4bb5ba9..460d4898e0a25 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -150,7 +150,7 @@ env_logger = "0.11" futures = "0.3" half = { version = "2.6.0", default-features = false } hashbrown = { version = "0.14.5", features = ["raw"] } -hex = { version = "0.4.3", optional = true } +hex = { version = "0.4.3"} indexmap = "2.10.0" itertools = "0.14" log = "^0.4" From 6de2af0ca73e333fe3b7d55d98e9c68cefd0f167 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Tue, 1 Jul 2025 20:20:52 -0700 Subject: [PATCH 51/58] taplo format Cargo.toml Signed-off-by: Corwin Joy --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 460d4898e0a25..aca4a2e24bd09 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -150,7 +150,7 @@ env_logger = "0.11" futures = "0.3" half = { version = "2.6.0", default-features = false } hashbrown = { version = "0.14.5", features = ["raw"] } -hex = { version = "0.4.3"} +hex = { version = "0.4.3" } indexmap = "2.10.0" itertools = "0.14" log = "^0.4" From 464e74db68991f63130301a0042f0164dc139839 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 3 Jul 2025 15:02:29 -0700 Subject: [PATCH 52/58] 1. Add CI test for parquet_encryption. 2. Document new configuration Signed-off-by: Corwin Joy --- .github/workflows/rust.yml | 2 ++ README.md | 2 ++ datafusion/core/src/test/mod.rs | 1 + 3 files changed, 5 insertions(+) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index ecb25483ce07e..9e15a2c7670aa 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -203,6 +203,8 @@ jobs: run: cargo check --profile ci --no-default-features -p datafusion --features=string_expressions - name: Check datafusion (unicode_expressions) run: cargo check --profile ci --no-default-features -p datafusion --features=unicode_expressions + - name: Check parquet encryption (parquet_encryption) + run: cargo check --profile ci --no-default-features -p datafusion --features=parquet_encryption # Check datafusion-functions crate features # diff --git a/README.md b/README.md index c142d8f366b2e..fb7f838a572b6 100644 --- a/README.md +++ b/README.md @@ -120,6 +120,7 @@ Default features: - `datetime_expressions`: date and time functions such as `to_timestamp` - `encoding_expressions`: `encode` and `decode` functions - `parquet`: support for reading the [Apache Parquet] format +- `parquet_encryption`: support for using [Parquet Modular Encryption] - `regex_expressions`: regular expression functions, such as `regexp_match` - `unicode_expressions`: Include unicode aware functions such as `character_length` - `unparser`: enables support to reverse LogicalPlans back into SQL @@ -134,6 +135,7 @@ Optional features: [apache avro]: https://avro.apache.org/ [apache parquet]: https://parquet.apache.org/ +[parquet modular encryption]: https://parquet.apache.org/docs/file-format/data-pages/encryption/ ## DataFusion API Evolution and Deprecation Guidelines diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 8719a16f4919f..68f83e7f1f115 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -38,6 +38,7 @@ use crate::test_util::{aggr_test_schema, arrow_test_data}; use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; +#[cfg(feature = "compression")] use datafusion_common::DataFusionError; use datafusion_datasource::source::DataSourceExec; From b0cad10934d774d1c663240306e8fb99d449bbb6 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 3 Jul 2025 17:22:37 -0700 Subject: [PATCH 53/58] Refactor cfg parquet_encryption feature into a few functions defined in encryption.rs Signed-off-by: Corwin Joy --- datafusion/common/Cargo.toml | 1 + datafusion/common/src/config.rs | 10 +-- datafusion/common/src/encryption.rs | 76 +++++++++++++++++++ .../common/src/file_options/parquet_writer.rs | 21 ++--- datafusion/common/src/lib.rs | 1 + .../datasource-parquet/src/file_format.rs | 32 +++----- datafusion/datasource-parquet/src/opener.rs | 3 +- datafusion/datasource-parquet/src/source.rs | 15 ++-- 8 files changed, 103 insertions(+), 56 deletions(-) create mode 100644 datafusion/common/src/encryption.rs diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 8d331db339007..83e539e31d726 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -48,6 +48,7 @@ parquet_encryption = [ pyarrow = ["pyo3", "arrow/pyarrow", "parquet"] force_hash_collisions = [] recursive_protection = ["dep:recursive"] +parquet = ["dep:parquet"] [dependencies] ahash = { workspace = true } diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index a373a9a2a4e9b..31159d4a85881 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -19,6 +19,8 @@ use arrow_ipc::CompressionType; +#[cfg(feature = "parquet_encryption")] +use crate::encryption::{FileDecryptionProperties, FileEncryptionProperties}; use crate::error::_config_err; use crate::parsers::CompressionTypeVariant; use crate::utils::get_available_parallelism; @@ -31,14 +33,6 @@ use std::str::FromStr; #[cfg(feature = "parquet_encryption")] use hex; -#[cfg(feature = "parquet_encryption")] -pub use parquet::encryption::decrypt::FileDecryptionProperties; -#[cfg(feature = "parquet_encryption")] -pub use parquet::encryption::encrypt::FileEncryptionProperties; -#[cfg(not(feature = "parquet_encryption"))] -pub struct FileDecryptionProperties; -#[cfg(not(feature = "parquet_encryption"))] -pub struct FileEncryptionProperties; /// A macro that wraps a configuration struct and automatically derives /// [`Default`] and [`ConfigField`] for it, allowing it to be used diff --git a/datafusion/common/src/encryption.rs b/datafusion/common/src/encryption.rs new file mode 100644 index 0000000000000..b8d378f45ec3c --- /dev/null +++ b/datafusion/common/src/encryption.rs @@ -0,0 +1,76 @@ +// 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. + +// Support optional features for encryption in Parquet files. +//! This module provides types and functions related to encryption in Parquet files. + +#[cfg(feature = "parquet_encryption")] +pub use parquet::encryption::decrypt::FileDecryptionProperties; +#[cfg(feature = "parquet_encryption")] +pub use parquet::encryption::encrypt::FileEncryptionProperties; + +#[cfg(not(feature = "parquet_encryption"))] +pub struct FileDecryptionProperties; +#[cfg(not(feature = "parquet_encryption"))] +pub struct FileEncryptionProperties; + +#[cfg(feature = "parquet")] +use crate::config::ParquetEncryptionOptions; +#[cfg(feature = "parquet")] +use parquet::file::properties::WriterPropertiesBuilder; +pub use crate::config::{ConfigFileDecryptionProperties, ConfigFileEncryptionProperties}; + +#[cfg(feature = "parquet")] +pub fn add_crypto_to_writer_properties( + crypto: &ParquetEncryptionOptions, + mut builder: WriterPropertiesBuilder, +) -> WriterPropertiesBuilder { + #[cfg(feature = "parquet_encryption")] + if let Some(file_encryption_properties) = &crypto.file_encryption { + builder = builder + .with_file_encryption_properties(file_encryption_properties.clone().into()); + } + builder +} + +#[cfg(feature = "parquet_encryption")] +pub fn map_encryption_to_config_encryption( + encryption: Option<&FileEncryptionProperties>, +) -> Option { + encryption.map(|fe| fe.into()) +} + +#[cfg(not(feature = "parquet_encryption"))] +pub fn map_encryption_to_config_encryption( + _encryption: Option<&FileEncryptionProperties>, +) -> Option { + None +} + +#[cfg(feature = "parquet_encryption")] +pub fn map_config_decryption_to_decryption( + decryption: Option<&ConfigFileDecryptionProperties>, +) -> Option { + decryption.map(|fd| fd.clone().into()) +} + +#[cfg(not(feature = "parquet_encryption"))] +pub fn map_config_decryption_to_decryption( + _decryption: Option<&ConfigFileDecryptionProperties>, +) -> Option { + None +} diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index de496b4f0019c..d493350e8f053 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -27,6 +27,7 @@ use crate::{ use arrow::datatypes::Schema; // TODO: handle once deprecated +use crate::encryption::add_crypto_to_writer_properties; #[allow(deprecated)] use parquet::{ arrow::ARROW_SCHEMA_META_KEY, @@ -101,12 +102,7 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { let mut builder = global.into_writer_properties_builder()?; - #[cfg(feature = "parquet_encryption")] - if let Some(file_encryption_properties) = &crypto.file_encryption { - builder = builder.with_file_encryption_properties( - file_encryption_properties.clone().into(), - ); - } + builder = add_crypto_to_writer_properties(crypto, builder); // check that the arrow schema is present in the kv_metadata, if configured to do so if !global.skip_arrow_metadata @@ -458,12 +454,9 @@ mod tests { }; use std::collections::HashMap; - use crate::config::{ - ConfigFileEncryptionProperties, ParquetColumnOptions, ParquetEncryptionOptions, - ParquetOptions, - }; - use super::*; + use crate::config::{ParquetColumnOptions, ParquetEncryptionOptions, ParquetOptions}; + use crate::encryption::map_encryption_to_config_encryption; const COL_NAME: &str = "configured"; @@ -592,11 +585,7 @@ mod tests { HashMap::from([(COL_NAME.into(), configured_col_props)]) }; - #[cfg(feature = "parquet_encryption")] - let fep: Option = - props.file_encryption_properties().map(|fe| fe.into()); - #[cfg(not(feature = "parquet_encryption"))] - let fep: Option = None; + let fep = map_encryption_to_config_encryption(props.file_encryption_properties()); #[allow(deprecated)] // max_statistics_size TableParquetOptions { diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 3ea7321ef3b4b..102307e7eb5af 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -41,6 +41,7 @@ pub mod config; pub mod cse; pub mod diagnostic; pub mod display; +pub mod encryption; pub mod error; pub mod file_options; pub mod format; diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index dd6d91d1f8494..b2b6155bb47c5 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -38,8 +38,9 @@ use datafusion_datasource::write::demux::DemuxedStreamReceiver; use arrow::compute::sum; use arrow::datatypes::{DataType, Field, FieldRef}; -use datafusion_common::config::{ - ConfigField, ConfigFileType, FileDecryptionProperties, TableParquetOptions, +use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; +use datafusion_common::encryption::{ + map_config_decryption_to_decryption, FileDecryptionProperties, }; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; @@ -352,17 +353,11 @@ impl FileFormat for ParquetFormat { Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?), None => None, }; - #[cfg(feature = "parquet_encryption")] let file_decryption_properties: Option = - match &self.options.crypto.file_decryption { - Some(cfd) => { - let fd: FileDecryptionProperties = cfd.clone().into(); - Some(fd) - } - None => None, - }; - #[cfg(not(feature = "parquet_encryption"))] - let file_decryption_properties: Option = None; + map_config_decryption_to_decryption( + self.options.crypto.file_decryption.as_ref(), + ); + let mut schemas: Vec<_> = futures::stream::iter(objects) .map(|object| { fetch_schema_with_location( @@ -419,17 +414,10 @@ impl FileFormat for ParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { - #[cfg(feature = "parquet_encryption")] let file_decryption_properties: Option = - match &self.options.crypto.file_decryption { - Some(cfd) => { - let fd: FileDecryptionProperties = cfd.clone().into(); - Some(fd) - } - None => None, - }; - #[cfg(not(feature = "parquet_encryption"))] - let file_decryption_properties: Option = None; + map_config_decryption_to_decryption( + self.options.crypto.file_decryption.as_ref(), + ); let stats = fetch_statistics( store.as_ref(), table_schema, diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 1fcab6f3c230b..d6a17eda818b5 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -31,7 +31,8 @@ use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::{FieldRef, SchemaRef, TimeUnit}; use arrow::error::ArrowError; -use datafusion_common::config::FileDecryptionProperties; +use datafusion_common::encryption::FileDecryptionProperties; + use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_datasource::PartitionedFile; use datafusion_physical_expr::PhysicalExprSchemaRewriter; diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 5fc91fa13ce74..1259241e8cf41 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -47,8 +47,10 @@ use datafusion_physical_plan::metrics::Count; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; +use datafusion_common::encryption::map_config_decryption_to_decryption; use itertools::Itertools; use object_store::ObjectStore; + /// Execution plan for reading one or more Parquet files. /// /// ```text @@ -475,15 +477,10 @@ impl FileSource for ParquetSource { Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _ }); - #[cfg(feature = "parquet_encryption")] - let file_decryption_properties = self - .table_parquet_options() - .crypto - .file_decryption - .as_ref() - .map(|props| Arc::new(props.clone().into())); - #[cfg(not(feature = "parquet_encryption"))] - let file_decryption_properties = None; + let file_decryption_properties = map_config_decryption_to_decryption( + self.table_parquet_options().crypto.file_decryption.as_ref(), + ) + .map(Arc::new); let coerce_int96 = self .table_parquet_options From 321596f983ed07d3cad8512fdc6769413a1b526d Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 3 Jul 2025 17:33:26 -0700 Subject: [PATCH 54/58] Remove #allow(unusued) that is no longer needed Signed-off-by: Corwin Joy --- datafusion/common/src/file_options/parquet_writer.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index d493350e8f053..db667f85c61f9 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -96,7 +96,6 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { global, column_specific_options, key_value_metadata, - #[allow(unused)] crypto, } = table_parquet_options; From 411d0a58c28b0bf285cae8961c5b1a95cf5bc744 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 3 Jul 2025 17:40:14 -0700 Subject: [PATCH 55/58] cargo fmt Signed-off-by: Corwin Joy --- datafusion/common/src/encryption.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/encryption.rs b/datafusion/common/src/encryption.rs index b8d378f45ec3c..ad9071a900a9c 100644 --- a/datafusion/common/src/encryption.rs +++ b/datafusion/common/src/encryption.rs @@ -30,9 +30,9 @@ pub struct FileEncryptionProperties; #[cfg(feature = "parquet")] use crate::config::ParquetEncryptionOptions; +pub use crate::config::{ConfigFileDecryptionProperties, ConfigFileEncryptionProperties}; #[cfg(feature = "parquet")] use parquet::file::properties::WriterPropertiesBuilder; -pub use crate::config::{ConfigFileDecryptionProperties, ConfigFileEncryptionProperties}; #[cfg(feature = "parquet")] pub fn add_crypto_to_writer_properties( From 34141416bb632ba6862183fc3af2b23cdd5fbdb7 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 3 Jul 2025 18:01:13 -0700 Subject: [PATCH 56/58] Fix error when cfg parquet_encryption is disabled. Signed-off-by: Corwin Joy --- datafusion/common/src/encryption.rs | 2 ++ datafusion/common/src/file_options/parquet_writer.rs | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/datafusion/common/src/encryption.rs b/datafusion/common/src/encryption.rs index ad9071a900a9c..5918e4e1b3b56 100644 --- a/datafusion/common/src/encryption.rs +++ b/datafusion/common/src/encryption.rs @@ -36,7 +36,9 @@ use parquet::file::properties::WriterPropertiesBuilder; #[cfg(feature = "parquet")] pub fn add_crypto_to_writer_properties( + #[allow(unused)] crypto: &ParquetEncryptionOptions, + #[allow(unused_mut)] mut builder: WriterPropertiesBuilder, ) -> WriterPropertiesBuilder { #[cfg(feature = "parquet_encryption")] diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index db667f85c61f9..cde0ea1299795 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -455,6 +455,7 @@ mod tests { use super::*; use crate::config::{ParquetColumnOptions, ParquetEncryptionOptions, ParquetOptions}; + #[cfg(feature = "parquet_encryption")] use crate::encryption::map_encryption_to_config_encryption; const COL_NAME: &str = "configured"; @@ -584,7 +585,10 @@ mod tests { HashMap::from([(COL_NAME.into(), configured_col_props)]) }; + #[cfg(feature = "parquet_encryption")] let fep = map_encryption_to_config_encryption(props.file_encryption_properties()); + #[cfg(not(feature = "parquet_encryption"))] + let fep = None; #[allow(deprecated)] // max_statistics_size TableParquetOptions { From 842aa894adb49907ff31de47b97032781a3c8e0a Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Thu, 3 Jul 2025 18:06:59 -0700 Subject: [PATCH 57/58] cargo fmt Signed-off-by: Corwin Joy --- datafusion/common/src/encryption.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/datafusion/common/src/encryption.rs b/datafusion/common/src/encryption.rs index 5918e4e1b3b56..5d50d4a9efd37 100644 --- a/datafusion/common/src/encryption.rs +++ b/datafusion/common/src/encryption.rs @@ -36,10 +36,8 @@ use parquet::file::properties::WriterPropertiesBuilder; #[cfg(feature = "parquet")] pub fn add_crypto_to_writer_properties( - #[allow(unused)] - crypto: &ParquetEncryptionOptions, - #[allow(unused_mut)] - mut builder: WriterPropertiesBuilder, + #[allow(unused)] crypto: &ParquetEncryptionOptions, + #[allow(unused_mut)] mut builder: WriterPropertiesBuilder, ) -> WriterPropertiesBuilder { #[cfg(feature = "parquet_encryption")] if let Some(file_encryption_properties) = &crypto.file_encryption { From 60d37acf397148c10da59275de6576cac0640d45 Mon Sep 17 00:00:00 2001 From: Corwin Joy Date: Wed, 9 Jul 2025 20:10:40 -0700 Subject: [PATCH 58/58] Remove unused mut. Signed-off-by: Corwin Joy --- datafusion/datasource-parquet/src/file_format.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index b2b6155bb47c5..1d38d07130bd6 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -958,8 +958,7 @@ pub async fn fetch_parquet_metadata( let file_size = meta.size; let fetch = ObjectStoreFetch::new(store, meta); - #[allow(unused_mut)] - let mut reader = ParquetMetaDataReader::new().with_prefetch_hint(size_hint); + let reader = ParquetMetaDataReader::new().with_prefetch_hint(size_hint); #[cfg(feature = "parquet_encryption")] let reader = reader.with_decryption_properties(decryption_properties);