Skip to content

feat(encryption) [5/N] Support encryption: Encryption Manager#2383

Open
xanderbailey wants to merge 16 commits intoapache:mainfrom
xanderbailey:xb/encryption_manager
Open

feat(encryption) [5/N] Support encryption: Encryption Manager#2383
xanderbailey wants to merge 16 commits intoapache:mainfrom
xanderbailey:xb/encryption_manager

Conversation

@xanderbailey
Copy link
Copy Markdown
Contributor

@xanderbailey xanderbailey commented Apr 28, 2026

Which issue does this PR close?

Part of #2034

What changes are included in this PR?

Stacked on #2340. Adds EncryptionManager — handles two-layer envelope encryption (master key → KEK → DEK) for Iceberg tables.

New files

  • manager.rsEncryptionManager with typed_builder construction:
    • encrypt() — wraps an OutputFile in an EncryptedOutputFile with a freshly generated DEK + AAD prefix
    • wrap_key_metadata() / unwrap_key_metadata() — KEK envelope wrap/unwrap for manifest list key metadata
    • KEK lifecycle: creation, rotation (730-day NIST SP 800-57 lifespan), caching (1-hour TTL via moka)
    • Builder exposes both encryption_keys(HashMap) (bulk, for production load from TableMetadata) and add_encryption_key(EncryptedKey) (one-at-a-time, ergonomic for tests)
  • io.rsEncryptedInputFile / EncryptedOutputFile wrappers that hold a StandardKeyMetadata and lazily build the AGS1 stream cipher on reader() / writer()

Design decisions

  • No NativeEncryptedInputFile / NativeEncryptedOutputFile wrappers — For PME, StandardKeyMetadata already carries the DEK and AAD prefix. Wrapper types that just bundle an InputFile/OutputFile with the same data are unnecessary indirection.
  • No decrypt() method on the manager — it had no manager state to use; callers do StandardKeyMetadata::decode(bytes) + EncryptedInputFile::new(input, metadata) directly. Mirrors the fact that Java's decrypt() is also a thin factory unrelated to manager state.
  • No AesGcmFileDecryptor / AesGcmFileEncryptor — the encrypted file types store StandardKeyMetadata directly and construct the underlying AesGcmFileRead / AesGcmFileWrite on demand. One fewer layer.
  • KEK timestamp AAD is required — missing/tampered timestamps fail with a clear DataInvalid error rather than silently passing None AAD (which would weaken the tampering defense).

How this differs from Java's StandardEncryptionManager

  • KEK management is explicit: Java's addManifestListKeyMetadata() mutates an internal map and callers need to downcast to StandardEncryptionManager to access the keys. Our wrap_key_metadata() returns (wrapped_key, Option<new_kek>) directly — no hidden mutation, no downcasting. Java reference.
  • No EncryptionUtil grab-bag: Java needs a static utility class (decryptManifestListKeyMetadata, encryptManifestListKeyMetadata, etc.) because the interface is too narrow. Here those are just methods on EncryptionManager.
  • table_key_id enforced at compile time: required via typed_builder, can't forget it. Unencrypted tables use Option<EncryptionManager> instead of Java's PlaintextEncryptionManager.
  • Config values match Java exactly: KEK lifespan (730 days), cache TTL (1 hour), AAD prefix length (16 bytes) are all hardcoded.

Usage notes (for follow-up PRs that wire this in)

load_manifest_list will do something like:

let unwrapped_km = em
    .unwrap_key_metadata(encrypted_key, table_metadata.encryption_keys_map())
    .await?;
let metadata = StandardKeyMetadata::decode(&unwrapped_km)?;
let input = file_io.new_input(&self.manifest_list)?;
EncryptedInputFile::new(input, metadata).read().await?

Are these changes tested?

Yes — 11 manager tests covering KEK creation/rotation/caching, wrap/unwrap roundtrips, AAD tampering and missing-timestamp rejection, plus a full encrypt-then-read roundtrip via EncryptedInputFile.

@xanderbailey xanderbailey changed the title feat(encryption) [4/N] Support encryption: Encryption Manager feat(encryption) [5/N] Support encryption: Encryption Manager Apr 28, 2026
@xanderbailey xanderbailey force-pushed the xb/encryption_manager branch from b7fe819 to 74f9e2f Compare April 29, 2026 08:44
default = moka::future::Cache::builder().time_to_live(DEFAULT_CACHE_TTL).build(),
setter(skip)
)]
kek_cache: moka::future::Cache<String, SensitiveBytes>,
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Put a cache here for the same reason Java does, KMS is not aware of the key_id and since we populate the cache when we create_kek it would be hard to do this purely within a CachingKmsService

@xanderbailey xanderbailey marked this pull request as draft April 29, 2026 20:16
@xanderbailey xanderbailey marked this pull request as ready for review April 29, 2026 21:15
@mbutrovich mbutrovich self-requested a review May 5, 2026 18:01
@mbutrovich
Copy link
Copy Markdown
Collaborator

mbutrovich commented May 5, 2026

Thanks @xanderbailey! Here's my summary of reviewing with Claude:

Correctness

1. Missing KEY_TIMESTAMP on a KEK should fail-fast, not silently pass None AAD

encryption_manager.rs:500-503 (in decrypt_dek) and the symmetric wrap site (:350-352):

let aad = kek.properties().get(KEK_CREATED_AT_PROPERTY).map(|ts| ts.as_bytes());

Java asserts Preconditions.checkState(keyEncryptionKeyTimestamp != null, "Key encryption key must be timestamped") (EncryptionUtil.java:161-162). Here a KEK missing the timestamp silently passes None on both wrap and unwrap — the pair roundtrips, but the tampering defense is gone. And if one side has the property and the other doesn't, you get a generic AES-GCM auth failure instead of a clear "corrupted KEK" error.

Suggest: require the property in both paths and return ErrorKind::DataInvalid with a clear message if it's missing. create_kek always sets it, so the only way to hit this is a malformed table — which is exactly the case we want to surface loudly.

2. Manager doesn't internalize newly created KEKs — deviates from the RFC's write flow

RFC §"Write Path", step 2.e: "Store as EncryptedKey (encrypted_by_id = kek_id) in encryption manager."

Today wrap_key_metadata takes &self, encryption_keys is frozen at builder time, and the new KEK is returned to the caller to persist. In the single-manifest-list-per-commit flow this is fine — but it does mean the RFC and implementation disagree.

Two reasonable resolutions:

  • Update the RFC wording to reflect the current design (caller persists the returned new KEK) — I actually like this better, it's a cleaner ownership story than Java's hidden mutation.
  • Or use interior mutability (e.g. ArcSwap<HashMap<...>> / RwLock<...>) so the manager can be reused across calls in one commit.

Either way, worth resolving with the RFC before this lands so [6/N] onward doesn't have to walk back either side.

API / naming

3. wrap_key_metadata return type

The Option<EncryptedKey> in (EncryptedKey, Option<EncryptedKey>) has subtle "persist-this-or-lose-it" semantics that's easy to miss at call sites. A named struct makes it self-documenting:

pub struct WrappedKeyMetadata {
    pub entry: EncryptedKey,
    pub new_kek: Option<EncryptedKey>, // Some only when a KEK was created/rotated
}

4. Public constant name

KEK_CREATED_AT_PROPERTY holds the value "KEY_TIMESTAMP" — which matches Java. But the Rust identifier diverges from Java's KEY_TIMESTAMP constant. Since this is public API, matching Java's identifier name makes cross-reading easier.

5. EncryptedInputFile / EncryptedOutputFile are AGS1-only

Names suggest generality, docstring says "AGS1 stream-encrypted". Java disambiguates with AesGcmInputFile vs NativeEncryptionInputFile. Since the RFC also has PME/native paths coming, consider AesGcmStreamInputFile / AesGcmStreamOutputFile to leave room.

6. RFC/impl naming drift

RFC §"EncryptionManager" lists encrypt_native() / generate_dek(); the PR has generate_native_key_metadata() (returning the full StandardKeyMetadata). Which way should the RFC move? Worth aligning in one place.

Behavioral divergences from Java (not spec-mandated, just worth a comment)

7. find_active_kek selection order

max_by_key(timestamp) picks the newest unexpired KEK. Java picks the first unexpired one by LinkedHashMap insertion order (StandardEncryptionManager.java:162-170). The Rust choice is deterministic (HashMap iteration isn't, so max_by_key is necessary) and arguably more correct. A one-line comment explaining the intentional divergence would help.

8. KEK key_id format

Rust: Uuid::new_v4().to_string(). Java: 16 random bytes, Base64-encoded (generateKeyId() at StandardEncryptionManager.java:228-232). Spec doesn't constrain format. Interop works since both sides just compare strings, but worth noting for tables that might be read by Java tooling.

Tests

9. Missing coverage

  • AAD tampering: mutate a KEK's KEY_TIMESTAMP property in table metadata, assert unwrap_key_metadata fails. This exercises the tampering defense the timestamp-as-AAD scheme is designed to catch.
  • Missing-timestamp rejection: once chore: Setup project layout #1 above is fixed, add a test asserting a KEK without KEY_TIMESTAMP fails wrap and unwrap with a clear error.
  • wrap_key_metadata twice on same manager with empty initial encryption_keys: currently test_kek_reuse_when_not_expired constructs a second manager with the first KEK pre-populated, which is exactly the path that works. Testing the same-manager case would pin down whichever decision comes out of Design of Serialization/Deserialization #2.

Nits

  • encryption_manager.rs:498 comment says "The Iceberg Spec uses the KEK timestamp as AAD". The spec doesn't mandate this — it's a Java implementation choice that we're matching. Reword as "matches Java's wrap scheme" to avoid overclaiming the spec.
  • crypto.rs:180-186TryFrom<SensitiveBytes> for SecureKey is fine, but wrap_dek_with_kek / unwrap_dek_with_kek (encryption_manager.rs:530, 542) clone the SensitiveBytes just to call try_from, duplicating zeroing key material into a second buffer. A TryFrom<&SensitiveBytes> impl (or a &SensitiveBytes AsRef-style path) would avoid the clone.
  • decrypt_dek wraps the inner error with "Failed to unwrap key metadata with KEK..." — the caller (unwrap_key_metadata) is already named for this context, so the wrapper message is mostly redundant. Not worth changing on its own, but consider when you touch that error path.
  • Confirm uuid was added to Cargo.toml as part of this PR (not visible in the diff).

@xanderbailey
Copy link
Copy Markdown
Contributor Author

  1. Missing KEY_TIMESTAMP on a KEK should fail-fast, not silently pass None AAD
    encryption_manager.rs:500-503 (in decrypt_dek) and the symmetric wrap site (:350-352):

let aad = kek.properties().get(KEK_CREATED_AT_PROPERTY).map(|ts| ts.as_bytes());
Java asserts Preconditions.checkState(keyEncryptionKeyTimestamp != null, "Key encryption key must be timestamped") (EncryptionUtil.java:161-162). Here a KEK missing the timestamp silently passes None on both wrap and unwrap — the pair roundtrips, but the tampering defense is gone. And if one side has the property and the other doesn't, you get a generic AES-GCM auth failure instead of a clear "corrupted KEK" error.

Suggest: require the property in both paths and return ErrorKind::DataInvalid with a clear message if it's missing. create_kek always sets it, so the only way to hit this is a malformed table — which is exactly the case we want to surface loudly.

7f0ffcc

@xanderbailey
Copy link
Copy Markdown
Contributor Author

  1. EncryptedInputFile / EncryptedOutputFile are AGS1-only
    Names suggest generality, docstring says "AGS1 stream-encrypted". Java disambiguates with AesGcmInputFile vs NativeEncryptionInputFile. Since the RFC also has PME/native paths coming, consider AesGcmStreamInputFile / AesGcmStreamOutputFile to leave room.

We don't have NativeEncryptionInputFile / NativeEncryptionOutputFile so I think this is maybe fine? WDYT?

@xanderbailey
Copy link
Copy Markdown
Contributor Author

  1. find_active_kek selection order
    max_by_key(timestamp) picks the newest unexpired KEK. Java picks the first unexpired one by LinkedHashMap insertion order (StandardEncryptionManager.java:162-170). The Rust choice is deterministic (HashMap iteration isn't, so max_by_key is necessary) and arguably more correct. A one-line comment explaining the intentional divergence would help.

Do we think we actually need a comment here? The code seems self documenting here?

@xanderbailey
Copy link
Copy Markdown
Contributor Author

  1. KEK key_id format
    Rust: Uuid::new_v4().to_string(). Java: 16 random bytes, Base64-encoded (generateKeyId() at StandardEncryptionManager.java:228-232). Spec doesn't constrain format. Interop works since both sides just compare strings, but worth noting for tables that might be read by Java tooling.

I think UUID is fine here maybe? both are 16 bytes and random

@mbutrovich
Copy link
Copy Markdown
Collaborator

  1. KEK key_id format
    Rust: Uuid::new_v4().to_string(). Java: 16 random bytes, Base64-encoded (generateKeyId() at StandardEncryptionManager.java:228-232). Spec doesn't constrain format. Interop works since both sides just compare strings, but worth noting for tables that might be read by Java tooling.

I think UUID is fine here maybe? both are 16 bytes and random

Agreed.

@mbutrovich
Copy link
Copy Markdown
Collaborator

  1. find_active_kek selection order
    max_by_key(timestamp) picks the newest unexpired KEK. Java picks the first unexpired one by LinkedHashMap insertion order (StandardEncryptionManager.java:162-170). The Rust choice is deterministic (HashMap iteration isn't, so max_by_key is necessary) and arguably more correct. A one-line comment explaining the intentional divergence would help.

Do we think we actually need a comment here? The code seems self documenting here?

Agreed.

@mbutrovich
Copy link
Copy Markdown
Collaborator

  1. EncryptedInputFile / EncryptedOutputFile are AGS1-only
    Names suggest generality, docstring says "AGS1 stream-encrypted". Java disambiguates with AesGcmInputFile vs NativeEncryptionInputFile. Since the RFC also has PME/native paths coming, consider AesGcmStreamInputFile / AesGcmStreamOutputFile to leave room.

We don't have NativeEncryptionInputFile / NativeEncryptionOutputFile so I think this is maybe fine? WDYT?

Yeah, seems fine.

Copy link
Copy Markdown
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!

default = moka::future::Cache::builder().time_to_live(DEFAULT_CACHE_TTL).build(),
setter(skip)
)]
kek_cache: moka::future::Cache<String, SensitiveBytes>,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Please don't use fully qualified name here, use them in imports as much as possible.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Comment thread crates/iceberg/src/encryption/io.rs
Comment thread crates/iceberg/src/encryption/manager.rs
Comment thread crates/iceberg/src/encryption/manager.rs
Comment thread crates/iceberg/src/encryption/io.rs Outdated
/// Transparently decrypts on read.
pub struct EncryptedInputFile {
inner: InputFile,
decryptor: Arc<AesGcmFileDecryptor>,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think we should just store StandardKeyMetadata here?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

And just use that to construct the the FileRead when we need it?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

The AesGcmFileDecryptor is the thing we actually need at read time so is it a good idea to just construct it once?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

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 

Comment thread crates/iceberg/src/encryption/io.rs Outdated
}

/// Returns the key metadata bytes (for storage in manifest/data files).
pub fn key_metadata(&self) -> &[u8] {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

We should return StandardKeyMetadata

/// 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> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Let's make it mod private first/

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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


/// Decrypt an encrypted input file, returning an [`EncryptedInputFile`]
/// that transparently decrypts on read.
pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> Result<EncryptedInputFile> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Suggested change
pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> Result<EncryptedInputFile> {
pub fn decrypt(&self, input: EncryptedInputeFile) -> Result<EncryptedInputFile> {

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This seems cyclical to me no?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, we got in trouble here. After second thought, I think we should remove this method. Java's implementation in fact has nothing to do with EncryptionManager. WDYT?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I think you're right yes, this was a good catch, we can remove this altogether. We can now create EncryptedInputFile from an InputFile and StandardKeyMetadata

/// 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(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

It's deprecated in java, remove it.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I think wrapKey and unwrapKey are deprecated. wrap_key_metadata maps to EncryptionUtil.encryptManifestListKeyMetadata() and unwrap_key_metadata maps to EncryptionUtil.decryptManifestListKeyMetadata() but I've tried to avoid the EncryptionUtil grab-bag in my implementation.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

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.

Copy link
Copy Markdown
Contributor Author

@xanderbailey xanderbailey May 8, 2026

Choose a reason for hiding this comment

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

Maybe we can keep these, EncryptionManager now just has three public methods:

encrypt

wrap_key_metadata

unwrap_key_metadata

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

The problem is it will take more time for me to understand how java's equivalent methods work. I think the io part is already ready for merging. I'm fine if you insist on putting them together, but it will take longer for the pr to be merged.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yep, that makes sense to me, happy to remove the EM from this PR and merge the IO

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Also I don't think you should use general name like wrap_key_metadata/unwrap_key_metadata. The logic here are manifest list specific, e.g. it requires looks up EncryptedKeys in table metadata, which others could be expressed using EncryptedInputFile.

/// Given an `EncryptedKey` entry (from a manifest list or snapshot) and
/// the full map of encryption keys from `TableMetadata`, returns the
/// unwrapped key metadata bytes (e.g. serialized `StandardKeyMetadata`).
pub async fn unwrap_key_metadata(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Ditto.

@xanderbailey xanderbailey force-pushed the xb/encryption_manager branch from 5311801 to 32fdd3a Compare May 6, 2026 09:50

/// Decrypt an encrypted input file, returning an [`EncryptedInputFile`]
/// that transparently decrypts on read.
pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> Result<EncryptedInputFile> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, we got in trouble here. After second thought, I think we should remove this method. Java's implementation in fact has nothing to do with EncryptionManager. WDYT?

/// 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)]
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

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

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Have removed for now, we can add it back in when we need it 5107145

/// 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(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

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.

Comment thread crates/iceberg/src/encryption/io.rs Outdated
/// Transparently decrypts on read.
pub struct EncryptedInputFile {
inner: InputFile,
decryptor: Arc<AesGcmFileDecryptor>,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

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 

@xanderbailey
Copy link
Copy Markdown
Contributor Author

xanderbailey commented May 8, 2026

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:

Yeah okay so this is a good take. If we do this then it becomes very clear that decrypt gives us nothing:

    pub fn decrypt(&self, input: InputFile, key_metadata: &[u8]) -> Result<EncryptedInputFile> {
        let metadata = StandardKeyMetadata::decode(key_metadata)?;
        Ok(EncryptedInputFile::new(input, metadata))
    }

@xanderbailey
Copy link
Copy Markdown
Contributor Author

Just a thought, worried that if we remove the wrapping and unwrapping of keys, the EM is basically empty (doesn't need kms etc). I could remove the EM from this PR and just have the io, we could merge that and then introduce the EM when we need it in the next PR that does the manifest list encryption decryption?

Or review the EM as is with the key wrapping logic?

@blackmwk
Copy link
Copy Markdown
Contributor

blackmwk commented May 9, 2026

Or review the EM as is with the key wrapping logic?

Ok, let's review them in the same pr.

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.

3 participants