[ 
https://issues.apache.org/jira/browse/PARQUET-1373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17148434#comment-17148434
 ] 

ASF GitHub Bot commented on PARQUET-1373:
-----------------------------------------

ggershinsky commented on a change in pull request #615:
URL: https://github.com/apache/parquet-mr/pull/615#discussion_r447508660



##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.crypto.keytools;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+public class KeyMaterial {
+  static final String KEY_MATERIAL_TYPE_FIELD = "keyMaterialType";
+  static final String KEY_MATERIAL_TYPE = "PKMT1";
+  static final String KEY_MATERIAL_INTERNAL_STORAGE_FIELD = "internalStorage";
+
+  static final String FOOTER_KEY_ID_IN_FILE = "footerKey";
+  static final String COLUMN_KEY_ID_IN_FILE_PREFIX = "columnKey";
+  
+  private static final String IS_FOOTER_KEY_FIELD = "isFooterKey";
+  private static final String DOUBLE_WRAPPING_FIELD = "doubleWrapping";
+  private static final String KMS_INSTANCE_ID_FIELD = "kmsInstanceID";
+  private static final String KMS_INSTANCE_URL_FIELD = "kmsInstanceURL";
+  private static final String MASTER_KEY_ID_FIELD = "masterKeyID";
+  private static final String WRAPPED_DEK_FIELD = "wrappedDEK";
+  private static final String KEK_ID_FIELD = "keyEncryptionKeyID";
+  private static final String WRAPPED_KEK_FIELD = "wrappedKEK";
+
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+  private final boolean isFooterKey;
+  private final String kmsInstanceID;
+  private final String kmsInstanceURL;
+  private final String masterKeyID;
+  private final boolean isDoubleWrapped;
+  private final String kekID;
+  private final String encodedWrappedKEK;
+  private final String encodedWrappedDEK;
+
+  private KeyMaterial(boolean isFooterKey, String kmsInstanceID, String 
kmsInstanceURL, String masterKeyID, 
+      boolean isDoubleWrapped, String kekID, String encodedWrappedKEK, String 
encodedWrappedDEK) {
+    this.isFooterKey = isFooterKey;
+    this.kmsInstanceID = kmsInstanceID;
+    this.kmsInstanceURL = kmsInstanceURL;
+    this.masterKeyID = masterKeyID;
+    this.isDoubleWrapped = isDoubleWrapped;
+    this.kekID = kekID;
+    this.encodedWrappedKEK = encodedWrappedKEK;
+    this.encodedWrappedDEK = encodedWrappedDEK;
+  }
+
+  static KeyMaterial parse(Map<String, String> keyMaterialJson) {
+    boolean isFooterKey = 
Boolean.valueOf(keyMaterialJson.get(IS_FOOTER_KEY_FIELD));
+    String kmsInstanceID = null;
+    String kmsInstanceURL = null;
+    if (isFooterKey) {
+      kmsInstanceID = keyMaterialJson.get(KMS_INSTANCE_ID_FIELD);
+      kmsInstanceURL = keyMaterialJson.get(KMS_INSTANCE_URL_FIELD);
+    }
+    boolean isDoubleWrapped = 
Boolean.valueOf(keyMaterialJson.get(DOUBLE_WRAPPING_FIELD));
+    String masterKeyID = keyMaterialJson.get(MASTER_KEY_ID_FIELD);
+    String  encodedWrappedDEK = keyMaterialJson.get(WRAPPED_DEK_FIELD);
+    String kekID = null;
+    String encodedWrappedKEK = null;
+    if (isDoubleWrapped) {
+      kekID = keyMaterialJson.get(KEK_ID_FIELD);
+      encodedWrappedKEK = keyMaterialJson.get(WRAPPED_KEK_FIELD);
+    }
+
+    return new KeyMaterial(isFooterKey, kmsInstanceID, kmsInstanceURL, 
masterKeyID, isDoubleWrapped, kekID, encodedWrappedKEK, encodedWrappedDEK);
+  }
+
+  static KeyMaterial parse(String keyMaterialString) {
+    Map<String, String> keyMaterialJson = null;
+    try {
+      keyMaterialJson = OBJECT_MAPPER.readValue(new 
StringReader(keyMaterialString),
+          new TypeReference<Map<String, String>>() {});
+    } catch (IOException e) {
+      throw new ParquetCryptoRuntimeException("Failed to parse key metadata " 
+ keyMaterialString, e);
+    }
+    String keyMaterialType = keyMaterialJson.get(KEY_MATERIAL_TYPE_FIELD);
+    if (!KEY_MATERIAL_TYPE.equals(keyMaterialType)) {
+      throw new ParquetCryptoRuntimeException("Wrong key material type: " + 
keyMaterialType + 
+          " vs " + KEY_MATERIAL_TYPE);
+    }
+    return parse(keyMaterialJson);
+  }
+
+  static String createSerialized(boolean isFooterKey, String kmsInstanceID, 
String kmsInstanceURL, String masterKeyID, 
+      boolean isDoubleWrapped, String kekID, String encodedWrappedKEK, String 
encodedWrappedDEK, boolean isInternalStorage) {
+    Map<String, String> keyMaterialMap = new HashMap<String, String>(10);
+    keyMaterialMap.put(KEY_MATERIAL_TYPE_FIELD, KEY_MATERIAL_TYPE);
+    if (isInternalStorage) {

Review comment:
       This field is serialized in KeyMaterial objects only for internal 
storage - because in this case, key material and key metadata is the same 
thing. This field is always deserialized in KeyMetadata objects.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Encryption key management tools 
> --------------------------------
>
>                 Key: PARQUET-1373
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1373
>             Project: Parquet
>          Issue Type: New Feature
>          Components: parquet-mr
>            Reporter: Gidon Gershinsky
>            Assignee: Gidon Gershinsky
>            Priority: Major
>
> Parquet Modular Encryption 
> ([PARQUET-1178|https://issues.apache.org/jira/browse/PARQUET-1178]) provides 
> an API that accepts keys, arbitrary key metadata and key retrieval callbacks 
> - which allows to implement basically any key management policy on top of it. 
> This Jira will add tools that implement a set of best practice elements for 
> key management. This is not an end-to-end key management, but rather a set of 
> components that might simplify design and development of an end-to-end 
> solution.
> This tool set is one of many possible. There is no goal to create a single or 
> “standard” toolkit for Parquet encryption keys. Parquet has a Crypto Factory 
> interface [(PARQUET-1817|https://issues.apache.org/jira/browse/PARQUET-1817]) 
> that allows to plug in different implementations of encryption key management.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to