Skip to content

Commit 055bb39

Browse files
committed
address review comments
1 parent 672e290 commit 055bb39

22 files changed

+1810
-1094
lines changed

parquet-hadoop/README.md

Lines changed: 119 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,119 @@
1+
<!--
2+
~ Licensed to the Apache Software Foundation (ASF) under one
3+
~ or more contributor license agreements. See the NOTICE file
4+
~ distributed with this work for additional information
5+
~ regarding copyright ownership. The ASF licenses this file
6+
~ to you under the Apache License, Version 2.0 (the
7+
~ "License"); you may not use this file except in compliance
8+
~ with the License. You may obtain a copy of the License at
9+
~
10+
~ http://www.apache.org/licenses/LICENSE-2.0
11+
~
12+
~ Unless required by applicable law or agreed to in writing,
13+
~ software distributed under the License is distributed on an
14+
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
~ KIND, either express or implied. See the License for the
16+
~ specific language governing permissions and limitations
17+
~ under the License.
18+
-->
19+
20+
21+
22+
## Class: ParquetOutputFormat
23+
24+
**Property:** `parquet.crypto.factory.class`
25+
**Description:** Class implementing EncryptionPropertiesFactory.
26+
**Default value:** None. If not set, the file won't be encrypted by a crypto factory.
27+
28+
29+
## Class: HadoopReadOptions
30+
31+
**Property:** `parquet.crypto.factory.class`
32+
**Description:** Class implementing DecryptionPropertiesFactory.
33+
**Default value:** None. If not set, the file won't be decrypted by a crypto factory.
34+
35+
36+
## Class: PropertiesDrivenCryptoFactory
37+
38+
**Property:** `parquet.encryption.column.keys`
39+
**Description:** List of columns to encrypt, with master key IDs (see HIVE-21848).Format: “<masterKeyID>:<colName>,<colName>;<masterKeyID>:<colName>...”
40+
**Default value:** None. If neither `column.keys` nor `footer.key` are set, the file won't be encrypted by the PropertiesDrivenCryptoFactory. If one of the two properties is set, an exception will be thrown.
41+
42+
---
43+
44+
**Property:** `parquet.encryption.footer.key`
45+
**Description:** Master key ID for footer encryption/signing.
46+
**Default value:** None. If neither `column.keys` nor `footer.key` are set, the file won't be encrypted by the PropertiesDrivenCryptoFactory. If one of the two properties is set, an exception will be thrown.
47+
48+
---
49+
50+
**Property:** `parquet.encryption.algorithm`
51+
**Description:** Parquet encryption algorithm. Can be `AES_GCM_V1` or `AES_GCM_CTR_V1`.
52+
**Default value:** `AES_GCM_V1`
53+
54+
---
55+
56+
**Property:** `parquet.encryption.plaintext.footer`
57+
**Description:** Write files in plaintext footer mode, that makes many footer fields visible (e.g. schema) but allows legacy readers to access unencrypted columns. The plaintext footer is signed with the footer key.
58+
If `false`, write files in encrypted footer mode, that fully encrypts the footer, and signs it with the footer key.
59+
**Default value:** `false`
60+
61+
---
62+
63+
**Property:** `parquet.encryption.kms.client.class`
64+
**Description:** Class implementing the KmsClient interface. "KMS" stands for “key management service”. The Client will interact with a KMS Server to wrap/unrwap encryption keys.
65+
**Default value:** None
66+
67+
---
68+
69+
**Property:** `parquet.encryption.kms.instance.id`
70+
**Description:** ID of the KMS instance that will be used for encryption (if multiple KMS instances are available).
71+
**Default value:** `DEFAULT`
72+
73+
---
74+
75+
**Property:** `parquet.encryption.kms.instance.url`
76+
**Description:** URL of the KMS instance.
77+
**Default value:** `DEFAULT`
78+
79+
---
80+
81+
**Property:** `parquet.encryption.key.access.token`
82+
**Description:** Authorization token that will be passed to KMS.
83+
**Default value:** None
84+
85+
---
86+
87+
**Property:** `parquet.encryption.double.wrapping`
88+
**Description:** Use double wrapping - where data encryption keys (DEKs) are encrypted with key encryption keys (KEKs), which in turn are encrypted with master keys.
89+
If `false`, DEKs are directly encrypted with master keys, KEKs are not used.
90+
**Default value:** `true`
91+
92+
---
93+
94+
**Property:** `parquet.encryption.cache.lifetime.seconds`
95+
**Description:** Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client objects).
96+
**Default value:** `600` (10 minutes)
97+
98+
---
99+
100+
**Property:** `parquet.encryption.wrap.locally`
101+
**Description:** Wrap keys locally - master keys are fetched from the KMS server and used to encrypt other keys (DEKs or KEKs).
102+
If `false` - key wrapping will be performed by a KMS server.
103+
**Default value:** `false`
104+
105+
---
106+
107+
**Property:** `parquet.encryption.key.material.store.internally`
108+
**Description:** Store key material inside Parquet file footers; this mode doesn’t produce additional files.
109+
If `false`, key material is stored in separate new files, created in the same folder - this mode enables key rotation for immutable Parquet files.
110+
**Default value:** `true`
111+
112+
---
113+
114+
**Property:** `parquet.encryption.data.key.length.bits`
115+
**Description:** Length of data encryption keys (DEKs), randomly generated by parquet key management tools. Can be 128, 192 or 256 bits.
116+
**Default value:** `128`
117+
118+
---
119+

parquet-hadoop/pom.xml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -126,6 +126,7 @@
126126
<groupId>com.squareup.okhttp3</groupId>
127127
<artifactId>okhttp</artifactId>
128128
<version>4.6.0</version>
129+
<scope>test</scope>
129130
</dependency>
130131
</dependencies>
131132

parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileDecryptor.java

Lines changed: 20 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,14 @@
2222
import org.apache.parquet.format.BlockCipher;
2323
import org.apache.parquet.format.EncryptionAlgorithm;
2424
import org.apache.parquet.hadoop.metadata.ColumnPath;
25+
import org.slf4j.Logger;
26+
import org.slf4j.LoggerFactory;
27+
2528
import java.util.Arrays;
2629
import java.util.HashMap;
2730

2831
public class InternalFileDecryptor {
32+
private static final Logger LOG = LoggerFactory.getLogger(InternalFileDecryptor.class);
2933

3034
private final FileDecryptionProperties fileDecryptionProperties;
3135
private final DecryptionKeyRetriever keyRetriever;
@@ -206,6 +210,10 @@ public void setFileCryptoMetaData(EncryptionAlgorithm algorithm,
206210
throw new ParquetCryptoRuntimeException("Decryptor re-use: Different footer key metadata");
207211
}
208212
}
213+
214+
if (LOG.isDebugEnabled()) {
215+
LOG.debug("File Decryptor. Algo: {}. Encrypted footer: {}", algorithm, encryptedFooter);
216+
}
209217
}
210218

211219
public InternalColumnDecryptionSetup setColumnCryptoMetadata(ColumnPath path, boolean encrypted,
@@ -240,8 +248,11 @@ public InternalColumnDecryptionSetup setColumnCryptoMetadata(ColumnPath path, bo
240248
}
241249
columnDecryptionSetup = new InternalColumnDecryptionSetup(path, true, true,
242250
getDataModuleDecryptor(null), getThriftModuleDecryptor(null), columnOrdinal, null);
243-
} else {
244-
// Column is encrypted with column-specific key
251+
252+
if (LOG.isDebugEnabled()) {
253+
LOG.debug("Column decryption (footer key): {}", path);
254+
}
255+
} else { // Column is encrypted with column-specific key
245256
byte[] columnKeyBytes = fileDecryptionProperties.getColumnKey(path);
246257
if ((null == columnKeyBytes) && (null != keyMetadata) && (null != keyRetriever)) {
247258
// No explicit column key given via API. Retrieve via key metadata.
@@ -251,12 +262,14 @@ public InternalColumnDecryptionSetup setColumnCryptoMetadata(ColumnPath path, bo
251262
throw new KeyAccessDeniedException("Column " + path + ": key access denied", e);
252263
}
253264
}
254-
255-
if (null == columnKeyBytes) { // Hidden column: encrypted, but key unavailable
256-
throw new ParquetCryptoRuntimeException("Column " + path + ": key unavailable");
257-
} else { // Key is available
258-
columnDecryptionSetup = new InternalColumnDecryptionSetup(path, true, false,
265+
if (null == columnKeyBytes) {
266+
throw new ParquetCryptoRuntimeException("Column " + path + "is encrypted with NULL column key");
267+
}
268+
columnDecryptionSetup = new InternalColumnDecryptionSetup(path, true, false,
259269
getDataModuleDecryptor(columnKeyBytes), getThriftModuleDecryptor(columnKeyBytes), columnOrdinal, keyMetadata);
270+
271+
if (LOG.isDebugEnabled()) {
272+
LOG.debug("Column decryption (column key): {}", path);
260273
}
261274
}
262275
}
@@ -300,4 +313,3 @@ public FileDecryptionProperties getDecryptionProperties() {
300313
return fileDecryptionProperties;
301314
}
302315
}
303-

parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileEncryptor.java

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,11 +22,15 @@
2222
import org.apache.parquet.format.BlockCipher;
2323
import org.apache.parquet.format.FileCryptoMetaData;
2424
import org.apache.parquet.hadoop.metadata.ColumnPath;
25+
import org.slf4j.Logger;
26+
import org.slf4j.LoggerFactory;
2527
import org.apache.parquet.format.EncryptionAlgorithm;
2628

2729
import java.util.HashMap;
30+
import java.util.Map;
2831

2932
public class InternalFileEncryptor {
33+
private static final Logger LOG = LoggerFactory.getLogger(InternalFileEncryptor.class);
3034

3135
private final EncryptionAlgorithm algorithm;
3236
private final FileEncryptionProperties fileEncryptionProperties;
@@ -42,6 +46,9 @@ public class InternalFileEncryptor {
4246

4347
public InternalFileEncryptor(FileEncryptionProperties fileEncryptionProperties) {
4448
this.fileEncryptionProperties = fileEncryptionProperties;
49+
if (LOG.isDebugEnabled()) {
50+
fileEncryptorLog();
51+
}
4552
algorithm = fileEncryptionProperties.getAlgorithm();
4653
footerKey = fileEncryptionProperties.getFooterKey();
4754
encryptFooter = fileEncryptionProperties.encryptedFooter();
@@ -171,4 +178,19 @@ public AesGcmEncryptor getSignedFooterEncryptor() {
171178
}
172179
return (AesGcmEncryptor) ModuleCipherFactory.getEncryptor(AesMode.GCM, footerKey);
173180
}
181+
182+
private void fileEncryptorLog() {
183+
String encryptedColumnList;
184+
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = fileEncryptionProperties.getEncryptedColumns();
185+
if (null != columnPropertyMap) {
186+
encryptedColumnList = "";
187+
for (Map.Entry<ColumnPath, ColumnEncryptionProperties> entry : columnPropertyMap.entrySet()) {
188+
encryptedColumnList += entry.getKey() + "; ";
189+
}
190+
} else {
191+
encryptedColumnList = "Every column will be encrypted with footer key.";
192+
}
193+
LOG.debug("File Encryptor. Algo: {}. Encrypted footer: {}. Encrypted columns: {}",
194+
fileEncryptionProperties.getAlgorithm(), fileEncryptionProperties.encryptedFooter(), encryptedColumnList);
195+
}
174196
}

parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/ExpiringCacheEntry.java

Lines changed: 0 additions & 46 deletions
This file was deleted.

parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyMaterialStore.java

Lines changed: 32 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,17 +26,47 @@
2626

2727
public interface FileKeyMaterialStore {
2828

29+
/**
30+
* Initializes key material store for a parquet file.
31+
* @param parquetFilePath Parquet file path
32+
* @param hadoopConfig Hadoop configuration
33+
* @param tempStore set true if this is a temporary store, used in key rotation
34+
*/
2935
public void initialize(Path parquetFilePath, Configuration hadoopConfig, boolean tempStore);
3036

37+
/**
38+
* Add key material for one encryption key.
39+
* @param keyIDInFile ID of the key in Parquet file
40+
* @param keyMaterial key material
41+
*/
3142
public void addKeyMaterial(String keyIDInFile, String keyMaterial);
43+
44+
/**
45+
* After key material was added for all keys in the given Parquet file,
46+
* save material in persistent store.
47+
*/
48+
public void saveMaterial();
3249

50+
/**
51+
* Get key material
52+
* @param keyIDInFile ID of a key in Parquet file
53+
* @return key material
54+
*/
3355
public String getKeyMaterial(String keyIDInFile);
3456

35-
public void saveMaterial();
36-
57+
/**
58+
* @return Set of all key IDs in this store (for the given Parquet file)
59+
*/
3760
public Set<String> getKeyIDSet();
3861

62+
/**
63+
* Remove key material from persistent store. Used in key rotation.
64+
*/
3965
public void removeMaterial();
4066

67+
/**
68+
* Move key material to another store. Used in key rotation.
69+
* @param targetKeyMaterialStore target store
70+
*/
4171
public void moveMaterialTo(FileKeyMaterialStore targetKeyMaterialStore);
4272
}

0 commit comments

Comments
 (0)