Skip to content

Commit 0355f65

Browse files
committed
key material documentation
1 parent 055bb39 commit 0355f65

File tree

4 files changed

+116
-35
lines changed

4 files changed

+116
-35
lines changed

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

Lines changed: 66 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -28,14 +28,35 @@
2828
import org.codehaus.jackson.map.ObjectMapper;
2929
import org.codehaus.jackson.type.TypeReference;
3030

31+
/**
32+
* KeyMaterial class represents the "key material", keeping the information that allows readers to recover an encryption key (see
33+
* description of the KeyMetadata class). The keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
34+
* "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key material is generated by encrypting the
35+
* "data encryption key" (DEK) by a "master key". In the double wrapping mode, the key material is generated by encrypting the DEK
36+
* by a "key encryption key" (KEK), that in turn is encrypted by a "master key".
37+
*
38+
* Key material is kept in a flat json object, with the following fields:
39+
* 1. "keyMaterialType" - a String, with the type of key material. In the current version, only one value is allowed - "PKMT1" (stands
40+
* for "parquet key management tools, version 1"). For external key material storage, this field is written in both "key metadata" and
41+
* "key material" jsons. For internal key material storage, this field is written only once in the common json.
42+
* 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer key, and keeps additional information (such as
43+
* KMS instance ID and URL). If false, means that the material belongs to a column key.
44+
* 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key material.
45+
* 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key material.
46+
* 5. "masterKeyID" - a String, with the ID of the master key used to generate the material.
47+
* 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
48+
* 7. "doubleWrapping" - a boolean. If true, means that the material was generated in double wrapping mode.
49+
* If false - in single wrapping mode.
50+
* 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the material. Written only in double wrapping mode.
51+
* 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in double wrapping mode.
52+
*/
3153
public class KeyMaterial {
3254
static final String KEY_MATERIAL_TYPE_FIELD = "keyMaterialType";
33-
static final String KEY_MATERIAL_TYPE = "PKMT1";
34-
static final String KEY_MATERIAL_INTERNAL_STORAGE_FIELD = "internalStorage";
55+
static final String KEY_MATERIAL_TYPE1 = "PKMT1";
3556

3657
static final String FOOTER_KEY_ID_IN_FILE = "footerKey";
3758
static final String COLUMN_KEY_ID_IN_FILE_PREFIX = "columnKey";
38-
59+
3960
private static final String IS_FOOTER_KEY_FIELD = "isFooterKey";
4061
private static final String DOUBLE_WRAPPING_FIELD = "doubleWrapping";
4162
private static final String KMS_INSTANCE_ID_FIELD = "kmsInstanceID";
@@ -68,60 +89,83 @@ private KeyMaterial(boolean isFooterKey, String kmsInstanceID, String kmsInstanc
6889
this.encodedWrappedDEK = encodedWrappedDEK;
6990
}
7091

92+
// parses external key material
93+
static KeyMaterial parse(String keyMaterialString) {
94+
Map<String, String> keyMaterialJson = null;
95+
try {
96+
keyMaterialJson = OBJECT_MAPPER.readValue(new StringReader(keyMaterialString),
97+
new TypeReference<Map<String, String>>() {});
98+
} catch (IOException e) {
99+
throw new ParquetCryptoRuntimeException("Failed to parse key metadata " + keyMaterialString, e);
100+
}
101+
// 1. External key material - extract "key material type", and make sure it is supported
102+
String keyMaterialType = keyMaterialJson.get(KEY_MATERIAL_TYPE_FIELD);
103+
if (!KEY_MATERIAL_TYPE1.equals(keyMaterialType)) {
104+
throw new ParquetCryptoRuntimeException("Wrong key material type: " + keyMaterialType +
105+
" vs " + KEY_MATERIAL_TYPE1);
106+
}
107+
// Parse other fields (common to internal and external key material)
108+
return parse(keyMaterialJson);
109+
}
110+
111+
// parses fields common to internal and external key material
71112
static KeyMaterial parse(Map<String, String> keyMaterialJson) {
113+
// 2. Check if "key material" belongs to file footer key
72114
boolean isFooterKey = Boolean.valueOf(keyMaterialJson.get(IS_FOOTER_KEY_FIELD));
73115
String kmsInstanceID = null;
74116
String kmsInstanceURL = null;
75117
if (isFooterKey) {
118+
// 3. For footer key, extract KMS Instance ID
76119
kmsInstanceID = keyMaterialJson.get(KMS_INSTANCE_ID_FIELD);
120+
// 4. For footer key, extract KMS Instance URL
77121
kmsInstanceURL = keyMaterialJson.get(KMS_INSTANCE_URL_FIELD);
78122
}
79-
boolean isDoubleWrapped = Boolean.valueOf(keyMaterialJson.get(DOUBLE_WRAPPING_FIELD));
123+
// 5. Extract master key ID
80124
String masterKeyID = keyMaterialJson.get(MASTER_KEY_ID_FIELD);
125+
// 6. Extract wrapped DEK
81126
String encodedWrappedDEK = keyMaterialJson.get(WRAPPED_DEK_FIELD);
82127
String kekID = null;
83128
String encodedWrappedKEK = null;
129+
// 7. Check if "key material" was generated in double wrapping mode
130+
boolean isDoubleWrapped = Boolean.valueOf(keyMaterialJson.get(DOUBLE_WRAPPING_FIELD));
84131
if (isDoubleWrapped) {
132+
// 8. In double wrapping mode, extract KEK ID
85133
kekID = keyMaterialJson.get(KEK_ID_FIELD);
134+
// 9. In double wrapping mode, extract wrapped KEK
86135
encodedWrappedKEK = keyMaterialJson.get(WRAPPED_KEK_FIELD);
87136
}
88137

89138
return new KeyMaterial(isFooterKey, kmsInstanceID, kmsInstanceURL, masterKeyID, isDoubleWrapped, kekID, encodedWrappedKEK, encodedWrappedDEK);
90139
}
91140

92-
static KeyMaterial parse(String keyMaterialString) {
93-
Map<String, String> keyMaterialJson = null;
94-
try {
95-
keyMaterialJson = OBJECT_MAPPER.readValue(new StringReader(keyMaterialString),
96-
new TypeReference<Map<String, String>>() {});
97-
} catch (IOException e) {
98-
throw new ParquetCryptoRuntimeException("Failed to parse key metadata " + keyMaterialString, e);
99-
}
100-
String keyMaterialType = keyMaterialJson.get(KEY_MATERIAL_TYPE_FIELD);
101-
if (!KEY_MATERIAL_TYPE.equals(keyMaterialType)) {
102-
throw new ParquetCryptoRuntimeException("Wrong key material type: " + keyMaterialType +
103-
" vs " + KEY_MATERIAL_TYPE);
104-
}
105-
return parse(keyMaterialJson);
106-
}
107-
108141
static String createSerialized(boolean isFooterKey, String kmsInstanceID, String kmsInstanceURL, String masterKeyID,
109142
boolean isDoubleWrapped, String kekID, String encodedWrappedKEK, String encodedWrappedDEK, boolean isInternalStorage) {
110143
Map<String, String> keyMaterialMap = new HashMap<String, String>(10);
111-
keyMaterialMap.put(KEY_MATERIAL_TYPE_FIELD, KEY_MATERIAL_TYPE);
144+
// 1. Write "key material type"
145+
keyMaterialMap.put(KEY_MATERIAL_TYPE_FIELD, KEY_MATERIAL_TYPE1);
112146
if (isInternalStorage) {
113-
keyMaterialMap.put(KEY_MATERIAL_INTERNAL_STORAGE_FIELD, "true");
147+
// for internal storage, key material and key metadata are the same.
148+
// adding the "internalStorage" field that belongs to KeyMetadata.
149+
keyMaterialMap.put(KeyMetadata.KEY_MATERIAL_INTERNAL_STORAGE_FIELD, "true");
114150
}
151+
// 2. Write isFooterKey
115152
keyMaterialMap.put(IS_FOOTER_KEY_FIELD, Boolean.toString(isFooterKey));
116153
if (isFooterKey) {
154+
// 3. For footer key, write KMS Instance ID
117155
keyMaterialMap.put(KMS_INSTANCE_ID_FIELD, kmsInstanceID);
156+
// 4. For footer key, write KMS Instance URL
118157
keyMaterialMap.put(KMS_INSTANCE_URL_FIELD, kmsInstanceURL);
119158
}
159+
// 5. Write master key ID
120160
keyMaterialMap.put(MASTER_KEY_ID_FIELD, masterKeyID);
161+
// 6. Write wrapped DEK
121162
keyMaterialMap.put(WRAPPED_DEK_FIELD, encodedWrappedDEK);
163+
// 7. Write isDoubleWrapped
122164
keyMaterialMap.put(DOUBLE_WRAPPING_FIELD, Boolean.toString(isDoubleWrapped));
123165
if (isDoubleWrapped) {
166+
// 8. In double wrapping mode, write KEK ID
124167
keyMaterialMap.put(KEK_ID_FIELD, kekID);
168+
// 9. In double wrapping mode, write wrapped KEK
125169
keyMaterialMap.put(WRAPPED_KEK_FIELD, encodedWrappedKEK);
126170
}
127171

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

Lines changed: 35 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,27 @@
2929
import org.codehaus.jackson.map.ObjectMapper;
3030
import org.codehaus.jackson.type.TypeReference;
3131

32+
/**
33+
* Parquet encryption specification defines "key metadata" as an arbitrary byte array, generated by file writers for each encryption key,
34+
* and passed to the low level API for storage in the file footer . The "key metadata" field is made available to file readers to enable
35+
* recovery of the key. This simple interface can be utilized for implementation of any key management scheme.
36+
*
37+
* The keytools package (PARQUET-1373) implements one approach, of many possible, to key management and to generation of the "key metadata"
38+
* fields. This approach, based on the "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual material,
39+
* required to recover a key, in a "key material" object (see the KeyMaterial class for details).
40+
*
41+
* KeyMetadata class writes (and reads) the "key metadata" field as a flat json object, with the following fields:
42+
* 1. "keyMaterialType" - a String, with the type of key material. In the current version, only one value is allowed - "PKMT1" (stands
43+
* for "parquet key management tools, version 1")
44+
* 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the "key metadata" field. If false, "key material"
45+
* is kept externally (outside Parquet files) - in this case, "key metadata" keeps a reference to the external "key material".
46+
* 3. "keyReference" - a String, with the reference to the external "key material". Written only if internalStorage is false.
47+
*
48+
* If internalStorage is true, "key material" is a part of "key metadata", and the json keeps additional fields, described in the
49+
* KeyMaterial class.
50+
*/
3251
public class KeyMetadata {
52+
static final String KEY_MATERIAL_INTERNAL_STORAGE_FIELD = "internalStorage";
3353
private static final String KEY_REFERENCE_FIELD = "keyReference";
3454

3555
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
@@ -54,30 +74,39 @@ static KeyMetadata parse(byte[] keyMetadataBytes) {
5474
throw new ParquetCryptoRuntimeException("Failed to parse key metadata " + keyMetaDataString, e);
5575
}
5676

77+
// 1. Extract "key material type", and make sure it is supported
5778
String keyMaterialType = keyMetadataJson.get(KeyMaterial.KEY_MATERIAL_TYPE_FIELD);
58-
if (!KeyMaterial.KEY_MATERIAL_TYPE.equals(keyMaterialType)) {
79+
if (!KeyMaterial.KEY_MATERIAL_TYPE1.equals(keyMaterialType)) {
5980
throw new ParquetCryptoRuntimeException("Wrong key material type: " + keyMaterialType +
60-
" vs " + KeyMaterial.KEY_MATERIAL_TYPE);
81+
" vs " + KeyMaterial.KEY_MATERIAL_TYPE1);
6182
}
6283

63-
boolean isInternalStorage = Boolean.parseBoolean(keyMetadataJson.get(KeyMaterial.KEY_MATERIAL_INTERNAL_STORAGE_FIELD));
84+
// 2. Check if "key material" is stored internally in Parquet file key metadata, or is stored externally
85+
boolean isInternalStorage = Boolean.parseBoolean(keyMetadataJson.get(KEY_MATERIAL_INTERNAL_STORAGE_FIELD));
6486
String keyReference;
6587
KeyMaterial keyMaterial;
6688

6789
if (isInternalStorage) {
68-
keyReference = null;
90+
// 3.1 "key material" is stored internally, inside "key metadata" - parse it
6991
keyMaterial = KeyMaterial.parse(keyMetadataJson);
92+
keyReference = null;
7093
} else {
94+
// 3.2 "key material" is stored externally. "key metadata" keeps a reference to it
7195
keyReference = keyMetadataJson.get(KEY_REFERENCE_FIELD);
7296
keyMaterial = null;
7397
}
98+
7499
return new KeyMetadata(isInternalStorage, keyReference, keyMaterial);
75100
}
76101

102+
// For external material only. For internal material, create serialized KeyMaterial directly
77103
static String createSerializedForExternalMaterial(String keyReference) {
78104
Map<String, String> keyMetadataMap = new HashMap<String, String>(3);
79-
keyMetadataMap.put(KeyMaterial.KEY_MATERIAL_TYPE_FIELD, KeyMaterial.KEY_MATERIAL_TYPE);
80-
keyMetadataMap.put(KeyMaterial.KEY_MATERIAL_INTERNAL_STORAGE_FIELD, "false");
105+
// 1. Write "key material type"
106+
keyMetadataMap.put(KeyMaterial.KEY_MATERIAL_TYPE_FIELD, KeyMaterial.KEY_MATERIAL_TYPE1);
107+
// 2. Write internal storage as false
108+
keyMetadataMap.put(KEY_MATERIAL_INTERNAL_STORAGE_FIELD, "false");
109+
// 3. For externally stored "key material", "key metadata" keeps only a reference to it
81110
keyMetadataMap.put(KEY_REFERENCE_FIELD, keyReference);
82111

83112
try {

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

Lines changed: 3 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,7 @@
2323
import java.security.SecureRandom;
2424
import java.util.Arrays;
2525
import java.util.HashMap;
26-
import java.util.HashSet;
2726
import java.util.Map;
28-
import java.util.Set;
2927

3028
import org.apache.hadoop.conf.Configuration;
3129
import org.apache.hadoop.fs.Path;
@@ -46,10 +44,8 @@
4644

4745
public class PropertiesDrivenCryptoFactory implements EncryptionPropertiesFactory, DecryptionPropertiesFactory {
4846
private static final Logger LOG = LoggerFactory.getLogger(PropertiesDrivenCryptoFactory.class);
49-
50-
private static final Integer[] ACCEPTABLE_DATA_KEY_LENGTHS = {128, 192, 256};
51-
private static final Set<Integer> ACCEPTABLE_DATA_KEY_LENGTHS_SET =
52-
new HashSet<>(Arrays.asList(ACCEPTABLE_DATA_KEY_LENGTHS));
47+
48+
private static final int[] ACCEPTABLE_DATA_KEY_LENGTHS = {128, 192, 256};
5349

5450
/**
5551
* List of columns to encrypt, with master key IDs (see HIVE-21848).
@@ -117,7 +113,7 @@ public FileEncryptionProperties getFileEncryptionProperties(Configuration fileHa
117113
int dekLengthBits = fileHadoopConfig.getInt(KeyToolkit.DATA_KEY_LENGTH_PROPERTY_NAME,
118114
KeyToolkit.DATA_KEY_LENGTH_DEFAULT);
119115

120-
if (!ACCEPTABLE_DATA_KEY_LENGTHS_SET.contains(dekLengthBits)) {
116+
if (Arrays.binarySearch(ACCEPTABLE_DATA_KEY_LENGTHS, dekLengthBits) < 0) {
121117
throw new ParquetCryptoRuntimeException("Wrong data key length : " + dekLengthBits);
122118
}
123119

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

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,18 @@ public abstract class RemoteKmsClient implements KmsClient {
4949
// MasterKey cache: master keys per key ID (per KMS Client). For local wrapping only.
5050
private ConcurrentMap<String, byte[]> masterKeyCache;
5151

52+
/**
53+
* KMS systems wrap keys by encrypting them by master keys, and attaching additional information (such as the version
54+
* number of the masker key) to the result of encryption. The master key version is required in key rotation.
55+
* Currently, the local wrapping mode does not support key rotation (because not all KMS systems allow to fetch a master
56+
* key by its ID and version number). Still, the local wrapping mode adds a placeholder for the master key version, that will
57+
* enable support for key rotation in this mode in the future, with appropriate KMS systems. This will also enable backward
58+
* compatibility, where future readers will be able to extract master key version in the files written by the current code.
59+
*
60+
* LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following fields:
61+
* 1. "masterKeyVersion" - a String, with the master key version. In the current version, only one value is allowed - "NO_VERSION".
62+
* 2. "encryptedKey" - a String, with the key encrypted by the master key (base64-encoded).
63+
*/
5264
static class LocalKeyWrap {
5365
public static final String LOCAL_WRAP_KEY_VERSION_FIELD = "masterKeyVersion";
5466
public static final String LOCAL_WRAP_ENCRYPTED_KEY_FIELD = "encryptedKey";

0 commit comments

Comments
 (0)