From 1eae1f472c2cc69aa81270d42573ac9447730d76 Mon Sep 17 00:00:00 2001 From: Adam Reeve Date: Tue, 14 Apr 2026 14:33:10 +1200 Subject: [PATCH 1/3] Make conversion from FileDecryptionProperties to ConfigFileDecryptionProperties fallible --- .../examples/data_io/parquet_encrypted.rs | 2 +- datafusion/common/src/config.rs | 25 +++++++++++++------ datafusion/core/src/dataframe/parquet.rs | 4 +-- datafusion/core/tests/parquet/encryption.rs | 4 +-- 4 files changed, 22 insertions(+), 13 deletions(-) diff --git a/datafusion-examples/examples/data_io/parquet_encrypted.rs b/datafusion-examples/examples/data_io/parquet_encrypted.rs index 26361e9b52be0..f73c538d1c4d9 100644 --- a/datafusion-examples/examples/data_io/parquet_encrypted.rs +++ b/datafusion-examples/examples/data_io/parquet_encrypted.rs @@ -71,7 +71,7 @@ pub async fn parquet_encrypted() -> datafusion::common::Result<()> { // Read encrypted parquet back as a DataFrame using matching decryption config let ctx: SessionContext = SessionContext::new(); let read_options = - ParquetReadOptions::default().file_decryption_properties((&decrypt).into()); + ParquetReadOptions::default().file_decryption_properties((&decrypt).try_into()?); let encrypted_parquet_df = ctx .read_parquet(tempfile.to_str().unwrap(), read_options) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 0eec3f948034a..aee1853b57c83 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -3024,8 +3024,18 @@ impl From for FileDecryptionProperties { } #[cfg(feature = "parquet_encryption")] -impl From<&Arc> for ConfigFileDecryptionProperties { - fn from(f: &Arc) -> Self { +impl TryFrom<&Arc> for ConfigFileDecryptionProperties { + type Error = DataFusionError; + + fn try_from(f: &Arc) -> Result { + let footer_key = f.footer_key(None).map_err(|e| { + DataFusionError::Configuration(format!( + "Could not retrieve footer key from FileDecryptionProperties. \ + Note that conversion to ConfigFileDecryptionProperties is not supported \ + when using a key retriever: {e}" + )) + })?; + let (column_names_vec, column_keys_vec) = f.column_keys(); let mut column_decryption_properties: HashMap< String, @@ -3039,14 +3049,12 @@ impl From<&Arc> for ConfigFileDecryptionProperties { } let aad_prefix = f.aad_prefix().cloned().unwrap_or_default(); - ConfigFileDecryptionProperties { - footer_key_as_hex: hex::encode( - f.footer_key(None).unwrap_or_default().as_ref(), - ), + Ok(ConfigFileDecryptionProperties { + footer_key_as_hex: hex::encode(footer_key.as_ref()), column_decryption_properties, aad_prefix_as_hex: hex::encode(aad_prefix), footer_signature_verification: f.check_plaintext_footer_integrity(), - } + }) } } @@ -3519,7 +3527,8 @@ mod tests { Arc::new(FileEncryptionProperties::from(config_encrypt.clone())); assert_eq!(file_encryption_properties, encryption_properties_built); - let config_decrypt = ConfigFileDecryptionProperties::from(&decryption_properties); + let config_decrypt = + ConfigFileDecryptionProperties::try_from(&decryption_properties).unwrap(); let decryption_properties_built = Arc::new(FileDecryptionProperties::from(config_decrypt.clone())); assert_eq!(decryption_properties, decryption_properties_built); diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index ccd5766f0a24d..7a2b05262caea 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -312,8 +312,8 @@ mod tests { // Read encrypted parquet let ctx: SessionContext = SessionContext::new(); - let read_options = - ParquetReadOptions::default().file_decryption_properties((&decrypt).into()); + let read_options = ParquetReadOptions::default() + .file_decryption_properties((&decrypt).try_into()?); ctx.register_parquet("roundtrip_parquet", &tempfile_str, read_options.clone()) .await?; diff --git a/datafusion/core/tests/parquet/encryption.rs b/datafusion/core/tests/parquet/encryption.rs index 8b3170e367457..12bdb600c2ac9 100644 --- a/datafusion/core/tests/parquet/encryption.rs +++ b/datafusion/core/tests/parquet/encryption.rs @@ -115,8 +115,8 @@ async fn round_trip_encryption() { // Read encrypted parquet let ctx: SessionContext = SessionContext::new(); - let options = - ParquetReadOptions::default().file_decryption_properties((&decrypt).into()); + let options = ParquetReadOptions::default() + .file_decryption_properties((&decrypt).try_into().unwrap()); let encrypted_batches = read_parquet_test_data( tempfile.into_os_string().into_string().unwrap(), From 6d380a1052b06b8becfc514d3426ec434377d618 Mon Sep 17 00:00:00 2001 From: Adam Reeve Date: Tue, 14 Apr 2026 16:09:21 +1200 Subject: [PATCH 2/3] Add unit test --- datafusion/common/src/config.rs | 36 +++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index aee1853b57c83..9ca159095dd1b 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -3646,6 +3646,42 @@ mod tests { assert_eq!(factory_options.get("key2"), Some(&"value 2".to_string())); } + #[cfg(feature = "parquet_encryption")] + struct ParquetEncryptionKeyRetriever {} + + #[cfg(feature = "parquet_encryption")] + impl parquet::encryption::decrypt::KeyRetriever for ParquetEncryptionKeyRetriever { + fn retrieve_key(&self, key_metadata: &[u8]) -> parquet::errors::Result> { + if !key_metadata.is_empty() { + Ok(b"1234567890123450".to_vec()) + } else { + Err(parquet::errors::ParquetError::General( + "Key metadata not provided".to_string(), + )) + } + } + } + + #[cfg(feature = "parquet_encryption")] + #[test] + fn conversion_from_key_retriever_to_config_file_decryption_properties() { + use crate::Result; + use crate::config::ConfigFileDecryptionProperties; + use crate::encryption::FileDecryptionProperties; + + let retriever = std::sync::Arc::new(ParquetEncryptionKeyRetriever {}); + let decryption_properties = + FileDecryptionProperties::with_key_retriever(retriever) + .build() + .unwrap(); + let config_file_decryption_properties: Result = + (&decryption_properties).try_into(); + assert!(config_file_decryption_properties.is_err()); + let err = config_file_decryption_properties.unwrap_err().to_string(); + assert!(err.contains("key retriever")); + assert!(err.contains("Key metadata not provided")); + } + #[cfg(feature = "parquet")] #[test] fn parquet_table_options_config_entry() { From 704d0f120301bf54952867296bfe5e0496cf0414 Mon Sep 17 00:00:00 2001 From: Adam Reeve Date: Wed, 15 Apr 2026 12:08:10 +1200 Subject: [PATCH 3/3] Add section to upgrade guide --- .../library-user-guide/upgrading/54.0.0.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/docs/source/library-user-guide/upgrading/54.0.0.md b/docs/source/library-user-guide/upgrading/54.0.0.md index c34c08ec194ac..4bb9f94daee2d 100644 --- a/docs/source/library-user-guide/upgrading/54.0.0.md +++ b/docs/source/library-user-guide/upgrading/54.0.0.md @@ -438,3 +438,22 @@ clusters (e.g., ZWJ emoji sequences). For ASCII and most common Unicode text, behavior is unchanged. [#17861]: https://github.com/apache/datafusion/pull/17861 + +### Conversion from `FileDecryptionProperties` to `ConfigFileDecryptionProperties` is now fallible + +Previously, `datafusion_common::config::ConfigFileDecryptionProperties` +implemented `From<&Arc>`. +If an error was encountered when retrieving the footer key without providing key metadata, +the error would be ignored and an empty footer key set in the result. +This could lead to obscure errors later. + +`ConfigFileDecryptionProperties` now instead implements `TryFrom<&Arc>`, +and errors retrieving the footer key will be propagated up. + +Code that uses `ConfigFileDecryptionProperties::from(&Arc)` +should be updated to use `try_from`, +and calls to `FileDecryptionProperties::into` should be changed to `try_into`, +with appropriate error handling added. + +See [#21602](https://github.com/apache/datafusion/issues/21602) and +[PR #21603](https://github.com/apache/datafusion/pull/21603) for details.