Skip to content

RFC for table encryption#2183

Draft
xanderbailey wants to merge 4 commits intoapache:mainfrom
xanderbailey:xb/encryption_rfc
Draft

RFC for table encryption#2183
xanderbailey wants to merge 4 commits intoapache:mainfrom
xanderbailey:xb/encryption_rfc

Conversation

@xanderbailey
Copy link
Contributor

Which issue does this PR close?

RFC for table encryption
Part of: #2034
Rough draft with some of the key parts: #2042

  • Closes #.

What changes are included in this PR?

Are these changes tested?

@mbutrovich
Copy link
Collaborator

Not specific feedback for the RFC, just sharing some context links and previous discussion that was helpful in getting the EncryptionFactory design into DataFusion that interfaces with the parquet crate. Comet makes good use of this design with a custom EncryptionFactory that uses JNI to interface with Spark-based custom KMSs.

apache/datafusion#16779
apache/datafusion#15216 (comment)

Also tagging @ggershinsky in case he has any cycles to read this. His guidance was instrumental on the DataFusion and Arrow-rs PME work, as well as Iceberg Java encryption implementation.

@ggershinsky
Copy link

Thanks, I'll be glad to have a look.

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

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.


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

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Only manifest list DEKs are wrapped by a KEK. Other DEKs are encrypted in the parent files, by the parent DEKs

load_manifest_list(file_io, table_metadata)
1. Look up encryption_key_id in table_metadata.encryption_keys

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also need to unwrap the KEK (via a KMS client)

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

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also need to wrap the KEK (via a KMS client)

@xanderbailey
Copy link
Contributor Author

Thanks for taking a look @ggershinsky I've tried to fill in some of the details here

Copy link
Contributor

@blackmwk blackmwk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @xanderbailey for this pr, generally LGTM, left some suggestions.


### Iceberg Spec: Encryption

The [Iceberg table spec](https://iceberg.apache.org/spec/#table-metadata) defines encryption
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This link may be a better reference.

│ ├── kms/
│ │ ├── mod.rs
│ │ └── in_memory.rs # InMemoryKms (testing only)
│ └── integration_tests.rs # End-to-end encryption round-trip tests
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In rust we usually put integration tests in tests dir, next to src dir.


```rust
#[async_trait]
pub trait EncryptionManager: Debug + Send + Sync {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we really need this to be a trait? I think a no-op kms client would be enough?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah we actually don't need this to be a trait the standard encryption manager is the only one we need

`InputFile` and `OutputFile` are enums with three variants each:

```rust
pub enum InputFile {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we this enum should sth as following:

pub enum EncryptedInputFile {
   Encrypted {
     key_metadata: KeyMetadata
     inner: InputFile
   }, 
   NativeedEncryted {
     key_material: NativeKeyMaterial,
     inner: InputFile
   }
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this requires a lot more changes to the current code since encrypted inputs and outputs won't be handled transparently. Or is the suggestion to have InputFile as an enum of Plain, EncryptedInputFile and EncryptedInputFile is itself an enum?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I find it a little difficult to imagine what it will look like given different choice? Could you give raise some examples? I'm even thinking if we actually need an enum. InputFile, EncryptedInputFile and NativeEncryptedInputFile are used in differenent cases. Let's use an example, ManifestReader, if we remove the enum, what we need is to add an extra api for read(EncryptedInputFile). This is acceptable to me, some we have a FileRead trait to abstract out the descryption process.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm leaning towards the example snippet @blackmwk above and I don't think we should expand Input/OutputFile to an enum. EncryptedFile should be a wrapper of Input/OutputFile rather than a variation.

By the same logic, having a wrapper Storage like EncryptionStorage may not be ideal. For encryption, we only need to (un)wrap Input/OutputFile in the FileIO level

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can do this if you both think this is better but this requires some reasonable surgery on structs like ManifestListWriter which today takes an OutputFile and will now need to take a Box<dyn FileWrite> and ManifestWriter will need Box<dyn FileWrite> and a location.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay I tried it out and it's actually not too bad. If you're both happy with the changes I mentioned above then I think this is a good approach.

// Via FileIOBuilder extension (works with RestCatalog and any extension-aware catalog)
let file_io = FileIOBuilder::new("s3")
.with_prop("s3.region", "us-east-1")
.with_extension(encryption_manager)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This has been removed.

.build()?;

// Or via convenience method on FileIO
let file_io = file_io.with_encryption_manager(encryption_manager);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

// Option A: EncryptionManager on the catalog
let catalog = GlueCatalogBuilder::default()
.with_storage_factory(Arc::new(OpenDalStorageFactory::S3))
.with_encryption_manager(encryption_manager)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think what we actually need from user is KmsClientFactory?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh interesting, where they can provide a way to construct a number of KMS clients?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mean the code to be as following:

let catalog = GlueCatalogBuilder::default()
    .with_storage_factory(Arc::new(OpenDalStorageFactory::S3))
    .with_kms_client_factory(new AwsKmsClientFactory())

.load("my_catalog", props)
.await?;

// Option B: Wrapping StorageFactory
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants