Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Core: Encryption basics #2638

Merged
merged 11 commits into from
Apr 6, 2022

Conversation

ggershinsky
Copy link
Contributor

@ggershinsky ggershinsky commented May 26, 2021

This is a PR based on the design doc https://docs.google.com/document/d/1kkcjr9KrlB9QagRX3ToulG_Rf-65NMSlVANheDNzJq4/edit#

It introduces a new set of encryption APIs/parameters for the native encryption formats (Parquet and ORC).
The basic parameters are the column keys for a file.
In addition, it passes a file/footer key and an AADPrefix, that are consumed by Parquet but can be ignored by ORC.

For examples of calling this API, see #2640 and #2639 .

@rdblue @jackye1995 @RussellSpitzer @flyrain @aokolnychyi

@jackye1995
Copy link
Contributor

jackye1995 commented May 26, 2021

The biggest question I have regarding this is that: should we have this concept of "native" encryption?

My original thinking is that we should use native encryption as long as the file format supports it and there are enough information provided in key_metadata.

For example, a user can specify all those native configurations and enable it, but use Avro that does not have column encryption, then it still could not be applied.

The only use case for not using native I think is for backwards compatibility with some other dependent systems. In that case, we only need a boolean config flag for the reader and writer, and I feel it is unnecessary to dedicate specific parameters only for native encryption.

@ggershinsky
Copy link
Contributor Author

Sure, TBD, but we need to sync up first on this point

there are enough information provided in key_metadata.

which (in my view) seems to be a major gap here - key_metadata is a field in manifest files, which keeps the wrapped encryption keys for each data file. The keys (DEKs) are either generated randomly by Iceberg, or fetched from a KMS. The key wrapping is always done via KMS calls. Key_metadata doesn't store the keys themselves, since this would make them exposed in manifest files. Therefore, passing key_metadata to the data file writers doesn't help, because these writers need the data encryption keys. Calling KMS to unwrap them doesn't make sense in this particular point... Therefore, we need a separate mechanism to deliver the DEK(s) and other parameters to the writers. The classes in this PR list the required parameters.
We have a similar issue on the reader side, but lets get to it after syncing on the writer path.

@ggershinsky
Copy link
Contributor Author

I've dug in the code a bit, there seems to be a practical solution to this. The workers (data/delete writers) can generate random DEKs for each file - like we do today in PME - and pack them in the ContentFile objects for the delivery to the driver (manifest writer). The manifest writer will interact with a KMS to wrap the DEKs (per the single/double wrap design), and will store the result in the key_material field. I'll change the PR accordingly.
As for the "native"-vs-general encryption - there are some interesting trade-offs. Both subjects TBD.

@flyrain
Copy link
Contributor

flyrain commented Jun 2, 2021

Hi @aokolnychyi, can you approve the workflows for this PR?

@rdblue
Copy link
Contributor

rdblue commented Jun 13, 2021

I'm also skeptical about adding a "native" encryption API. It isn't clear what that is from the name and I think that using Parquet and ORC encryption is expected for those files. I would like to make the transition to configuring Parquet and ORC encryption a bit more seamless with a reasonable fallback if you want to use stream encryption for those formats.

@ggershinsky
Copy link
Contributor Author

Yep, this PR is not a public API or anything visible to the user. It is a fully internal set of classes, that carry the barebone parameters required to activate Parquet or ORC encryption (the only formats with native encryption today). These parameters are mostly the column encryption keys; plus the footer key required for Parquet (will be ignored by ORC). Plus the AAD prefix - also Parquet-specific, but can be left for a future version, when we're done with basic table encryption, and start working on end-to-end table integrity protection.

(regarding the public API, I also think there are situations where a user might want to explicitly chose either "native" Parquet/ORC column encryption - or flat stream encryption for same Parquet/ORC files. But this is likely a future/advanced option, and in any case, out of scope for this PR)

@ggershinsky ggershinsky marked this pull request as draft June 27, 2021 10:59
@ggershinsky ggershinsky changed the title Core: Native encryption params Core: Encryption basics Nov 4, 2021
@github-actions github-actions bot added the AWS label Jan 19, 2022
}

/**
* Creates the builder.
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: we normally have an empty line between line 48 and 49.

}

/**
* Set column encryption keys.
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: Empty line here as well.

@@ -59,4 +60,18 @@
* @return true if the file exists, false otherwise
*/
boolean exists();

// TODO remove this comment after review
Copy link
Contributor

Choose a reason for hiding this comment

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

Brainstorming an idea with trait-like solution:

  1. Create a new interface for encryption file or reuse the interface EncryptedInputFile, which will have method getNativeDecryptionParameters and setNativeDecryptionParameters.
  2. The subclass(HadoopInputFile, S3InputFile, etc) can implement both InputFile and the EncryptedInputFile.
  3. In case of the caller side, we can invoke it like this ((EncryptedInputFile)inputFile).getNativeDecryptionParameters()

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 like this idea. Regarding the details - using EncryptedInputFile directly won't work (because this is a producer/decryptor of input files; an object shouldn't be both an input file and its decryptor) - but we can add a new interface, something like NativeEncryption, and make the classes implement it. Moreover, we can apply this to output files, not only to input files. I'll play with this.

/**
* Parameters of native encryption (if used for this file)
*/
default NativeFileCryptoParameters nativeEncryptionParameters() {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we reuse keyMetadata instead of introducing nativeEncryptionParameters ? Here are reasons:

  1. Both keyMetadata and nativeEncryptionParameters are used for keys and crypto parameters.
  2. Each file will have either one, not both. It depends whether streaming encryption or native encryption is used for this file.

NativeFileCryptoParameters can implement interface EncryptionKeyMetadata in that sense.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not quite; all encrypted files (inc natively encrypted) have key_metadata, since it is required to decrypt them. But, as I mentioned in the previous comment, I'll play with applying a new interface to the output too, so maybe the EncryptedOutputFile doesn't need to change.

}

public byte[] decrypt(byte[] ciphertext, byte[] aad) {
int plainTextLength = ciphertext.length - AesGcmEncryptor.GCM_TAG_LENGTH - AesGcmEncryptor.NONCE_LENGTH;
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: How about moving AesGcmEncryptor.GCM_TAG_LENGTH, AesGcmEncryptor.NONCE_LENGTH, AesGcmEncryptor.GCM_TAG_LENGTH_BITS under the class Ciper? So that class AesGcmDecryptor doesn't invoke the static methods in class AesGcmEncryptor.

public AesGcmDecryptor(byte[] keyBytes) {
int keyLength = keyBytes.length;
if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
throw new IllegalArgumentException("Wrong key length " + keyLength);
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested to add the error message that key length can only be 16/24/32.

Copy link
Contributor

Choose a reason for hiding this comment

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

We need the same error message as line 42.

@ggershinsky ggershinsky marked this pull request as ready for review January 20, 2022 11:54
/**
* a minimum client interface to connect to a key management service (KMS).
*/
public interface NativelyEncryptedFile {
Copy link
Contributor

@flyrain flyrain Jan 27, 2022

Choose a reason for hiding this comment

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

Should we move it to module api instead of putting it in core? It also requires to create an interface for NativeFileCryptoParameters. Not sure that's a better way. Would like to hear people's thoughts. cc @RussellSpitzer

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It depends on the NativeFileCryptoParameters class, located in core

Copy link
Contributor

Choose a reason for hiding this comment

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

We will need an interface for NativeFileCryptoParameters in that case, which should located in module API.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

NativeFileCryptoParameters is the only class planned for these properties; having an interface just for one implementation won't be efficient. Also, this is an internal class, not designed to be a part of Iceberg API.
However, thinking about the KmsClient interface in the next PR - this one will have many implementations, and it is a part of the user-facing API in the envelope encryption. I'll move it to the API module.

@flyrain
Copy link
Contributor

flyrain commented Feb 2, 2022

Hi @ggershinsky, the PR LGTM. Can you check the CI failure?

@ggershinsky ggershinsky marked this pull request as draft February 2, 2022 06:37
@ggershinsky ggershinsky marked this pull request as ready for review February 2, 2022 06:37
@ggershinsky ggershinsky closed this Feb 2, 2022
@ggershinsky ggershinsky reopened this Feb 2, 2022
@ggershinsky
Copy link
Contributor Author

ggershinsky commented Feb 2, 2022

Done. It was a CI glitch; I've re-opened the PR, all tests pass.

Copy link
Contributor

@flyrain flyrain left a comment

Choose a reason for hiding this comment

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

+1

@RussellSpitzer
Copy link
Member

@ggershinsky Can you please fix the conflicts?

@jackye1995 could you take another pass on this? I've looked it over internally and I think we should be good to go here. I'd like to start making progress merging some of these in.

@ggershinsky
Copy link
Contributor Author

ggershinsky commented Feb 22, 2022

Can you please fix the conflicts?

Done

@ggershinsky ggershinsky force-pushed the native-encryption-params branch from 8e640da to 65d121b Compare February 23, 2022 11:41
@RussellSpitzer
Copy link
Member

We've done a bunch of review on this internally, but would like to get some more feedback before we merge it. @jackye1995 Since you are the other main architect of this, could you take a look?


public AesGcmEncryptor(byte[] keyBytes) {
int keyLength = keyBytes.length;
if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

can use Preconditions.checkArgument

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sgtm


public AesGcmDecryptor(byte[] keyBytes) {
int keyLength = keyBytes.length;
if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

can use Preconditions.checkArgument

public byte[] decrypt(byte[] ciphertext, byte[] aad) {
int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
if (plainTextLength < 1) {
throw new RuntimeException("Cannot decrypt cipher text of length " + ciphertext.length +
Copy link
Contributor

Choose a reason for hiding this comment

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

should this be an IllegalStateException to be more specific? (with Preconditions.checkState)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sgtm

return new NativeFileCryptoParameters(fileKey, fileEncryptionAlgorithm);
}

// TODO add back column encryption keys
Copy link
Contributor

@jackye1995 jackye1995 Mar 14, 2022

Choose a reason for hiding this comment

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

nit: these are new features and we already mentioned these in the top class javadoc, can we remove the TODOs?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sure

*/
public class NativeFileCryptoParameters {
private ByteBuffer fileKey;
private String fileEncryptionAlgorithm;
Copy link
Contributor

Choose a reason for hiding this comment

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

why not use the enum EncryptionAlgorithm?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

a good point. I was thinking of making the native parameters totally bottom-up (Parquet/ORC) and independent of Iceberg parameters. But it makes sense to start the mapping here and re-use the EncryptionAlgorithm enum.

@ggershinsky
Copy link
Contributor Author

hi @jackye1995 , thanks for the review. I've sent a commit that addresses it.

private final SecureRandom randomGenerator;

public AesGcmEncryptor(byte[] keyBytes) {
int keyLength = keyBytes.length;
Copy link
Contributor

Choose a reason for hiding this comment

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

Given this is public method, should we do checkNotNull(keyBytes)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sure, why not

try {
this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
} catch (GeneralSecurityException e) {
throw new RuntimeException("Failed to create GCM cipher", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Wonder if you want to define an Iceberg Crypto Runtime Exception just like you did for Parquet (ParquetCryptoRuntimeException). The reason is RuntimeException is a very generic type and we will lose some meaning when converting from GeneralSecurityException.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Mm, the callers won't have a type-specific reaction to this. The text itself provides the info for a post-mortem analysis. So probably this goes down to Iceberg's coding practices.

}

public byte[] encrypt(byte[] plainText, byte[] aad) {
byte[] nonce = new byte[NONCE_LENGTH];
Copy link
Contributor

Choose a reason for hiding this comment

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

In parquet, you have a check of excessive use of one single key(GCM_RANDOM_IV_SAME_KEY_MAX_OPS). Do you still want to do that here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This Iceberg class is designed for encryption of keys, not for encryption of data; so the risk of exceeding 2 billion operations during a process lifetime (meaning creation of 2 billion parquet files), is not real.

}
cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
} catch (GeneralSecurityException e) {
throw new RuntimeException("Failed to encrypt", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Same comments above and other places.

}
cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0);
} catch (AEADBadTagException e) {
throw new RuntimeException("GCM tag check failed", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we add more information in this exception for debugging friendly? We had several issues earlier with this exception and felt hard to know what is going on.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

agreed, this one is tricky. I'll add the info.

* A combination of GCM and CTR that can be used for file types like Parquet,
* so that all modules except pages are encrypted by GCM to ensure integrity,
* and CTR is used for efficient encryption of bulk data.
* The tradeoff is that attackers would be able to tamper page data.
Copy link
Contributor

Choose a reason for hiding this comment

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

This comment is a little misleading if we don't say "attackers would be able to tamper data" for AES_CTR

Copy link
Contributor Author

Choose a reason for hiding this comment

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

above, we have "CTR is used for .. encryption of .. data". But I'll update the comment.

private EncryptionAlgorithm fileEncryptionAlgorithm;

private NativeFileCryptoParameters(ByteBuffer fileKey, EncryptionAlgorithm fileEncryptionAlgorithm) {
Preconditions.checkState(fileKey != null, "File encryption key is not supplied");
Copy link
Contributor

Choose a reason for hiding this comment

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

key length checking also?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

key length is checked elsewhere, we shouldn't run the checks in each method that passes a key.

@RussellSpitzer
Copy link
Member

@shangxinli Do you have any remaining issues?

@shangxinli
Copy link
Contributor

LGTM

@RussellSpitzer RussellSpitzer merged commit 3d8e436 into apache:master Apr 6, 2022
@RussellSpitzer
Copy link
Member

Thanks @ggershinsky ! Good to get this first step in!

Thanks for review @jackye1995 , @shangxinli , and @flyrain !

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

Successfully merging this pull request may close these issues.

6 participants