|
28 | 28 | import org.codehaus.jackson.map.ObjectMapper; |
29 | 29 | import org.codehaus.jackson.type.TypeReference; |
30 | 30 |
|
| 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 | + */ |
31 | 53 | public class KeyMaterial { |
32 | 54 | 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"; |
35 | 56 |
|
36 | 57 | static final String FOOTER_KEY_ID_IN_FILE = "footerKey"; |
37 | 58 | static final String COLUMN_KEY_ID_IN_FILE_PREFIX = "columnKey"; |
38 | | - |
| 59 | + |
39 | 60 | private static final String IS_FOOTER_KEY_FIELD = "isFooterKey"; |
40 | 61 | private static final String DOUBLE_WRAPPING_FIELD = "doubleWrapping"; |
41 | 62 | private static final String KMS_INSTANCE_ID_FIELD = "kmsInstanceID"; |
@@ -68,60 +89,83 @@ private KeyMaterial(boolean isFooterKey, String kmsInstanceID, String kmsInstanc |
68 | 89 | this.encodedWrappedDEK = encodedWrappedDEK; |
69 | 90 | } |
70 | 91 |
|
| 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 |
71 | 112 | static KeyMaterial parse(Map<String, String> keyMaterialJson) { |
| 113 | + // 2. Check if "key material" belongs to file footer key |
72 | 114 | boolean isFooterKey = Boolean.valueOf(keyMaterialJson.get(IS_FOOTER_KEY_FIELD)); |
73 | 115 | String kmsInstanceID = null; |
74 | 116 | String kmsInstanceURL = null; |
75 | 117 | if (isFooterKey) { |
| 118 | + // 3. For footer key, extract KMS Instance ID |
76 | 119 | kmsInstanceID = keyMaterialJson.get(KMS_INSTANCE_ID_FIELD); |
| 120 | + // 4. For footer key, extract KMS Instance URL |
77 | 121 | kmsInstanceURL = keyMaterialJson.get(KMS_INSTANCE_URL_FIELD); |
78 | 122 | } |
79 | | - boolean isDoubleWrapped = Boolean.valueOf(keyMaterialJson.get(DOUBLE_WRAPPING_FIELD)); |
| 123 | + // 5. Extract master key ID |
80 | 124 | String masterKeyID = keyMaterialJson.get(MASTER_KEY_ID_FIELD); |
| 125 | + // 6. Extract wrapped DEK |
81 | 126 | String encodedWrappedDEK = keyMaterialJson.get(WRAPPED_DEK_FIELD); |
82 | 127 | String kekID = null; |
83 | 128 | 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)); |
84 | 131 | if (isDoubleWrapped) { |
| 132 | + // 8. In double wrapping mode, extract KEK ID |
85 | 133 | kekID = keyMaterialJson.get(KEK_ID_FIELD); |
| 134 | + // 9. In double wrapping mode, extract wrapped KEK |
86 | 135 | encodedWrappedKEK = keyMaterialJson.get(WRAPPED_KEK_FIELD); |
87 | 136 | } |
88 | 137 |
|
89 | 138 | return new KeyMaterial(isFooterKey, kmsInstanceID, kmsInstanceURL, masterKeyID, isDoubleWrapped, kekID, encodedWrappedKEK, encodedWrappedDEK); |
90 | 139 | } |
91 | 140 |
|
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 | | - |
108 | 141 | static String createSerialized(boolean isFooterKey, String kmsInstanceID, String kmsInstanceURL, String masterKeyID, |
109 | 142 | boolean isDoubleWrapped, String kekID, String encodedWrappedKEK, String encodedWrappedDEK, boolean isInternalStorage) { |
110 | 143 | 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); |
112 | 146 | 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"); |
114 | 150 | } |
| 151 | + // 2. Write isFooterKey |
115 | 152 | keyMaterialMap.put(IS_FOOTER_KEY_FIELD, Boolean.toString(isFooterKey)); |
116 | 153 | if (isFooterKey) { |
| 154 | + // 3. For footer key, write KMS Instance ID |
117 | 155 | keyMaterialMap.put(KMS_INSTANCE_ID_FIELD, kmsInstanceID); |
| 156 | + // 4. For footer key, write KMS Instance URL |
118 | 157 | keyMaterialMap.put(KMS_INSTANCE_URL_FIELD, kmsInstanceURL); |
119 | 158 | } |
| 159 | + // 5. Write master key ID |
120 | 160 | keyMaterialMap.put(MASTER_KEY_ID_FIELD, masterKeyID); |
| 161 | + // 6. Write wrapped DEK |
121 | 162 | keyMaterialMap.put(WRAPPED_DEK_FIELD, encodedWrappedDEK); |
| 163 | + // 7. Write isDoubleWrapped |
122 | 164 | keyMaterialMap.put(DOUBLE_WRAPPING_FIELD, Boolean.toString(isDoubleWrapped)); |
123 | 165 | if (isDoubleWrapped) { |
| 166 | + // 8. In double wrapping mode, write KEK ID |
124 | 167 | keyMaterialMap.put(KEK_ID_FIELD, kekID); |
| 168 | + // 9. In double wrapping mode, write wrapped KEK |
125 | 169 | keyMaterialMap.put(WRAPPED_KEK_FIELD, encodedWrappedKEK); |
126 | 170 | } |
127 | 171 |
|
|
0 commit comments