ggershinsky commented on code in PR #2183: URL: https://github.com/apache/iceberg-rust/pull/2183#discussion_r2882362872
########## docs/rfcs/0003_table_encryption.md: ########## @@ -0,0 +1,597 @@ +<!-- + ~ 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. +--> + +# Table Encryption + +## Background + +### Iceberg Spec: Encryption + +The [Iceberg table spec](https://iceberg.apache.org/spec/#table-metadata) defines encryption +as a first-class concept. Tables may store an `encryption-keys` map in their metadata, +snapshots may reference an `encryption-key-id`, and manifest files carry optional +`key_metadata` bytes. Data files themselves can be encrypted either at the stream level +(AES-GCM envelope encryption, the "AGS1" format) or natively by the file format (e.g. +Parquet Modular Encryption). + +The Java implementation (`org.apache.iceberg.encryption`) is the reference and has been +production-tested. It defines: + +- **`EncryptionManager`** -- orchestrates encrypt/decrypt of `InputFile`/`OutputFile` +- **`KeyManagementClient`** -- pluggable KMS integration (wrap/unwrap keys) +- **`EncryptedInputFile` / `EncryptedOutputFile`** -- thin wrappers pairing a raw file handle + with its `EncryptionKeyMetadata` +- **`StandardEncryptionManager`** -- envelope encryption with key caching, AGS1 streams, + and Parquet native encryption support +- **`StandardKeyMetadata`** -- Avro-serialized key metadata (wrapped DEK, AAD prefix, file length) +- **`AesGcmInputStream` / `AesGcmOutputStream`** -- block-based stream encryption (AGS1 format) + +### Relationship to Storage Trait RFC + +[RFC 0002 (Making Storage a Trait)](https://github.com/apache/iceberg-rust/pull/2116) proposes +converting `Storage` from an enum to a trait and removing the `Extensions` mechanism from +`FileIOBuilder`. This encryption RFC is designed to work both with the current `Extensions`-based +`FileIO` and with the future trait-based storage. Specific adaptation points are called out below. + +--- + +## High-Level Architecture + +The encryption system uses two-layer envelope encryption, adapted from the Java implementation +to Rust's ownership and async model. + +### Key Hierarchy + +``` +Master Key (in KMS) + └── wraps → KEK (Key Encryption Key) — stored in table metadata as EncryptedKey + └── wraps → DEK (Data Encryption Key) — stored in StandardKeyMetadata per file Review Comment: some DEKs (those for manifest list files) are also stored in table metadata as EncryptedKey. These DEKs are indeed packaged in a StandardKeyMetadata (along with AAD prefix and file length). The serialized StandardKeyMetadata is encrypted/wrapped by the KEK, and stored in the table metadata / encrypted_keys structure. The manifest file DEKs are packaged in StandardKeyMetadata, and stored as-is (without encryption) in manifest list files. The latter are encrypted then. The data file DEKs are packaged in StandardKeyMetadata, and stored as-is (without encryption) in manifest files. The latter are encrypted then. ########## docs/rfcs/0003_table_encryption.md: ########## @@ -0,0 +1,597 @@ +<!-- + ~ 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. +--> + +# Table Encryption + +## Background + +### Iceberg Spec: Encryption + +The [Iceberg table spec](https://iceberg.apache.org/spec/#table-metadata) defines encryption +as a first-class concept. Tables may store an `encryption-keys` map in their metadata, +snapshots may reference an `encryption-key-id`, and manifest files carry optional +`key_metadata` bytes. Data files themselves can be encrypted either at the stream level +(AES-GCM envelope encryption, the "AGS1" format) or natively by the file format (e.g. +Parquet Modular Encryption). + +The Java implementation (`org.apache.iceberg.encryption`) is the reference and has been +production-tested. It defines: + +- **`EncryptionManager`** -- orchestrates encrypt/decrypt of `InputFile`/`OutputFile` +- **`KeyManagementClient`** -- pluggable KMS integration (wrap/unwrap keys) +- **`EncryptedInputFile` / `EncryptedOutputFile`** -- thin wrappers pairing a raw file handle + with its `EncryptionKeyMetadata` +- **`StandardEncryptionManager`** -- envelope encryption with key caching, AGS1 streams, + and Parquet native encryption support +- **`StandardKeyMetadata`** -- Avro-serialized key metadata (wrapped DEK, AAD prefix, file length) +- **`AesGcmInputStream` / `AesGcmOutputStream`** -- block-based stream encryption (AGS1 format) + +### Relationship to Storage Trait RFC + +[RFC 0002 (Making Storage a Trait)](https://github.com/apache/iceberg-rust/pull/2116) proposes +converting `Storage` from an enum to a trait and removing the `Extensions` mechanism from +`FileIOBuilder`. This encryption RFC is designed to work both with the current `Extensions`-based +`FileIO` and with the future trait-based storage. Specific adaptation points are called out below. + +--- + +## High-Level Architecture + +The encryption system uses two-layer envelope encryption, adapted from the Java implementation +to Rust's ownership and async model. + +### Key Hierarchy + +``` +Master Key (in KMS) + └── wraps → KEK (Key Encryption Key) — stored in table metadata as EncryptedKey + └── wraps → DEK (Data Encryption Key) — stored in StandardKeyMetadata per file + └── encrypts → file content (AGS1 stream or Parquet native) +``` + +- **Master keys** live in the KMS and never leave it +- **KEKs** are wrapped by the master key and stored in `TableMetadata.encryption_keys` +- **DEKs** are wrapped by a KEK and stored per-file in `StandardKeyMetadata` Review Comment: Only manifest list DEKs are wrapped by a KEK. Other DEKs are encrypted in the parent files, by the parent DEKs ########## docs/rfcs/0003_table_encryption.md: ########## @@ -0,0 +1,597 @@ +<!-- + ~ 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. +--> + +# Table Encryption + +## Background + +### Iceberg Spec: Encryption + +The [Iceberg table spec](https://iceberg.apache.org/spec/#table-metadata) defines encryption +as a first-class concept. Tables may store an `encryption-keys` map in their metadata, +snapshots may reference an `encryption-key-id`, and manifest files carry optional +`key_metadata` bytes. Data files themselves can be encrypted either at the stream level +(AES-GCM envelope encryption, the "AGS1" format) or natively by the file format (e.g. +Parquet Modular Encryption). + +The Java implementation (`org.apache.iceberg.encryption`) is the reference and has been +production-tested. It defines: + +- **`EncryptionManager`** -- orchestrates encrypt/decrypt of `InputFile`/`OutputFile` +- **`KeyManagementClient`** -- pluggable KMS integration (wrap/unwrap keys) +- **`EncryptedInputFile` / `EncryptedOutputFile`** -- thin wrappers pairing a raw file handle + with its `EncryptionKeyMetadata` +- **`StandardEncryptionManager`** -- envelope encryption with key caching, AGS1 streams, + and Parquet native encryption support +- **`StandardKeyMetadata`** -- Avro-serialized key metadata (wrapped DEK, AAD prefix, file length) +- **`AesGcmInputStream` / `AesGcmOutputStream`** -- block-based stream encryption (AGS1 format) + +### Relationship to Storage Trait RFC + +[RFC 0002 (Making Storage a Trait)](https://github.com/apache/iceberg-rust/pull/2116) proposes +converting `Storage` from an enum to a trait and removing the `Extensions` mechanism from +`FileIOBuilder`. This encryption RFC is designed to work both with the current `Extensions`-based +`FileIO` and with the future trait-based storage. Specific adaptation points are called out below. + +--- + +## High-Level Architecture + +The encryption system uses two-layer envelope encryption, adapted from the Java implementation +to Rust's ownership and async model. + +### Key Hierarchy + +``` +Master Key (in KMS) + └── wraps → KEK (Key Encryption Key) — stored in table metadata as EncryptedKey + └── wraps → DEK (Data Encryption Key) — stored in StandardKeyMetadata per file + └── encrypts → file content (AGS1 stream or Parquet native) +``` + +- **Master keys** live in the KMS and never leave it +- **KEKs** are wrapped by the master key and stored in `TableMetadata.encryption_keys` +- **DEKs** are wrapped by a KEK and stored per-file in `StandardKeyMetadata` +- KEKs are cached in memory (moka async cache with configurable TTL) to avoid redundant KMS calls +- KEK rotation occurs automatically when a KEK exceeds its configurable lifespan (default 730 days per NIST SP 800-57) + +### Component Overview + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ User / Table Scan │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────┐ +│ EncryptionManager (trait) │ +│ │ +│ StandardEncryptionManager: │ +│ - Two-layer envelope: Master → KEK → DEK │ +│ - KEK cache (moka async, configurable TTL) │ +│ - Automatic KEK rotation │ +│ - encrypt() / decrypt() for AGS1 stream files │ +│ - encrypt_native() for Parquet Modular Encryption │ +│ - wrap/unwrap_key_metadata() for manifest list keys │ +│ - generate_dek() with KEK management │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ │ + ▼ ▼ +┌──────────────────────────┐ ┌──────────────────────────────────────────────┐ +│ KeyManagementClient │ │ KEK Cache │ +│ (trait) │ │ │ +│ │ │ - moka::future::Cache with configurable TTL │ +│ wrap_key(key, key_id) │ │ - Thread-safe async │ +│ unwrap_key(wrapped, id) │ │ - Caches plaintext KEK bytes per key ID │ +│ initialize(props) │ │ │ +└──────────────────────────┘ └──────────────────────────────────────────────┘ + │ + ▼ +┌──────────────────────────┐ +│ KMS Implementations │ +│ │ +│ - InMemoryKms (testing) │ +│ - AWS KMS (future) │ +│ - Azure KV (future) │ +│ - GCP KMS (future) │ +└──────────────────────────┘ +``` + +### Data Flow + +#### Read Path (Decryption) + +``` +TableMetadata + └── encryption_keys: {key_id → EncryptedKey} + │ +Snapshot │ + └── encryption_key_id ──────┘ (V3 format only) + │ + ▼ + load_manifest_list(file_io, table_metadata) + 1. Look up encryption_key_id in table_metadata.encryption_keys + 2. em.unwrap_key_metadata() → plaintext key metadata + 3. file_io.new_encrypted_input(path, key_metadata) → AGS1-decrypting InputFile + │ + ▼ +ManifestFile + └── key_metadata: Option<Vec<u8>> + │ + load_manifest(file_io) + 1. If key_metadata present: file_io.new_encrypted_input() → AGS1-decrypting InputFile + 2. If not: file_io.new_input() + │ + ▼ +FileScanTask + └── key_metadata: Option<Vec<u8>> + │ + ArrowReader::create_parquet_record_batch_stream_builder_with_key_metadata() + 1. If key_metadata present: + a. file_io.new_native_encrypted_input(path, key_metadata) → NativeEncrypted InputFile + b. Build FileDecryptionProperties from NativeKeyMaterial (DEK + AAD prefix) + c. Pass to ParquetRecordBatchStreamBuilder + 2. If not: standard Parquet read +``` + +#### Write Path (Encryption) + +``` +RollingFileWriter::new_output_file() + 1. If file_io.encryption_manager() is Some: + a. file_io.new_native_encrypted_output(path) → EncryptedOutputFile + b. EncryptionManager generates DEK, wraps with KEK + c. OutputFile::NativeEncrypted carries NativeKeyMaterial for Parquet writer + d. Store key_metadata bytes on DataFile + 2. ParquetWriter detects NativeEncrypted, configures FileEncryptionProperties + +SnapshotProducer::commit() + 1. Manifest writing: + a. file_io.new_encrypted_output(path) → AGS1-encrypting OutputFile + b. Store key_metadata on ManifestFile entry + 2. Manifest list writing: + a. file_io.new_encrypted_output(path) → AGS1-encrypting OutputFile + b. em.wrap_key_metadata() → EncryptedKey for table metadata + c. Store key_id on Snapshot.encryption_key_id + 3. Table updates include AddEncryptionKey for new KEKs Review Comment: Also need to wrap the KEK (via a KMS client) ########## docs/rfcs/0003_table_encryption.md: ########## @@ -0,0 +1,597 @@ +<!-- + ~ 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. +--> + +# Table Encryption + +## Background + +### Iceberg Spec: Encryption + +The [Iceberg table spec](https://iceberg.apache.org/spec/#table-metadata) defines encryption +as a first-class concept. Tables may store an `encryption-keys` map in their metadata, +snapshots may reference an `encryption-key-id`, and manifest files carry optional +`key_metadata` bytes. Data files themselves can be encrypted either at the stream level +(AES-GCM envelope encryption, the "AGS1" format) or natively by the file format (e.g. +Parquet Modular Encryption). + +The Java implementation (`org.apache.iceberg.encryption`) is the reference and has been +production-tested. It defines: + +- **`EncryptionManager`** -- orchestrates encrypt/decrypt of `InputFile`/`OutputFile` +- **`KeyManagementClient`** -- pluggable KMS integration (wrap/unwrap keys) +- **`EncryptedInputFile` / `EncryptedOutputFile`** -- thin wrappers pairing a raw file handle + with its `EncryptionKeyMetadata` +- **`StandardEncryptionManager`** -- envelope encryption with key caching, AGS1 streams, + and Parquet native encryption support +- **`StandardKeyMetadata`** -- Avro-serialized key metadata (wrapped DEK, AAD prefix, file length) +- **`AesGcmInputStream` / `AesGcmOutputStream`** -- block-based stream encryption (AGS1 format) + +### Relationship to Storage Trait RFC + +[RFC 0002 (Making Storage a Trait)](https://github.com/apache/iceberg-rust/pull/2116) proposes +converting `Storage` from an enum to a trait and removing the `Extensions` mechanism from +`FileIOBuilder`. This encryption RFC is designed to work both with the current `Extensions`-based +`FileIO` and with the future trait-based storage. Specific adaptation points are called out below. + +--- + +## High-Level Architecture + +The encryption system uses two-layer envelope encryption, adapted from the Java implementation +to Rust's ownership and async model. + +### Key Hierarchy + +``` +Master Key (in KMS) + └── wraps → KEK (Key Encryption Key) — stored in table metadata as EncryptedKey + └── wraps → DEK (Data Encryption Key) — stored in StandardKeyMetadata per file + └── encrypts → file content (AGS1 stream or Parquet native) +``` + +- **Master keys** live in the KMS and never leave it +- **KEKs** are wrapped by the master key and stored in `TableMetadata.encryption_keys` +- **DEKs** are wrapped by a KEK and stored per-file in `StandardKeyMetadata` +- KEKs are cached in memory (moka async cache with configurable TTL) to avoid redundant KMS calls +- KEK rotation occurs automatically when a KEK exceeds its configurable lifespan (default 730 days per NIST SP 800-57) + +### Component Overview + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ User / Table Scan │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────┐ +│ EncryptionManager (trait) │ +│ │ +│ StandardEncryptionManager: │ +│ - Two-layer envelope: Master → KEK → DEK │ +│ - KEK cache (moka async, configurable TTL) │ +│ - Automatic KEK rotation │ +│ - encrypt() / decrypt() for AGS1 stream files │ +│ - encrypt_native() for Parquet Modular Encryption │ +│ - wrap/unwrap_key_metadata() for manifest list keys │ +│ - generate_dek() with KEK management │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ │ + ▼ ▼ +┌──────────────────────────┐ ┌──────────────────────────────────────────────┐ +│ KeyManagementClient │ │ KEK Cache │ +│ (trait) │ │ │ +│ │ │ - moka::future::Cache with configurable TTL │ +│ wrap_key(key, key_id) │ │ - Thread-safe async │ +│ unwrap_key(wrapped, id) │ │ - Caches plaintext KEK bytes per key ID │ +│ initialize(props) │ │ │ +└──────────────────────────┘ └──────────────────────────────────────────────┘ + │ + ▼ +┌──────────────────────────┐ +│ KMS Implementations │ +│ │ +│ - InMemoryKms (testing) │ +│ - AWS KMS (future) │ +│ - Azure KV (future) │ +│ - GCP KMS (future) │ +└──────────────────────────┘ +``` + +### Data Flow + +#### Read Path (Decryption) + +``` +TableMetadata + └── encryption_keys: {key_id → EncryptedKey} + │ +Snapshot │ + └── encryption_key_id ──────┘ (V3 format only) + │ + ▼ + load_manifest_list(file_io, table_metadata) + 1. Look up encryption_key_id in table_metadata.encryption_keys Review Comment: Also need to unwrap the KEK (via a KMS client) -- 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]
