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


##########
crates/iceberg/src/encryption/manager.rs:
##########
@@ -0,0 +1,666 @@
+// 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, RwLock};
+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::io::EncryptedOutputFile;
+use super::key_metadata::StandardKeyMetadata;
+use super::kms::KeyManagementClient;
+use crate::io::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
+            .write()
+            .expect("encryption_keys lock poisoned")
+            .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 = RwLock::new(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).
+    /// Newly created KEKs and wrapped manifest-list entries are inserted here 
so
+    /// callers can snapshot the full set at commit time via 
[`EncryptionManager::encryption_keys`].
+    #[builder(default = RwLock::new(HashMap::new()), via_mutators)]
+    encryption_keys: RwLock<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) -> EncryptedOutputFile {
+        let dek = SecureKey::generate(self.key_size);
+        let aad_prefix = Self::generate_aad_prefix();
+        let metadata = 
StandardKeyMetadata::new(dek.as_bytes()).with_aad_prefix(&aad_prefix);
+        EncryptedOutputFile::new(raw_output, metadata)
+    }
+
+    /// Wrap a manifest list key metadata with a KEK for storage in table 
metadata.
+    ///
+    /// Stores the resulting wrapped entry (and any newly created KEK) in the
+    /// manager's internal `encryption_keys` map. Callers persist the full set
+    /// at commit time via [`Self::encryption_keys`].
+    ///
+    /// Returns the `key_id` of the wrapped entry, which should be recorded on
+    /// the snapshot as `encryption_key_id` so readers can locate it later.
+    pub async fn encrypt_manifest_list_key_metadata(
+        &self,
+        key_metadata: &StandardKeyMetadata,
+    ) -> Result<String> {
+        let kek = match self.find_active_kek()? {
+            Some(existing) => existing,
+            None => self.create_kek().await?,
+        };
+
+        let kek_bytes = self.unwrap_key_encryption_key(&kek).await?;
+
+        // Use the KEK timestamp as AAD to prevent timestamp tampering attacks.
+        let aad = Self::kek_timestamp_aad(&kek)?;
+        let serialized = key_metadata.encode()?;
+        let wrapped_metadata = self.wrap_dek_with_kek(&serialized, &kek_bytes, 
Some(aad))?;
+
+        let wrapped_key = EncryptedKey::builder()
+            .key_id(Uuid::new_v4().to_string())
+            .encrypted_key_metadata(wrapped_metadata)
+            .encrypted_by_id(kek.key_id())
+            .build();
+
+        let wrapped_key_id = wrapped_key.key_id().to_string();
+        self.insert_encryption_key(wrapped_key);
+        Ok(wrapped_key_id)
+    }
+
+    /// Decrypt a manifest list key metadata previously wrapped via
+    /// [`Self::encrypt_manifest_list_key_metadata`].
+    ///
+    /// Looks up the entry by `encryption_key_id` (typically read from the
+    /// snapshot) in the manager's `encryption_keys` map.
+    pub async fn decrypt_manifest_list_key_metadata(
+        &self,
+        encryption_key_id: &str,
+    ) -> Result<StandardKeyMetadata> {
+        let encrypted_key = self
+            .encryption_keys
+            .read()
+            .expect("encryption_keys lock poisoned")
+            .get(encryption_key_id)
+            .cloned()
+            .ok_or_else(|| {
+                Error::new(
+                    ErrorKind::DataInvalid,
+                    format!("Encryption key '{encryption_key_id}' not found"),
+                )
+            })?;
+
+        let kek_key_id = encrypted_key.encrypted_by_id().ok_or_else(|| {
+            Error::new(
+                ErrorKind::DataInvalid,
+                format!(
+                    "EncryptedKey '{}' has no encrypted_by_id",
+                    encrypted_key.key_id()
+                ),
+            )
+        })?;
+
+        let bytes = self
+            .decrypt_dek(kek_key_id, encrypted_key.encrypted_key_metadata())
+            .await?;
+
+        StandardKeyMetadata::decode(&bytes)
+    }
+
+    /// Snapshot of the encryption keys held by this manager.
+    ///
+    /// Use at commit time to persist newly created KEKs and wrapped
+    /// manifest-list entries into `TableMetadata.encryption_keys`.
+    pub fn encryption_keys(&self) -> HashMap<String, EncryptedKey> {

Review Comment:
   Very nice! 
[60fa395](https://github.com/apache/iceberg-rust/pull/2383/commits/60fa3957e009309b7cb8326e90386f07a8bf8fcb)



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