[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-07-01 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java
##
@@ -28,14 +28,35 @@
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 
+/**
+ * KeyMaterial class represents the "key material", keeping the information 
that allows readers to recover an encryption key (see 
+ * description of the KeyMetadata class). The keytools package (PARQUET-1373) 
implements the "envelope encryption" pattern, in a 
+ * "single wrapping" or "double wrapping" mode. In the single wrapping mode, 
the key material is generated by encrypting the 
+ * "data encryption key" (DEK) by a "master key". In the double wrapping mode, 
the key material is generated by encrypting the DEK 
+ * by a "key encryption key" (KEK), that in turn is encrypted by a "master 
key".
+ * 
+ * Key material is kept in a flat json object, with the following fields:
+ * 1. "keyMaterialType" - a String, with the type of  key material. In the 
current version, only one value is allowed - "PKMT1" (stands 
+ * for "parquet key management tools, version 1"). For external key 
material storage, this field is written in both "key metadata" and 
+ * "key material" jsons. For internal key material storage, this field is 
written only once in the common json.
+ * 2. "isFooterKey" - a boolean. If true, means that the material belongs to a 
file footer key, and keeps additional information (such as

Review comment:
   a good point, thanks for noticing this. will be fixed.





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-30 Thread GitBox


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



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

Review comment:
   > .. With the correct annotations it can map a java object automatically.
   
   This approach seems to be optimal for objects with a fixed structure / 
fields - because it searches for all object fields in the json file. In our 
case, many fields are not always written (eg kms instance or url, if the key is 
for a column, and not for the footer; and other examples). Searching for them 
always is an overhead. Moreover, in case of internal storage, we don't need to 
parse two objects - key metadata and key material, because they are the same, 
so parsing one object is sufficient. The code we have today, performs only the 
search/parse of the relevant objects/fields, so it is optimal in that sense - 
and is also well-defined in one place. We can add more comments to the code to 
make the field parsing logic crystal clear. What do you think?
   
   > 
   > the format of these json objects is important for compatibility. We shall 
specify them or at least give an example in the comments.
   
   Sounds good. In addition to the comments mentioned above (that will be added 
to the relevant code lines), we will add a class comment to each relevant 
class, that documents the structure of the corresponding json.
   





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-30 Thread GitBox


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



##
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 ACCEPTABLE_DATA_KEY_LENGTHS_SET =
+new HashSet<>(Arrays.asList(ACCEPTABLE_DATA_KEY_LENGTHS));

Review comment:
   Sure, we'll change this.





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-30 Thread GitBox


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



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

Review comment:
   > .. With the correct annotations it can map a java object automatically.
   
   This approach seems to be optimal for objects with a fixed structure / 
fields - because it searches for all object fields in the json file. In our 
case, many fields are not always written (eg kms instance or url, if the key is 
for a column, and not for the footer; and other examples). Searching for them 
always is an overhead. Moreover, in case of internal storage, we don't need to 
parse two objects - key metadata and key material, because they are the same, 
so parsing one object is sufficient. The code we have today, performs only the 
search/parse of the relevant objects/fields, so it is optimal in that sense - 
and also well-defined in one place. We can add more comments to the code to 
make the field parsing logic crystal clear. What do you think?
   
   > 
   > 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.
   
   Sounds good. In addition to the comments mentioned above (that will be added 
to the relevant code lines), we will add a class comment to each relevant 
class, that documents the structure of the corresponding json.
   





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-30 Thread GitBox


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 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 keyMaterialJson = null;
+try {
+  keyMaterialJson = OBJECT_MAPPER.readValue(new 
StringReader(keyMaterialString),
+  new TypeReference>() {});
+} 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 

[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-15 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java
##
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class HadoopFSKeyMaterialStore implements FileKeyMaterialStore {
+  
+  public final static String KEY_MATERIAL_FILE_PREFIX = "_KEY_MATERIAL_FOR_";
+  public static final String TEMP_FILE_PREFIX = "_TMP";
+  public final static String KEY_MATERIAL_FILE_SUFFFIX = ".json";
+  private static final ObjectMapper objectMapper = new ObjectMapper();
+
+  private FileSystem hadoopFileSystem;
+  private Map keyMaterialMap;
+  private Path keyMaterialFile;
+  
+  HadoopFSKeyMaterialStore(FileSystem hadoopFileSystem) {
+this.hadoopFileSystem = hadoopFileSystem;
+  }
+
+  @Override
+  public void initialize(Path parquetFilePath, Configuration hadoopConfig, 
boolean tempStore) {
+String fullPrefix = (tempStore? TEMP_FILE_PREFIX : "");
+fullPrefix += KEY_MATERIAL_FILE_PREFIX;
+keyMaterialFile = new Path(parquetFilePath.getParent(),
+  fullPrefix + parquetFilePath.getName() + KEY_MATERIAL_FILE_SUFFFIX);
+  }
+
+  @Override
+  public void addKeyMaterial(String keyIDInFile, String keyMaterial) throws 
ParquetCryptoRuntimeException {
+if (null == keyMaterialMap) {
+  keyMaterialMap = new HashMap<>();
+}
+keyMaterialMap.put(keyIDInFile, keyMaterial);
+  }
+
+  @Override
+  public String getKeyMaterial(String keyIDInFile)  throws 
ParquetCryptoRuntimeException {
+if (null == keyMaterialMap) {
+  loadKeyMaterialMap();
+}
+return keyMaterialMap.get(keyIDInFile);
+  }
+  
+  private void loadKeyMaterialMap() {
+try (FSDataInputStream keyMaterialStream = 
hadoopFileSystem.open(keyMaterialFile)) {
+  JsonNode keyMaterialJson = objectMapper.readTree(keyMaterialStream);
+  keyMaterialMap = objectMapper.readValue(keyMaterialJson,
+new TypeReference>() { });
+} catch (IOException e) {
+  throw new ParquetCryptoRuntimeException("Failed to get key material from 
" + keyMaterialFile, e);
+}
+  }
+
+  @Override
+  public void saveMaterial() throws ParquetCryptoRuntimeException {
+// TODO needed? Path qualifiedPath = 
parquetFilePath.makeQualified(hadoopFileSystem);
+try (FSDataOutputStream keyMaterialStream = 
hadoopFileSystem.create(keyMaterialFile)) {
+  objectMapper.writeValue(keyMaterialStream, keyMaterialMap);
+} catch (IOException e) {
+  throw new ParquetCryptoRuntimeException("Failed to save key material in 
" + keyMaterialFile, e);
+}
+  }
+
+  @Override
+  public Set getKeyIDSet() throws ParquetCryptoRuntimeException {
+if (null == keyMaterialMap) {
+  loadKeyMaterialMap();
+}
+
+return keyMaterialMap.keySet();

Review comment:
   need to walk back this one. We process the footer key entry first, and 
then remove it from the set/map to loop on the columns only.





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-15 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyToolkit.java
##
@@ -0,0 +1,299 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.AesGcmDecryptor;
+import org.apache.parquet.crypto.AesGcmEncryptor;
+import org.apache.parquet.crypto.AesMode;
+import org.apache.parquet.crypto.KeyAccessDeniedException;
+import org.apache.parquet.crypto.ModuleCipherFactory;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.hadoop.BadConfigurationException;
+import org.apache.parquet.hadoop.util.ConfigurationUtil;
+import org.apache.parquet.hadoop.util.HiddenFileFilter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+public class KeyToolkit {
+
+  public static final String KMS_CLIENT_CLASS_PROPERTY_NAME = 
"encryption.kms.client.class";
+  public static final String KMS_INSTANCE_ID_PROPERTY_NAME = 
"encryption.kms.instance.id";
+  public static final String DOUBLE_WRAPPING_PROPERTY_NAME = 
"encryption.double.wrapping";
+  public static final String KEY_ACCESS_TOKEN_PROPERTY_NAME = 
"encryption.key.access.token";
+  public static final String TOKEN_LIFETIME_PROPERTY_NAME = 
"encryption.key.access.token.lifetime";
+  public static final String KMS_INSTANCE_URL_PROPERTY_NAME = 
"encryption.kms.instance.url";
+  public static final String WRAP_LOCALLY_PROPERTY_NAME = 
"encryption.wrap.locally";
+  public static final String KEY_MATERIAL_INTERNAL_PROPERTY_NAME = 
"encryption.key.material.internal.storage";

Review comment:
   Every property name in README.md starts with `"parquet."`. We'll do the 
same for the encryption properties.





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-14 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/samples/VaultClient.java
##
@@ -0,0 +1,153 @@
+/*
+ * 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.samples;
+
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+
+import org.apache.parquet.crypto.KeyAccessDeniedException;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.crypto.keytools.KmsClient;
+import org.apache.parquet.crypto.keytools.RemoteKmsClient;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+
+public class VaultClient extends RemoteKmsClient {

Review comment:
   re samples - we have two classes there, `InMemoryKMS` and `VaultClient`. 
   `InMemoryKMS` is leveraged in the unitest for the Key Tools feature. 
   So indeed it should make sense to move it to the tests.
   As for the `VaultClient` - one option is to move it also to the tests 
location, commenting out the HTTP calls so the `okhttp3` dependency can be 
removed from the pom. The comments will explain the code and how to build it if 
needed. The other option is to remove `VaultClient` altogether from 
apache/parquet-mr (I can keep it eg in my fork, as a pointer if anybody's 
interested to see a sample). What do you think?





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-14 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyWrapper.java
##
@@ -0,0 +1,227 @@
+/*
+ * 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.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.crypto.keytools.KeyToolkit.KeyEncryptionKey;
+import org.codehaus.jackson.map.ObjectMapper;
+
+public class FileKeyWrapper {
+
+  public static final int KEK_LENGTH = 16;
+  public static final int KEK_ID_LENGTH = 16;
+
+  // For every token: a map of MEK_ID to (KEK ID and KEK)
+  private static final ConcurrentMap>> KEKMapPerToken =
+  new ConcurrentHashMap<>(KeyToolkit.INITIAL_PER_TOKEN_CACHE_SIZE);
+  private static volatile long lastKekCacheCleanupTimestamp = 
System.currentTimeMillis() + 60l * 1000; // grace period of 1 minute;
+
+  //A map of MEK_ID to (KEK ID and KEK) - for the current token
+  private final ConcurrentMap KEKPerMasterKeyID;
+
+  private static final ObjectMapper objectMapper = new ObjectMapper();
+
+  private final long cacheEntryLifetime;
+
+  private final KmsClient kmsClient;
+  private final String kmsInstanceID;
+  private final String kmsInstanceURL;
+  private final FileKeyMaterialStore keyMaterialStore;
+  private final Configuration hadoopConfiguration;
+  private final SecureRandom random;
+  private final boolean doubleWrapping;
+
+  private short keyCounter;
+  private String accessToken;
+
+  public FileKeyWrapper(Configuration configuration, FileKeyMaterialStore 
keyMaterialStore) {
+this.hadoopConfiguration = configuration;
+
+cacheEntryLifetime = 1000l * 
hadoopConfiguration.getLong(KeyToolkit.TOKEN_LIFETIME_PROPERTY_NAME, 
+KeyToolkit.DEFAULT_CACHE_ENTRY_LIFETIME); 
+
+kmsInstanceID = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_ID_PROPERTY_NAME, 
+KmsClient.DEFAULT_KMS_INSTANCE_ID);
+
+doubleWrapping =  
hadoopConfiguration.getBoolean(KeyToolkit.DOUBLE_WRAPPING_PROPERTY_NAME, true);
+accessToken = 
hadoopConfiguration.getTrimmed(KeyToolkit.KEY_ACCESS_TOKEN_PROPERTY_NAME, 
KmsClient.DEFAULT_ACCESS_TOKEN);
+
+kmsClient = KeyToolkit.getKmsClient(kmsInstanceID, configuration, 
accessToken, cacheEntryLifetime);
+
+kmsInstanceURL = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_URL_PROPERTY_NAME, 
+RemoteKmsClient.DEFAULT_KMS_INSTANCE_URL);
+
+this.keyMaterialStore = keyMaterialStore;
+
+random = new SecureRandom();
+keyCounter = 0;
+
+// Check caches upon each file writing (clean once in cacheEntryLifetime)
+KeyToolkit.checkKmsCacheForExpiredTokens(cacheEntryLifetime);
+if (doubleWrapping) {
+  checkKekCacheForExpiredTokens();
+
+  ExpiringCacheEntry> 
KEKCacheEntry = KEKMapPerToken.get(accessToken);
+  if ((null == KEKCacheEntry) || KEKCacheEntry.isExpired()) {
+synchronized (KEKMapPerToken) {
+  KEKCacheEntry = KEKMapPerToken.get(accessToken);
+  if ((null == KEKCacheEntry) || KEKCacheEntry.isExpired()) {
+KEKCacheEntry = new ExpiringCacheEntry<>(new 
ConcurrentHashMap(), cacheEntryLifetime);
+KEKMapPerToken.put(accessToken, KEKCacheEntry);
+  }
+}
+  }
+  KEKPerMasterKeyID = KEKCacheEntry.getCachedItem();
+} else {
+  KEKPerMasterKeyID = null;
+}
+  }
+
+  public byte[] getEncryptionKeyMetadata(byte[] dataKey, String masterKeyID, 
boolean isFooterKey) {
+return getEncryptionKeyMetadata(dataKey, masterKeyID, isFooterKey, null);
+  }
+
+  static void removeCacheEntriesForToken(String accessToken) {
+synchronized(KEKMapPerToken) {

Review comment:
   if the previous suggestion works, this is indeed not needed





[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-14 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/PropertiesDrivenCryptoFactory.java
##
@@ -0,0 +1,203 @@
+/*
+ * 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.security.SecureRandom;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.ColumnEncryptionProperties;
+import org.apache.parquet.crypto.DecryptionKeyRetriever;
+import org.apache.parquet.crypto.DecryptionPropertiesFactory;
+import org.apache.parquet.crypto.EncryptionPropertiesFactory;
+import org.apache.parquet.crypto.FileDecryptionProperties;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.crypto.ParquetCipher;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.hadoop.api.WriteSupport.WriteContext;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+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";
+  
+  public static final int DEK_LENGTH = 16;
+
+  private static final SecureRandom random = new SecureRandom();
+
+  @Override
+  public FileEncryptionProperties getFileEncryptionProperties(Configuration 
fileHadoopConfig, Path tempFilePath,
+  WriteContext fileWriteContext) throws ParquetCryptoRuntimeException {
+
+String footerKeyId = 
fileHadoopConfig.getTrimmed(FOOTER_KEY_PROPERTY_NAME); 
+String columnKeysStr = 
fileHadoopConfig.getTrimmed(COLUMN_KEYS_PROPERTY_NAME);
+
+// File shouldn't be encrypted
+if (stringIsEmpty(footerKeyId) && stringIsEmpty(columnKeysStr)) {
+  return null; 
+}
+
+if (stringIsEmpty(footerKeyId)) {
+  throw new ParquetCryptoRuntimeException("Undefined footer key");
+}
+
+FileKeyMaterialStore keyMaterialStore = null;
+boolean keyMaterialInternalStorage = 
fileHadoopConfig.getBoolean(KeyToolkit.KEY_MATERIAL_INTERNAL_PROPERTY_NAME, 
true);

Review comment:
   In this particular case, the properties is used twice is two independent 
paths - file write path (get encryption properties), and file read path (get 
decryption properties). But we can use a const (final bool) for its default 
value. And will check other properties.





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-14 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java
##
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class HadoopFSKeyMaterialStore implements FileKeyMaterialStore {
+  
+  public final static String KEY_MATERIAL_FILE_PREFIX = "_KEY_MATERIAL_FOR_";
+  public static final String TEMP_FILE_PREFIX = "_TMP";
+  public final static String KEY_MATERIAL_FILE_SUFFFIX = ".json";
+  private static final ObjectMapper objectMapper = new ObjectMapper();
+
+  private FileSystem hadoopFileSystem;
+  private Map keyMaterialMap;
+  private Path keyMaterialFile;
+  
+  HadoopFSKeyMaterialStore(FileSystem hadoopFileSystem) {
+this.hadoopFileSystem = hadoopFileSystem;
+  }
+
+  @Override
+  public void initialize(Path parquetFilePath, Configuration hadoopConfig, 
boolean tempStore) {
+String fullPrefix = (tempStore? TEMP_FILE_PREFIX : "");
+fullPrefix += KEY_MATERIAL_FILE_PREFIX;
+keyMaterialFile = new Path(parquetFilePath.getParent(),
+  fullPrefix + parquetFilePath.getName() + KEY_MATERIAL_FILE_SUFFFIX);
+  }
+
+  @Override
+  public void addKeyMaterial(String keyIDInFile, String keyMaterial) throws 
ParquetCryptoRuntimeException {
+if (null == keyMaterialMap) {
+  keyMaterialMap = new HashMap<>();
+}
+keyMaterialMap.put(keyIDInFile, keyMaterial);
+  }
+
+  @Override
+  public String getKeyMaterial(String keyIDInFile)  throws 
ParquetCryptoRuntimeException {
+if (null == keyMaterialMap) {
+  loadKeyMaterialMap();
+}
+return keyMaterialMap.get(keyIDInFile);
+  }
+  
+  private void loadKeyMaterialMap() {
+try (FSDataInputStream keyMaterialStream = 
hadoopFileSystem.open(keyMaterialFile)) {
+  JsonNode keyMaterialJson = objectMapper.readTree(keyMaterialStream);
+  keyMaterialMap = objectMapper.readValue(keyMaterialJson,
+new TypeReference>() { });
+} catch (IOException e) {
+  throw new ParquetCryptoRuntimeException("Failed to get key material from 
" + keyMaterialFile, e);
+}
+  }
+
+  @Override
+  public void saveMaterial() throws ParquetCryptoRuntimeException {
+// TODO needed? Path qualifiedPath = 
parquetFilePath.makeQualified(hadoopFileSystem);

Review comment:
   will remove the comment.





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-13 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyUnwrapper.java
##
@@ -0,0 +1,216 @@
+/*
+ * 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.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.crypto.DecryptionKeyRetriever;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.crypto.keytools.KeyToolkit.KeyWithMasterID;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+import static org.apache.parquet.crypto.keytools.KeyToolkit.stringIsEmpty;
+
+public class FileKeyUnwrapper implements DecryptionKeyRetriever {
+  // For every token: a map of KEK_ID to KEK bytes
+  private static final ConcurrentMap>> KEKMapPerToken = new 
ConcurrentHashMap<>();

Review comment:
   The KEK purpose is to minimize the Parquet-KMS interaction. Without it, 
each thread would have to go to the KMS to unwrap a key. By making the cache 
static, we provide the KEK to all threads (with the same token), if one of them 
has already unwrapped it. Will add this to the comments.
   
   We'll check the use of singletons for this. The three maps/caches are not 
identical, but we'll make an effort to unify the cache handling.
   
   ConcurrentMap has a segment synchronization for write operations, and allows 
for synchronization-free read operations; this makes it faster than HasMap with 
synchronized methods. 





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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #615: PARQUET-1373: Encryption key tools

2020-06-13 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyToolkit.java
##
@@ -0,0 +1,299 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.AesGcmDecryptor;
+import org.apache.parquet.crypto.AesGcmEncryptor;
+import org.apache.parquet.crypto.AesMode;
+import org.apache.parquet.crypto.KeyAccessDeniedException;
+import org.apache.parquet.crypto.ModuleCipherFactory;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.hadoop.BadConfigurationException;
+import org.apache.parquet.hadoop.util.ConfigurationUtil;
+import org.apache.parquet.hadoop.util.HiddenFileFilter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+public class KeyToolkit {
+
+  public static final String KMS_CLIENT_CLASS_PROPERTY_NAME = 
"encryption.kms.client.class";
+  public static final String KMS_INSTANCE_ID_PROPERTY_NAME = 
"encryption.kms.instance.id";
+  public static final String DOUBLE_WRAPPING_PROPERTY_NAME = 
"encryption.double.wrapping";
+  public static final String KEY_ACCESS_TOKEN_PROPERTY_NAME = 
"encryption.key.access.token";
+  public static final String TOKEN_LIFETIME_PROPERTY_NAME = 
"encryption.key.access.token.lifetime";
+  public static final String KMS_INSTANCE_URL_PROPERTY_NAME = 
"encryption.kms.instance.url";
+  public static final String WRAP_LOCALLY_PROPERTY_NAME = 
"encryption.wrap.locally";
+  public static final String KEY_MATERIAL_INTERNAL_PROPERTY_NAME = 
"encryption.key.material.internal.storage";
+
+  public static final String KEY_MATERIAL_TYPE_FIELD = "keyMaterialType";
+  public static final String KEY_MATERIAL_TYPE = "PKMT1";
+  public static final String KEY_MATERIAL_INTERNAL_STORAGE_FIELD = 
"internalStorage";
+  public static final String KEY_REFERENCE_FIELD = "keyReference";
+  public static final String DOUBLE_WRAPPING_FIELD = "doubleWrapping";
+
+  public static final String KMS_INSTANCE_ID_FIELD = "kmsInstanceID";
+  public static final String KMS_INSTANCE_URL_FIELD = "kmsInstanceURL";
+
+  public static final String MASTER_KEY_ID_FIELD = "masterKeyID";
+  public static final String WRAPPED_DEK_FIELD = "wrappedDEK";
+  public static final String KEK_ID_FIELD = "keyEncryptionKeyID";
+  public static final String WRAPPED_KEK_FIELD = "wrappedKEK";
+
+  public static final String FOOTER_KEY_ID_IN_FILE = "kf";
+  public static final String KEY_ID_IN_FILE_PREFIX = "k";

Review comment:
   changed to `"footerKey"`, and `COLUMN_KEY_ID_IN_FILE_PREFIX = 
"columnKey"`





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