alamb commented on code in PR #16779: URL: https://github.com/apache/datafusion/pull/16779#discussion_r2248598467
########## datafusion-examples/examples/parquet_encrypted_with_kms.rs: ########## @@ -0,0 +1,288 @@ +// 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 arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use arrow_schema::SchemaRef; +use datafusion::common::{extensions_options, DataFusionError}; +use datafusion::config::{EncryptionFactoryOptions, TableParquetOptions}; +use datafusion::dataframe::DataFrameWriteOptions; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::listing::ListingOptions; +use datafusion::error::Result; +use datafusion::execution::parquet_encryption::EncryptionFactory; +use datafusion::parquet::encryption::{ + decrypt::FileDecryptionProperties, encrypt::FileEncryptionProperties, +}; +use datafusion::prelude::SessionContext; +use futures::StreamExt; +use object_store::path::Path; +use parquet_key_management::crypto_factory::{ + CryptoFactory, DecryptionConfiguration, EncryptionConfiguration, +}; +use parquet_key_management::kms::KmsConnectionConfig; +use parquet_key_management::test_kms::TestKmsClientFactory; +use std::collections::HashSet; +use std::fmt::Formatter; +use std::sync::Arc; +use tempfile::TempDir; + +const ENCRYPTION_FACTORY_ID: &str = "example.memory_kms_encryption"; + +/// This example demonstrates reading and writing Parquet files that +/// are encrypted using Parquet Modular Encryption, and uses the Review Comment: instead of actually integrating with a KMS in the example, could we potentially stub it out and avoid the dependency on the new crate? I realize this is like a "end to end" integration test, but I think that might make more sense to keep somewhere outside the core datafusion crate 🤔 ########## datafusion-examples/Cargo.toml: ########## @@ -73,6 +73,7 @@ futures = { workspace = true } log = { workspace = true } mimalloc = { version = "0.1", default-features = false } object_store = { workspace = true, features = ["aws", "http"] } +parquet-key-management = { version = "0.3", features = ["_test_utils"] } Review Comment: I am a little worried at taking a new dependency, even though I realize it is only for the examples -- the new dependency makes it that more effort to keep dependencies up to date, for example It looks like this package is fairly new and has only 2 contributors https://github.com/G-Research/parquet-key-management-rs (I realize @adamreeve you are one of them ) ########## datafusion/common/src/file_options/parquet_writer.rs: ########## @@ -96,13 +95,11 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { global, column_specific_options, key_value_metadata, - crypto, + crypto: _, Review Comment: does this mean the crypto options on the table parquet options are ignored now (perhaps replaced by the crypto factory). If so I suggest we remove the field or mark it deprecated ########## datafusion/core/src/execution/context/parquet.rs: ########## @@ -94,6 +96,18 @@ impl SessionContext { ) -> Result<()> { plan_to_parquet(self.task_ctx(), plan, path, writer_properties).await } + + /// Registers a Parquet [`EncryptionFactory`] with an associated unique identifier. + /// If an encryption factory with the same identifier was already registered, it is replaced and returned. + #[cfg(feature = "parquet_encryption")] Review Comment: I suggest we keep the API surface area lower by only adding the API to the runtime_env rather than also adding this wrapper to SessionContext So users would have to do ```rust ctx.runtime_env() .register_parquet_encryption_factory(id, encryption_factory) ``` Rather than what you have here ```rust ctx.register_parquet_encryption_factory(id, encryption_factory) ``` I don't feel strongly about this ########## datafusion/common/src/encryption.rs: ########## @@ -28,24 +28,7 @@ pub struct FileDecryptionProperties; #[cfg(not(feature = "parquet_encryption"))] pub struct FileEncryptionProperties; -#[cfg(feature = "parquet")] Review Comment: this is certainly nicer -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
