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

[PARQUET-2364] "Encrypt all columns" option #1178

Merged
merged 1 commit into from
Oct 26, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,27 +33,36 @@ public class FileEncryptionProperties {

private static final ParquetCipher ALGORITHM_DEFAULT = ParquetCipher.AES_GCM_V1;
private static final boolean ENCRYPTED_FOOTER_DEFAULT = true;
private static final boolean COMPLETE_COLUMN_ENCRYPTION_DEFAULT = false;

private final EncryptionAlgorithm algorithm;
private final boolean encryptedFooter;
private final byte[] footerKey;
private final byte[] footerKeyMetadata;
private final byte[] fileAAD;
private final Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap;
private final boolean completeColumnEncryption;

private FileEncryptionProperties(ParquetCipher cipher,
byte[] footerKey, byte[] footerKeyMetadata, boolean encryptedFooter,
byte[] aadPrefix, boolean storeAadPrefixInFile,
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap) {
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap,
boolean completeColumnEncryption) {

if (null == footerKey) {
throw new IllegalArgumentException("Footer key is null");
}
if (! (footerKey.length == 16 || footerKey.length == 24 || footerKey.length == 32)) {
throw new IllegalArgumentException("Wrong footer key length " + footerKey.length);
}
if (null != columnPropertyMap && columnPropertyMap.size() == 0) {
throw new IllegalArgumentException("No encrypted columns");
if (null != columnPropertyMap) {
if (columnPropertyMap.size() == 0) {
throw new IllegalArgumentException("No encrypted columns");
}
} else {
if (completeColumnEncryption) {
throw new IllegalArgumentException("Encrypted columns are not specified, cannot complete");
}
}

SecureRandom random = new SecureRandom();
Expand Down Expand Up @@ -88,6 +97,7 @@ private FileEncryptionProperties(ParquetCipher cipher,
this.footerKeyMetadata = footerKeyMetadata;
this.encryptedFooter = encryptedFooter;
this.columnPropertyMap = columnPropertyMap;
this.completeColumnEncryption = completeColumnEncryption;
}

/**
Expand All @@ -109,10 +119,12 @@ public static class Builder {
private byte[] aadPrefix;
private Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap;
private boolean storeAadPrefixInFile;
private boolean completeColumnEncryption;

private Builder(byte[] footerKey) {
this.parquetCipher = ALGORITHM_DEFAULT;
this.encryptedFooter = ENCRYPTED_FOOTER_DEFAULT;
this.completeColumnEncryption = COMPLETE_COLUMN_ENCRYPTION_DEFAULT;
this.footerKeyBytes = new byte[footerKey.length];
System.arraycopy(footerKey, 0, this.footerKeyBytes, 0, footerKey.length);
}
Expand Down Expand Up @@ -229,11 +241,17 @@ public Builder withEncryptedColumns(Map<ColumnPath, ColumnEncryptionProperties>
return this;
}

public Builder withCompleteColumnEncryption() {
this.completeColumnEncryption = true;

return this;
}

public FileEncryptionProperties build() {
return new FileEncryptionProperties(parquetCipher,
footerKeyBytes, footerKeyMetadata, encryptedFooter,
aadPrefix, storeAadPrefixInFile,
columnPropertyMap);
columnPropertyMap, completeColumnEncryption);
}
}

Expand Down Expand Up @@ -261,9 +279,14 @@ public ColumnEncryptionProperties getColumnProperties(ColumnPath columnPath) {
ColumnEncryptionProperties columnProperties = columnPropertyMap.get(columnPath);
if (null != columnProperties) {
return columnProperties;
} else {
// plaintext column
return ColumnEncryptionProperties.builder(columnPath, false).build();
} else { // not set explicitly
if (completeColumnEncryption) {
// encrypted with footer key
return ColumnEncryptionProperties.builder(columnPath, true).build();
} else {
// plaintext column
return ColumnEncryptionProperties.builder(columnPath, false).build();
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.parquet.crypto.ColumnDecryptionProperties;
Expand All @@ -45,11 +44,8 @@
import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import okhttp3.ConnectionSpec;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Request;
import okhttp3.RequestBody;
import okhttp3.Response;

import java.io.IOException;
Expand Down Expand Up @@ -100,6 +96,9 @@
* ENCRYPT_COLUMNS_PLAINTEXT_FOOTER: Encrypt six columns, with different keys.
* Do not encrypt footer (to enable legacy readers)
* - plaintext footer mode.
* ENCRYPT_COLUMNS_PLAINTEXT_FOOTER_COMPLETE: Encrypt six columns with different keys
* Do not encrypt footer. Encrypt the rest of the columns
* with the footer key.
* ENCRYPT_COLUMNS_AND_FOOTER_AAD: Encrypt six columns and the footer, with different
* keys. Supply aad_prefix for file identity
* verification.
Expand Down Expand Up @@ -255,6 +254,21 @@ public FileEncryptionProperties getEncryptionProperties() {
.withFooterKeyMetadata(footerKeyMetadata).build();
}
},
ENCRYPT_COLUMNS_PLAIN_FOOTER_COMPLETE {
/**
* Encryption configuration 8: Encrypt six columns with different keys.
* Encrypt the rest of the columns with the footer key. Don't encrypt footer.
*/
public FileEncryptionProperties getEncryptionProperties() {
Map<ColumnPath, ColumnEncryptionProperties> columnPropertiesMap = getColumnEncryptionPropertiesMap();
return FileEncryptionProperties.builder(FOOTER_ENCRYPTION_KEY)
.withFooterKeyMetadata(footerKeyMetadata)
.withEncryptedColumns(columnPropertiesMap)
.withCompleteColumnEncryption()
.withPlaintextFooter()
.build();
}
},
NO_ENCRYPTION {
public FileEncryptionProperties getEncryptionProperties() {
return null;
Expand Down Expand Up @@ -402,6 +416,14 @@ private void testReadEncryptedParquetFiles(Path root, List<SingleRow> data) {
.named("FormatTestObject").toString());
}

// Project column encrypted with footer key
if ((decryptionConfiguration == DecryptionConfiguration.NO_DECRYPTION) &&
(encryptionConfiguration == EncryptionConfiguration.ENCRYPT_COLUMNS_PLAIN_FOOTER_COMPLETE)) {
conf.set("parquet.read.schema", Types.buildMessage()
.optional(INT32).named(SingleRow.PLAINTEXT_INT32_FIELD_NAME)
.named("FormatTestObject").toString());
}

int rowNum = 0;
try (ParquetReader<Group> reader = ParquetReader.builder(new GroupReadSupport(), file)
.withConf(conf)
Expand Down Expand Up @@ -471,6 +493,9 @@ private void downloadInteropFiles(Path rootPath, boolean readOnlyEncrypted, OkHt
if (EncryptionConfiguration.UNIFORM_ENCRYPTION_PLAINTEXT_FOOTER == encryptionConfiguration) {
continue;
}
if (EncryptionConfiguration.ENCRYPT_COLUMNS_PLAIN_FOOTER_COMPLETE == encryptionConfiguration) {
continue;
}
String fileName = getFileName(encryptionConfiguration);
Path file = new Path(rootPath, fileName);
if (!fs.exists(file)) {
Expand Down Expand Up @@ -500,6 +525,9 @@ private void testInteropReadEncryptedParquetFiles(Path root, boolean readOnlyEnc
if (EncryptionConfiguration.UNIFORM_ENCRYPTION_PLAINTEXT_FOOTER == encryptionConfiguration) {
continue;
}
if (EncryptionConfiguration.ENCRYPT_COLUMNS_PLAIN_FOOTER_COMPLETE == encryptionConfiguration) {
continue;
}
Path file = new Path(root, getFileName(encryptionConfiguration));
LOG.info("==> Decryption configuration {}", decryptionConfiguration);
FileDecryptionProperties fileDecryptionProperties = decryptionConfiguration.getDecryptionProperties();
Expand Down Expand Up @@ -593,7 +621,7 @@ private void checkResult(String file, DecryptionConfiguration decryptionConfigur
return;
}
}
// Encryption_configuration 7 has null encryptor, so parquet is plaintext.
// Last encryption_configuration has null encryptor, so parquet is plaintext.
// An exception is expected to be thrown if the file is being decrypted.
if (encryptionConfiguration == EncryptionConfiguration.NO_ENCRYPTION) {
if ((decryptionConfiguration == DecryptionConfiguration.DECRYPT_WITH_KEY_RETRIEVER) ||
Expand Down