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



##########
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:
       Is it correct that `isInternalStorage` is serialized but not 
deserialized in this object?

##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java
##########
@@ -0,0 +1,166 @@
+/*

Review comment:
       jackson can do much more. With the correct annotations it can map a java 
object automatically. Feel free to use the following example here and at the 
other similar objects.
   ```java
   @JsonInclude(Include.NON_NULL)
   @JsonIgnoreProperties(ignoreUnknown = true)
   @JsonAutoDetect(fieldVisibility = Visibility.ANY)
   public class KeyMaterial {
     static final String KEY_MATERIAL_TYPE = "PKMT1";
   
     private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
   
     static KeyMaterial parse(String keyMaterialString) {
       try {
         KeyMaterial keyMaterial = OBJECT_MAPPER.readValue(keyMaterialString, 
KeyMaterial.class);
         keyMaterial.validate();
         return keyMaterial;
       } catch (IOException e) {
         throw new ParquetCryptoRuntimeException("Failed to parse key metadata 
" + keyMaterialString, e);
       }
     }
   
     @JsonProperty("doubleWrapping")
     private boolean doubleWrapped;
     @JsonProperty("wrappedDEK")
     private String encodedWrappedDEK;
     @JsonProperty("wrappedKEK")
     private String encodedWrappedKEK;
     @JsonProperty("isFooterKey")
     private boolean footerKey;
     private boolean internalStorage;
     @JsonProperty("keyEncryptionKeyID")
     private String kekID;
     private String keyMaterialType;
     private String kmsInstanceID;
     private String kmsInstanceURL;
     private String masterKeyID;
   
     private KeyMaterial() {
     }
   
     KeyMaterial(boolean footerKey,
         String kmsInstanceID,
         String kmsInstanceURL,
         String masterKeyID,
         boolean doubleWrapped,
         String kekID,
         String encodedWrappedKEK,
         String encodedWrappedDEK,
         boolean internalStorage,
         String keyMaterialType) {
       this.footerKey = footerKey;
       this.kmsInstanceID = kmsInstanceID;
       this.kmsInstanceURL = kmsInstanceURL;
       this.masterKeyID = masterKeyID;
       this.doubleWrapped = doubleWrapped;
       this.kekID = kekID;
       this.encodedWrappedKEK = encodedWrappedKEK;
       this.encodedWrappedDEK = encodedWrappedDEK;
       this.internalStorage = internalStorage;
       this.keyMaterialType = keyMaterialType;
     }
   
     String getKekID() {
       return kekID;
     }
   
     String getKeyMaterialType() {
       return KEY_MATERIAL_TYPE;
     }
   
     String getKmsInstanceID() {
       return kmsInstanceID;
     }
   
     String getKmsInstanceURL() {
       return kmsInstanceURL;
     }
   
     String getMasterKeyID() {
       return masterKeyID;
     }
   
     String getWrappedDEK() {
       return encodedWrappedDEK;
     }
   
     String getWrappedKEK() {
       return encodedWrappedKEK;
     }
   
     boolean isDoubleWrapped() {
       return doubleWrapped;
     }
   
     boolean isFooterKey() {
       return footerKey;
     }
   
     boolean isInternalStorage() {
       return internalStorage;
     }
   
     String serialize() {
       try {
         validate();
         return OBJECT_MAPPER.writeValueAsString(this);
       } catch (IOException e) {
         throw new ParquetCryptoRuntimeException("Failed to serialize key 
material", e);
       }
     }
   
     private void validate() {
       if (!KEY_MATERIAL_TYPE.equals(keyMaterialType)) {
         throw new ParquetCryptoRuntimeException("Wrong key material type: " + 
keyMaterialType +
             " vs " + KEY_MATERIAL_TYPE);
       }
       if (footerKey && (kmsInstanceID == null || kmsInstanceURL == null)) {
         // ...
       }
       if (doubleWrapped && (kekID == null || encodedWrappedKEK == null)) {
         // ...
       }
       // ... ?
     }
   }
   ```
   
   I think, the format of these json objects is important for compatibility. We 
shall specify them or at least give an example in the comments.

##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/PropertiesDrivenCryptoFactory.java
##########
@@ -36,38 +39,62 @@
 import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
 import org.apache.parquet.hadoop.api.WriteSupport.WriteContext;
 import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.apache.parquet.crypto.keytools.KeyToolkit.stringIsEmpty;
 
 public class PropertiesDrivenCryptoFactory implements 
EncryptionPropertiesFactory, DecryptionPropertiesFactory {
-
-  public static final String COLUMN_KEYS_PROPERTY_NAME = 
"encryption.column.keys";
-  public static final String FOOTER_KEY_PROPERTY_NAME = 
"encryption.footer.key";
-  public static final String ENCRYPTION_ALGORITHM_PROPERTY_NAME = 
"encryption.algorithm";
-  public static final String PLAINTEXT_FOOTER_PROPERTY_NAME = 
"encryption.plaintext.footer";
+  private static final Logger LOG = 
LoggerFactory.getLogger(PropertiesDrivenCryptoFactory.class);
   
-  public static final int DEK_LENGTH = 16;
-
-  private static final SecureRandom random = new SecureRandom();
+  private static final Integer[] ACCEPTABLE_DATA_KEY_LENGTHS = {128, 192, 256};
+  private static final Set<Integer> ACCEPTABLE_DATA_KEY_LENGTHS_SET =
+    new HashSet<>(Arrays.asList(ACCEPTABLE_DATA_KEY_LENGTHS));

Review comment:
       Seems a bit overkill to check if a number equals to one of three. I 
would use a simple `int[]` with sorted values and use `Arrays.binarySearch` to 
check if a value is in the array or not.




----------------------------------------------------------------
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


Reply via email to