blackmwk commented on code in PR #2383:
URL: https://github.com/apache/iceberg-rust/pull/2383#discussion_r3208382491


##########
crates/iceberg/src/encryption/manager.rs:
##########
@@ -0,0 +1,713 @@
+// 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.
+
+//! Encryption manager for file-level encryption and two-layer envelope key 
management.
+//!
+//! [`EncryptionManager`] provides file-level `decrypt` / `encrypt`
+//! operations matching Java's 
`org.apache.iceberg.encryption.EncryptionManager`,
+//! using envelope encryption:
+//! - A master key (in KMS) wraps a Key Encryption Key (KEK)
+//! - The KEK wraps Data Encryption Keys (DEKs) locally
+
+use std::collections::HashMap;
+use std::fmt;
+use std::sync::Arc;
+use std::time::Duration;
+
+use aes_gcm::aead::OsRng;
+use aes_gcm::aead::rand_core::RngCore;
+use chrono::Utc;
+use moka::future::Cache;
+use uuid::Uuid;
+
+const MILLIS_IN_DAY: i64 = 24 * 60 * 60 * 1000;
+
+use super::crypto::{AesGcmCipher, AesKeySize, SecureKey, SensitiveBytes};
+use super::file_decryptor::AesGcmFileDecryptor;
+use super::file_encryptor::AesGcmFileEncryptor;
+use super::io::{EncryptedInputFile, EncryptedOutputFile};
+use super::key_metadata::StandardKeyMetadata;
+use super::kms::KeyManagementClient;
+use crate::io::{InputFile, OutputFile};
+use crate::spec::EncryptedKey;
+use crate::{Error, ErrorKind, Result};
+
+/// Property key for the KEK creation timestamp (milliseconds since epoch).
+/// Matches Java's `StandardEncryptionManager.KEY_TIMESTAMP`.
+pub const KEK_CREATED_AT_PROPERTY: &str = "KEY_TIMESTAMP";
+
+/// Default KEK lifespan in days, per NIST SP 800-57.
+const DEFAULT_KEK_LIFESPAN_DAYS: i64 = 730;
+
+/// Default cache TTL for unwrapped KEKs.
+const DEFAULT_CACHE_TTL: Duration = Duration::from_secs(3600);
+
+/// Default AAD prefix length in bytes.
+/// Matches Java's `TableProperties.ENCRYPTION_AAD_LENGTH_DEFAULT`.
+const AAD_PREFIX_LENGTH: usize = 16;
+
+/// File-level encryption manager using two-layer envelope encryption.
+///
+/// Uses an async cache for unwrapped KEK bytes to avoid repeated KMS calls.
+#[derive(typed_builder::TypedBuilder)]
+#[builder(mutators(
+    /// Add an encryption key (KEK or wrapped key metadata entry).
+    pub fn add_encryption_key(&mut self, key: EncryptedKey) {
+        self.encryption_keys.insert(key.key_id().to_string(), key);
+    }
+    /// Set all encryption keys from table metadata.
+    pub fn encryption_keys(&mut self, keys: HashMap<String, EncryptedKey>) {
+        self.encryption_keys = keys;
+    }
+))]
+pub struct EncryptionManager {
+    kms_client: Arc<dyn KeyManagementClient>,
+    #[builder(
+        default = Cache::builder().time_to_live(DEFAULT_CACHE_TTL).build(),
+        setter(skip)
+    )]
+    kek_cache: Cache<String, SensitiveBytes>,
+    /// AES key size for DEK generation. Defaults to 128-bit.
+    #[builder(default = AesKeySize::default())]
+    key_size: AesKeySize,
+    /// Master key ID from table property `encryption.key-id`.
+    #[builder(setter(into))]
+    table_key_id: String,
+    /// All encryption keys from table metadata (KEKs and wrapped key metadata 
entries).
+    #[builder(default, via_mutators)]
+    encryption_keys: HashMap<String, EncryptedKey>,
+}
+
+impl fmt::Debug for EncryptionManager {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.debug_struct("EncryptionManager")
+            .field("key_size", &self.key_size)
+            .field("table_key_id", &self.table_key_id)
+            .finish_non_exhaustive()
+    }
+}
+
+impl EncryptionManager {
+    /// Encrypt a file with AGS1 stream encryption.
+    ///
+    /// Returns an [`EncryptedOutputFile`] that transparently encrypts on
+    /// write, along with key metadata for later decryption.
+    pub fn encrypt(&self, raw_output: OutputFile) -> 
Result<EncryptedOutputFile> {
+        let dek = SecureKey::generate(self.key_size);
+        let aad_prefix = Self::generate_aad_prefix();
+
+        let key_metadata_bytes = StandardKeyMetadata::new(dek.as_bytes())
+            .with_aad_prefix(&aad_prefix)
+            .encode()?;
+
+        let encryptor = Arc::new(AesGcmFileEncryptor::new(dek.as_bytes(), 
aad_prefix)?);
+
+        Ok(EncryptedOutputFile::new(
+            raw_output,
+            key_metadata_bytes,
+            encryptor,
+        ))
+    }
+
+    /// Decrypt an encrypted input file, returning an [`EncryptedInputFile`]
+    /// that transparently decrypts on read.
+    pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> 
Result<EncryptedInputFile> {
+        let metadata = StandardKeyMetadata::decode(key_metadata)?;
+
+        let decryptor = Arc::new(AesGcmFileDecryptor::new(
+            metadata.encryption_key().as_bytes(),
+            metadata.aad_prefix().unwrap_or_default(),
+        )?);
+
+        Ok(EncryptedInputFile::new(input, decryptor))
+    }
+
+    /// Generate key material for Parquet Modular Encryption (PME).
+    ///
+    /// Returns a [`StandardKeyMetadata`] containing a fresh DEK and AAD 
prefix.
+    /// The caller should pass this to the Parquet writer to configure
+    /// `FileEncryptionProperties`, and serialize it for storage in the 
manifest.
+    #[allow(dead_code)]

Review Comment:
   Why we need to add this annotation? If it's not used for now, let's remove 
it.



##########
crates/iceberg/src/encryption/manager.rs:
##########
@@ -0,0 +1,706 @@
+// 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.
+
+//! Encryption manager for file-level encryption and two-layer envelope key 
management.
+//!
+//! [`EncryptionManager`] provides file-level `decrypt` / `encrypt`
+//! operations matching Java's 
`org.apache.iceberg.encryption.EncryptionManager`,
+//! using envelope encryption:
+//! - A master key (in KMS) wraps a Key Encryption Key (KEK)
+//! - The KEK wraps Data Encryption Keys (DEKs) locally
+
+use std::collections::HashMap;
+use std::fmt;
+use std::sync::Arc;
+use std::time::Duration;
+
+use aes_gcm::aead::OsRng;
+use aes_gcm::aead::rand_core::RngCore;
+use chrono::Utc;
+use uuid::Uuid;
+
+const MILLIS_IN_DAY: i64 = 24 * 60 * 60 * 1000;
+
+use super::crypto::{AesGcmCipher, AesKeySize, SecureKey, SensitiveBytes};
+use super::encrypted_io::{EncryptedInputFile, EncryptedOutputFile};
+use super::file_decryptor::AesGcmFileDecryptor;
+use super::file_encryptor::AesGcmFileEncryptor;
+use super::key_metadata::StandardKeyMetadata;
+use super::kms::KeyManagementClient;
+use crate::io::{InputFile, OutputFile};
+use crate::spec::EncryptedKey;
+use crate::{Error, ErrorKind, Result};
+
+/// Property key for the KEK creation timestamp (milliseconds since epoch).
+/// Matches Java's `StandardEncryptionManager.KEY_TIMESTAMP`.
+pub const KEK_CREATED_AT_PROPERTY: &str = "KEY_TIMESTAMP";
+
+/// Default KEK lifespan in days, per NIST SP 800-57.
+const DEFAULT_KEK_LIFESPAN_DAYS: i64 = 730;
+
+/// Default cache TTL for unwrapped KEKs.
+const DEFAULT_CACHE_TTL: Duration = Duration::from_secs(3600);
+
+/// Default AAD prefix length in bytes.
+/// Matches Java's `TableProperties.ENCRYPTION_AAD_LENGTH_DEFAULT`.
+const AAD_PREFIX_LENGTH: usize = 16;
+
+/// File-level encryption manager using two-layer envelope encryption.
+///
+/// Uses an async cache for unwrapped KEK bytes to avoid repeated KMS calls.
+#[derive(typed_builder::TypedBuilder)]
+pub struct EncryptionManager {
+    kms_client: Arc<dyn KeyManagementClient>,
+    #[builder(
+        default = 
moka::future::Cache::builder().time_to_live(DEFAULT_CACHE_TTL).build(),
+        setter(skip)
+    )]
+    kek_cache: moka::future::Cache<String, SensitiveBytes>,
+    /// AES key size for DEK generation. Defaults to 128-bit.
+    #[builder(default = AesKeySize::default())]
+    key_size: AesKeySize,
+    /// Master key ID from table property `encryption.key-id`.
+    #[builder(setter(into))]
+    table_key_id: String,
+    /// All encryption keys from table metadata (KEKs and wrapped key metadata 
entries).
+    #[builder(default)]
+    encryption_keys: HashMap<String, EncryptedKey>,
+}
+
+impl fmt::Debug for EncryptionManager {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.debug_struct("EncryptionManager")
+            .field("key_size", &self.key_size)
+            .field("table_key_id", &self.table_key_id)
+            .finish_non_exhaustive()
+    }
+}
+
+impl EncryptionManager {
+    /// Encrypt a file with AGS1 stream encryption.
+    ///
+    /// Returns an [`EncryptedOutputFile`] that transparently encrypts on
+    /// write, along with key metadata for later decryption.
+    pub fn encrypt(&self, raw_output: OutputFile) -> 
Result<EncryptedOutputFile> {
+        let dek = SecureKey::generate(self.key_size);
+        let aad_prefix = Self::generate_aad_prefix();
+
+        let key_metadata_bytes = StandardKeyMetadata::new(dek.as_bytes())
+            .with_aad_prefix(&aad_prefix)
+            .encode()?;
+
+        let encryptor = Arc::new(AesGcmFileEncryptor::new(dek.as_bytes(), 
aad_prefix)?);
+
+        Ok(EncryptedOutputFile::new(
+            raw_output,
+            key_metadata_bytes,
+            encryptor,
+        ))
+    }
+
+    /// Decrypt an encrypted input file, returning an [`EncryptedInputFile`]
+    /// that transparently decrypts on read.
+    pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> 
Result<EncryptedInputFile> {

Review Comment:
   Yes, we got in trouble here. After second thought, I think we should remove 
this method. [Java's 
implementation](https://github.com/apache/iceberg/blob/bfec39f64666b8d49dc5061a6c5cfa96062f613a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java#L104)
 in fact has nothing to do with `EncryptionManager`.  WDYT? 



##########
crates/iceberg/src/encryption/io.rs:
##########
@@ -0,0 +1,154 @@
+// 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.
+
+//! Encrypted file wrappers for InputFile / OutputFile.
+
+use std::sync::Arc;
+
+use bytes::Bytes;
+
+use super::file_decryptor::AesGcmFileDecryptor;
+use super::file_encryptor::AesGcmFileEncryptor;
+use crate::io::{FileMetadata, FileRead, FileWrite, InputFile, OutputFile};
+
+/// An AGS1 stream-encrypted input file wrapping a plain [`InputFile`].
+///
+/// Transparently decrypts on read.
+pub struct EncryptedInputFile {
+    inner: InputFile,
+    decryptor: Arc<AesGcmFileDecryptor>,

Review Comment:
   `AesGcmFileDecryptor` actually didn't store any state? What actually matters 
is `AesGcmFileRead`, right? I'm not a big fan of adding too much abstraction. 
Why not just make things as following:
   ```
   pub struct EncryptedInputFile {
      key_metadata: StandardKeyMetadata,
      inner: InputFile
   }
   
   // Similar to EncryptedOutputFIle 
   ```



##########
crates/iceberg/src/encryption/manager.rs:
##########
@@ -0,0 +1,706 @@
+// 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.
+
+//! Encryption manager for file-level encryption and two-layer envelope key 
management.
+//!
+//! [`EncryptionManager`] provides file-level `decrypt` / `encrypt`
+//! operations matching Java's 
`org.apache.iceberg.encryption.EncryptionManager`,
+//! using envelope encryption:
+//! - A master key (in KMS) wraps a Key Encryption Key (KEK)
+//! - The KEK wraps Data Encryption Keys (DEKs) locally
+
+use std::collections::HashMap;
+use std::fmt;
+use std::sync::Arc;
+use std::time::Duration;
+
+use aes_gcm::aead::OsRng;
+use aes_gcm::aead::rand_core::RngCore;
+use chrono::Utc;
+use uuid::Uuid;
+
+const MILLIS_IN_DAY: i64 = 24 * 60 * 60 * 1000;
+
+use super::crypto::{AesGcmCipher, AesKeySize, SecureKey, SensitiveBytes};
+use super::encrypted_io::{EncryptedInputFile, EncryptedOutputFile};
+use super::file_decryptor::AesGcmFileDecryptor;
+use super::file_encryptor::AesGcmFileEncryptor;
+use super::key_metadata::StandardKeyMetadata;
+use super::kms::KeyManagementClient;
+use crate::io::{InputFile, OutputFile};
+use crate::spec::EncryptedKey;
+use crate::{Error, ErrorKind, Result};
+
+/// Property key for the KEK creation timestamp (milliseconds since epoch).
+/// Matches Java's `StandardEncryptionManager.KEY_TIMESTAMP`.
+pub const KEK_CREATED_AT_PROPERTY: &str = "KEY_TIMESTAMP";
+
+/// Default KEK lifespan in days, per NIST SP 800-57.
+const DEFAULT_KEK_LIFESPAN_DAYS: i64 = 730;
+
+/// Default cache TTL for unwrapped KEKs.
+const DEFAULT_CACHE_TTL: Duration = Duration::from_secs(3600);
+
+/// Default AAD prefix length in bytes.
+/// Matches Java's `TableProperties.ENCRYPTION_AAD_LENGTH_DEFAULT`.
+const AAD_PREFIX_LENGTH: usize = 16;
+
+/// File-level encryption manager using two-layer envelope encryption.
+///
+/// Uses an async cache for unwrapped KEK bytes to avoid repeated KMS calls.
+#[derive(typed_builder::TypedBuilder)]
+pub struct EncryptionManager {
+    kms_client: Arc<dyn KeyManagementClient>,
+    #[builder(
+        default = 
moka::future::Cache::builder().time_to_live(DEFAULT_CACHE_TTL).build(),
+        setter(skip)
+    )]
+    kek_cache: moka::future::Cache<String, SensitiveBytes>,
+    /// AES key size for DEK generation. Defaults to 128-bit.
+    #[builder(default = AesKeySize::default())]
+    key_size: AesKeySize,
+    /// Master key ID from table property `encryption.key-id`.
+    #[builder(setter(into))]
+    table_key_id: String,
+    /// All encryption keys from table metadata (KEKs and wrapped key metadata 
entries).
+    #[builder(default)]
+    encryption_keys: HashMap<String, EncryptedKey>,
+}
+
+impl fmt::Debug for EncryptionManager {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.debug_struct("EncryptionManager")
+            .field("key_size", &self.key_size)
+            .field("table_key_id", &self.table_key_id)
+            .finish_non_exhaustive()
+    }
+}
+
+impl EncryptionManager {
+    /// Encrypt a file with AGS1 stream encryption.
+    ///
+    /// Returns an [`EncryptedOutputFile`] that transparently encrypts on
+    /// write, along with key metadata for later decryption.
+    pub fn encrypt(&self, raw_output: OutputFile) -> 
Result<EncryptedOutputFile> {
+        let dek = SecureKey::generate(self.key_size);
+        let aad_prefix = Self::generate_aad_prefix();
+
+        let key_metadata_bytes = StandardKeyMetadata::new(dek.as_bytes())
+            .with_aad_prefix(&aad_prefix)
+            .encode()?;
+
+        let encryptor = Arc::new(AesGcmFileEncryptor::new(dek.as_bytes(), 
aad_prefix)?);
+
+        Ok(EncryptedOutputFile::new(
+            raw_output,
+            key_metadata_bytes,
+            encryptor,
+        ))
+    }
+
+    /// Decrypt an encrypted input file, returning an [`EncryptedInputFile`]
+    /// that transparently decrypts on read.
+    pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> 
Result<EncryptedInputFile> {
+        let metadata = StandardKeyMetadata::decode(key_metadata)?;
+
+        let decryptor = Arc::new(AesGcmFileDecryptor::new(
+            metadata.encryption_key().as_bytes(),
+            metadata.aad_prefix().unwrap_or_default(),
+        )?);
+
+        Ok(EncryptedInputFile::new(input, decryptor))
+    }
+
+    /// Generate key material for Parquet Modular Encryption (PME).
+    ///
+    /// Returns a [`StandardKeyMetadata`] containing a fresh DEK and AAD 
prefix.
+    /// The caller should pass this to the Parquet writer to configure
+    /// `FileEncryptionProperties`, and serialize it for storage in the 
manifest.
+    pub fn generate_native_key_metadata(&self) -> Result<StandardKeyMetadata> {
+        let dek = SecureKey::generate(self.key_size);
+        let aad_prefix = Self::generate_aad_prefix();
+        
Ok(StandardKeyMetadata::new(dek.as_bytes()).with_aad_prefix(&aad_prefix))
+    }
+
+    /// Wrap key metadata bytes with a KEK for storage in table metadata.
+    ///
+    /// Returns `(wrapped_entry, optional_new_kek)`. The wrapped entry
+    /// contains the key metadata encrypted by the KEK, and should be stored
+    /// in `TableMetadata.encryption_keys`. The optional second element is a
+    /// newly created KEK — present only when no active KEK existed (first
+    /// write) or the existing KEK expired (rotation). When `Some`, the
+    /// caller must also persist this KEK in table metadata so that future
+    /// `unwrap_key_metadata` calls can find it.
+    pub async fn wrap_key_metadata(

Review Comment:
   Thanks for explaination. Seems it's not used for now? I prefer to remove it 
if it's not used and add it back later to make the pr small.



-- 
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]

Reply via email to