You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by sh...@apache.org on 2022/04/14 16:08:41 UTC
[parquet-mr] branch master updated: writer constructor with encryptor (#954)
This is an automated email from the ASF dual-hosted git repository.
shangxinli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git
The following commit(s) were added to refs/heads/master by this push:
new fb3a9051a writer constructor with encryptor (#954)
fb3a9051a is described below
commit fb3a9051a62acc65b20ec58f383a67566d76cc3d
Author: ggershinsky <gg...@users.noreply.github.com>
AuthorDate: Thu Apr 14 19:08:36 2022 +0300
writer constructor with encryptor (#954)
Co-authored-by: Gidon Gershinsky <gg...@apple.com>
---
.../parquet/crypto/InternalFileEncryptor.java | 12 ++++--
.../apache/parquet/hadoop/ParquetFileWriter.java | 50 +++++++++++++++++-----
2 files changed, 47 insertions(+), 15 deletions(-)
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileEncryptor.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileEncryptor.java
index d9619d147..cca7f6b79 100755
--- a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileEncryptor.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileEncryptor.java
@@ -84,13 +84,13 @@ public class InternalFileEncryptor {
}
}
- public InternalColumnEncryptionSetup getColumnSetup(ColumnPath columnPath,
+ public InternalColumnEncryptionSetup getColumnSetup(ColumnPath columnPath,
boolean createIfNull, int ordinal) {
InternalColumnEncryptionSetup internalColumnProperties = columnMap.get(columnPath);
if (null != internalColumnProperties) {
if (ordinal != internalColumnProperties.getOrdinal()) {
- throw new ParquetCryptoRuntimeException("Column ordinal doesnt match " + columnPath +
+ throw new ParquetCryptoRuntimeException("Column ordinal doesnt match " + columnPath +
": " + ordinal + ", "+internalColumnProperties.getOrdinal());
}
return internalColumnProperties;
@@ -113,7 +113,7 @@ public class InternalFileEncryptor {
getDataModuleEncryptor(null), getThriftModuleEncryptor(null));
} else {
internalColumnProperties = new InternalColumnEncryptionSetup(columnProperties, ordinal,
- getDataModuleEncryptor(columnProperties.getKeyBytes()),
+ getDataModuleEncryptor(columnProperties.getKeyBytes()),
getThriftModuleEncryptor(columnProperties.getKeyBytes()));
}
} else { // unencrypted column
@@ -179,6 +179,10 @@ public class InternalFileEncryptor {
return (AesGcmEncryptor) ModuleCipherFactory.getEncryptor(AesMode.GCM, footerKey);
}
+ public FileEncryptionProperties getEncryptionProperties() {
+ return fileEncryptionProperties;
+ }
+
private void fileEncryptorLog() {
String encryptedColumnList;
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = fileEncryptionProperties.getEncryptedColumns();
@@ -190,7 +194,7 @@ public class InternalFileEncryptor {
} else {
encryptedColumnList = "Every column will be encrypted with footer key.";
}
- LOG.debug("File Encryptor. Algo: {}. Encrypted footer: {}. Encrypted columns: {}",
+ LOG.debug("File Encryptor. Algo: {}. Encrypted footer: {}. Encrypted columns: {}",
fileEncryptionProperties.getAlgorithm(), fileEncryptionProperties.encryptedFooter(), encryptedColumnList);
}
}
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
index 210d6c488..e2df7af33 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
@@ -303,7 +303,7 @@ public class ParquetFileWriter {
int statisticsTruncateLength, boolean pageWriteChecksumEnabled)
throws IOException{
this(file, schema, mode, rowGroupSize, maxPaddingSize, columnIndexTruncateLength,
- statisticsTruncateLength, pageWriteChecksumEnabled, null);
+ statisticsTruncateLength, pageWriteChecksumEnabled, null, null);
}
public ParquetFileWriter(OutputFile file, MessageType schema, Mode mode,
@@ -311,6 +311,24 @@ public class ParquetFileWriter {
int statisticsTruncateLength, boolean pageWriteChecksumEnabled,
FileEncryptionProperties encryptionProperties)
throws IOException {
+ this(file, schema, mode, rowGroupSize, maxPaddingSize, columnIndexTruncateLength,
+ statisticsTruncateLength, pageWriteChecksumEnabled, encryptionProperties, null);
+ }
+
+ public ParquetFileWriter(OutputFile file, MessageType schema, Mode mode,
+ long rowGroupSize, int maxPaddingSize, int columnIndexTruncateLength,
+ int statisticsTruncateLength, boolean pageWriteChecksumEnabled,
+ InternalFileEncryptor encryptor)
+ throws IOException {
+ this(file, schema, mode, rowGroupSize, maxPaddingSize, columnIndexTruncateLength,
+ statisticsTruncateLength, pageWriteChecksumEnabled, null, encryptor);
+ }
+
+ private ParquetFileWriter(OutputFile file, MessageType schema, Mode mode,
+ long rowGroupSize, int maxPaddingSize, int columnIndexTruncateLength,
+ int statisticsTruncateLength, boolean pageWriteChecksumEnabled,
+ FileEncryptionProperties encryptionProperties, InternalFileEncryptor encryptor)
+ throws IOException {
TypeUtil.checkValidWriteSchema(schema);
this.schema = schema;
@@ -336,20 +354,30 @@ public class ParquetFileWriter {
this.metadataConverter = new ParquetMetadataConverter(statisticsTruncateLength);
- if (null == encryptionProperties) {
+ if (null == encryptionProperties && null == encryptor) {
this.fileEncryptor = null;
- } else {
- // Verify that every encrypted column is in file schema
- Map<ColumnPath, ColumnEncryptionProperties> columnEncryptionProperties = encryptionProperties.getEncryptedColumns();
- if (null != columnEncryptionProperties) { // if null, every column in file schema will be encrypted with footer key
- for (Map.Entry<ColumnPath, ColumnEncryptionProperties> entry : columnEncryptionProperties.entrySet()) {
- String[] path = entry.getKey().toArray();
- if(!schema.containsPath(path)) {
- throw new ParquetCryptoRuntimeException("Encrypted column " + Arrays.toString(path) + " not in file schema");
- }
+ return;
+ }
+
+ if (null == encryptionProperties) {
+ encryptionProperties = encryptor.getEncryptionProperties();
+ }
+
+ // Verify that every encrypted column is in file schema
+ Map<ColumnPath, ColumnEncryptionProperties> columnEncryptionProperties = encryptionProperties.getEncryptedColumns();
+ if (null != columnEncryptionProperties) { // if null, every column in file schema will be encrypted with footer key
+ for (Map.Entry<ColumnPath, ColumnEncryptionProperties> entry : columnEncryptionProperties.entrySet()) {
+ String[] path = entry.getKey().toArray();
+ if (!schema.containsPath(path)) {
+ throw new ParquetCryptoRuntimeException("Encrypted column " + Arrays.toString(path) + " not in file schema");
}
}
+ }
+
+ if (null == encryptor) {
this.fileEncryptor = new InternalFileEncryptor(encryptionProperties);
+ } else {
+ this.fileEncryptor = encryptor;
}
}