[jira] [Commented] (PARQUET-1373) Encryption key management tools

2020-06-15 Thread ASF GitHub Bot (Jira)


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

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


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


[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




[jira] [Commented] (PARQUET-1866) Replace Hadoop ZSTD with JNI-ZSTD

2020-06-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1866:
-

dbtsai commented on pull request #797:
URL: https://github.com/apache/parquet-mr/pull/797#issuecomment-643920535


   LGTM. This will help Spark community to adopt zstd easier. Thanks for the 
great work!



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


> Replace Hadoop ZSTD with JNI-ZSTD
> -
>
> Key: PARQUET-1866
> URL: https://issues.apache.org/jira/browse/PARQUET-1866
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
> Fix For: 1.12.0
>
>
> The parquet-mr repo has been using 
> [ZSTD-JNI|https://github.com/luben/zstd-jni/tree/master/src/main/java/com/github/luben/zstd]
>  for the parquet-cli project. It is a cleaner approach to use this JNI than 
> using Hadoop ZSTD compression, because 1) on the developing box, installing 
> Hadoop is cumbersome, 2) Older version of Hadoop doesn't support ZSTD. 
> Upgrading Hadoop is another pain. This Jira is to replace Hadoop ZSTD with 
> ZSTD-JNI for parquet-hadoop project. 
> According to the author of ZSTD-JNI, Flink, Spark, Cassandra all use ZSTD-JNI 
> for ZSTD.
> Another approach is to use https://github.com/airlift/aircompressor which is 
> a pure Java implementation. But it seems the compression level is not 
> adjustable in aircompressor. 



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


[GitHub] [parquet-mr] dbtsai commented on pull request #797: PARQUET-1866: Replace Hadoop ZSTD with JNI-ZSTD (#793)

2020-06-15 Thread GitBox


dbtsai commented on pull request #797:
URL: https://github.com/apache/parquet-mr/pull/797#issuecomment-643920535


   LGTM. This will help Spark community to adopt zstd easier. Thanks for the 
great work!



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




[jira] [Commented] (PARQUET-1876) Port ZSTD-JNI support to 1.10.x brach

2020-06-15 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1876:
---

I don't think this is a bug. This is a feature request for a maintenance 
release. Usually, we do not do such things. 

> Port ZSTD-JNI support to 1.10.x brach
> -
>
> Key: PARQUET-1876
> URL: https://issues.apache.org/jira/browse/PARQUET-1876
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.10.2
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
> Fix For: 1.10.2
>
>
> I hear the need to port the zstd-jni support to 1.10.x because of easiness to 
> use ZSTD. 
> cc [~dbtsai]



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


[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




[jira] [Commented] (PARQUET-1373) Encryption key management tools

2020-06-15 Thread ASF GitHub Bot (Jira)


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

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


> Encryption key management tools 

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

2020-06-15 Thread GitBox


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



##
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();

Review comment:
   As per the java naming conventions:
   ```suggestion
 private static final SecureRandom RANDOM = new SecureRandom();
   ```

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/TestPropertiesDrivenEncryption.java
##
@@ -0,0 +1,507 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.keytools.KeyToolkit;
+import org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory;
+import org.apache.parquet.crypto.keytools.samples.InMemoryKMS;
+import org.apache.parquet.crypto.mocks.RemoteKmsClientMock;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroupFactory;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Types;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import 

[jira] [Commented] (PARQUET-1373) Encryption key management tools

2020-06-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1373:
-

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



##
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();

Review comment:
   As per the java naming conventions:
   ```suggestion
 private static final SecureRandom RANDOM = new SecureRandom();
   ```

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/TestPropertiesDrivenEncryption.java
##
@@ -0,0 +1,507 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.keytools.KeyToolkit;
+import org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory;
+import org.apache.parquet.crypto.keytools.samples.InMemoryKMS;
+import org.apache.parquet.crypto.mocks.RemoteKmsClientMock;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroupFactory;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Types;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+import 

[jira] [Commented] (PARQUET-1373) Encryption key management tools

2020-06-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1373:
-

ggershinsky commented on pull request #615:
URL: https://github.com/apache/parquet-mr/pull/615#issuecomment-644145175


   > > ConcurrentMap has a segment synchronization for write operations, and 
allows for synchronization-free read operations; this makes it faster than 
HasMap with synchronized methods.
   > 
   > Yes, I know how `ConcurrentHashMap` works. What I wanted to say that you 
are using synchronization as well. As you already use a `ConcurrentMap` you 
might implement these synchronized code parts by using the methods of 
`ConcurrentMap`. I've put some examples that might work.
   
   Sounds good, and thank you for the examples! We've already applied your code 
(not pushed yet), it indeed allowed to remove the explicit synchronization, 
making the cache implementation cleaner and faster.
   
   > 
   > Please, check why Travis fails.
   > 
   
   Sorry, should have mentioned that it will take a few more commits to fully 
address this round of the comments (and fix the unitests). Once all commits are 
in, I will squash them to simplify the review, and will post a comment here.
   
   > Another point of view came up about handling sensitive data in memory. 
Java does not clean memory after garbage collecting objects. It means that 
sensitive data must be manually cleaned after used otherwise it might get 
compromised by another java application in the same jvm or even by another 
process after the jvm exists. Because of the same reason `String` objects shall 
never contain sensitive information as the `char[]` behind the object might not 
get garbage collected after the `String` object itself gets dropped.
   > I did not find any particular bad practice in the code or any examples of 
the listed situations just wanted to highlight that we shall think about this 
as well.
   
   Yep, keeping secret data in Java strings is a notorious problem. I think the 
general consensus is not to rely on gc or explicit byte wiping - but to 
remember that these Java processes must run in a trusted environment anyway, 
simply because they work with confidential information, ranging from the 
encryption keys to the sensitive data itself. Micro-managing the memory with 
confidential information is always hard, and is basically impossible with Java. 
It goes beyond Parquet. One example - the KMS Client implementations send 
secret tokens and fetch explicit encryption keys, using a custom HTTP library. 
There is no guarantee this library doesn't use strings (most likely, it does). 
Another example - the secret tokens are passed as a Hadoop property from Spark 
or another framework; this is likely to be implemented with strings. Moreover, 
the tokens are built in an access control system, then sent to a user, then 
sent to a Spark driver, then sent to Spark workers (or other framework 
components) - there is no way to control this, except to rely on HTTPS for the 
transport security, and on running framework drivers/workers in a trusted 
environment for the memory security.



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)


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

2020-06-15 Thread GitBox


ggershinsky commented on pull request #615:
URL: https://github.com/apache/parquet-mr/pull/615#issuecomment-644145175


   > > ConcurrentMap has a segment synchronization for write operations, and 
allows for synchronization-free read operations; this makes it faster than 
HasMap with synchronized methods.
   > 
   > Yes, I know how `ConcurrentHashMap` works. What I wanted to say that you 
are using synchronization as well. As you already use a `ConcurrentMap` you 
might implement these synchronized code parts by using the methods of 
`ConcurrentMap`. I've put some examples that might work.
   
   Sounds good, and thank you for the examples! We've already applied your code 
(not pushed yet), it indeed allowed to remove the explicit synchronization, 
making the cache implementation cleaner and faster.
   
   > 
   > Please, check why Travis fails.
   > 
   
   Sorry, should have mentioned that it will take a few more commits to fully 
address this round of the comments (and fix the unitests). Once all commits are 
in, I will squash them to simplify the review, and will post a comment here.
   
   > Another point of view came up about handling sensitive data in memory. 
Java does not clean memory after garbage collecting objects. It means that 
sensitive data must be manually cleaned after used otherwise it might get 
compromised by another java application in the same jvm or even by another 
process after the jvm exists. Because of the same reason `String` objects shall 
never contain sensitive information as the `char[]` behind the object might not 
get garbage collected after the `String` object itself gets dropped.
   > I did not find any particular bad practice in the code or any examples of 
the listed situations just wanted to highlight that we shall think about this 
as well.
   
   Yep, keeping secret data in Java strings is a notorious problem. I think the 
general consensus is not to rely on gc or explicit byte wiping - but to 
remember that these Java processes must run in a trusted environment anyway, 
simply because they work with confidential information, ranging from the 
encryption keys to the sensitive data itself. Micro-managing the memory with 
confidential information is always hard, and is basically impossible with Java. 
It goes beyond Parquet. One example - the KMS Client implementations send 
secret tokens and fetch explicit encryption keys, using a custom HTTP library. 
There is no guarantee this library doesn't use strings (most likely, it does). 
Another example - the secret tokens are passed as a Hadoop property from Spark 
or another framework; this is likely to be implemented with strings. Moreover, 
the tokens are built in an access control system, then sent to a user, then 
sent to a Spark driver, then sent to Spark workers (or other framework 
components) - there is no way to control this, except to rely on HTTPS for the 
transport security, and on running framework drivers/workers in a trusted 
environment for the memory security.



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 edited a comment on pull request #615: PARQUET-1373: Encryption key tools

2020-06-15 Thread GitBox


ggershinsky edited a comment on pull request #615:
URL: https://github.com/apache/parquet-mr/pull/615#issuecomment-644145175


   > > ConcurrentMap has a segment synchronization for write operations, and 
allows for synchronization-free read operations; this makes it faster than 
HasMap with synchronized methods.
   > 
   > Yes, I know how `ConcurrentHashMap` works. What I wanted to say that you 
are using synchronization as well. As you already use a `ConcurrentMap` you 
might implement these synchronized code parts by using the methods of 
`ConcurrentMap`. I've put some examples that might work.
   
   Sounds good, and thank you for the examples! We've already applied your code 
(not pushed yet), it indeed allowed to remove the explicit synchronization, 
making the cache implementation cleaner and faster.
   
   > 
   > Please, check why Travis fails.
   > 
   
   Sorry, should have mentioned that it will take a few more commits to fully 
address this round of the comments (and fix the unitests). Once all commits are 
in, I will squash them to simplify the review, and will post a comment here.
   
   > Another point of view came up about handling sensitive data in memory. 
Java does not clean memory after garbage collecting objects. It means that 
sensitive data must be manually cleaned after used otherwise it might get 
compromised by another java application in the same jvm or even by another 
process after the jvm exists. Because of the same reason `String` objects shall 
never contain sensitive information as the `char[]` behind the object might not 
get garbage collected after the `String` object itself gets dropped.
   > I did not find any particular bad practice in the code or any examples of 
the listed situations just wanted to highlight that we shall think about this 
as well.
   
   Yep, keeping secret data in Java strings is a notorious problem. I think the 
general consensus is not to rely on gc or explicit byte wiping - but to 
remember that these Java processes must run in a trusted environment anyway, 
simply because they work with confidential information, ranging from the 
encryption keys to the sensitive data itself. Micro-managing the memory with 
confidential information is always hard, and is basically impossible with Java. 
It goes beyond Parquet. One example - the KMS Client implementations send 
secret tokens and fetch explicit encryption keys, using a custom HTTP library. 
There is no guarantee this library doesn't use strings (most likely, it does). 
Another example - the secret tokens are passed as a Hadoop property from Spark 
or another framework; this is likely to be implemented with strings. Moreover, 
the tokens are built in an access control system, then sent to a user, then 
sent to a Spark driver, then sent to Spark workers (or other framework 
components) - there is no way to control this, except to rely on HTTPS for the 
transport security, and on running framework drivers/workers in a trusted 
environment for the memory security.
   
   In other words, our threat model is simple. We don't trust the storage - 
encrypted Parquet files can be accessed by malicious parties, but they won't be 
able to read them. We do trust the framework hosts (where the JVM runs) - if 
these are breached, the secret data can be stolen from any part of host memory 
/ disc pages; not just the Parquet lib memory, but framework memory, HTTP libs, 
etc. Memory protection is a holy grail in this field, addressed by technologies 
like VMs, containers, hardware enclaves, etc, etc. Parquet encryption is 
focused on data-in-storage protection; data-in-memory protection is covered by 
other technologies.



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




[jira] [Commented] (PARQUET-1373) Encryption key management tools

2020-06-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1373:
-

ggershinsky edited a comment on pull request #615:
URL: https://github.com/apache/parquet-mr/pull/615#issuecomment-644145175


   > > ConcurrentMap has a segment synchronization for write operations, and 
allows for synchronization-free read operations; this makes it faster than 
HasMap with synchronized methods.
   > 
   > Yes, I know how `ConcurrentHashMap` works. What I wanted to say that you 
are using synchronization as well. As you already use a `ConcurrentMap` you 
might implement these synchronized code parts by using the methods of 
`ConcurrentMap`. I've put some examples that might work.
   
   Sounds good, and thank you for the examples! We've already applied your code 
(not pushed yet), it indeed allowed to remove the explicit synchronization, 
making the cache implementation cleaner and faster.
   
   > 
   > Please, check why Travis fails.
   > 
   
   Sorry, should have mentioned that it will take a few more commits to fully 
address this round of the comments (and fix the unitests). Once all commits are 
in, I will squash them to simplify the review, and will post a comment here.
   
   > Another point of view came up about handling sensitive data in memory. 
Java does not clean memory after garbage collecting objects. It means that 
sensitive data must be manually cleaned after used otherwise it might get 
compromised by another java application in the same jvm or even by another 
process after the jvm exists. Because of the same reason `String` objects shall 
never contain sensitive information as the `char[]` behind the object might not 
get garbage collected after the `String` object itself gets dropped.
   > I did not find any particular bad practice in the code or any examples of 
the listed situations just wanted to highlight that we shall think about this 
as well.
   
   Yep, keeping secret data in Java strings is a notorious problem. I think the 
general consensus is not to rely on gc or explicit byte wiping - but to 
remember that these Java processes must run in a trusted environment anyway, 
simply because they work with confidential information, ranging from the 
encryption keys to the sensitive data itself. Micro-managing the memory with 
confidential information is always hard, and is basically impossible with Java. 
It goes beyond Parquet. One example - the KMS Client implementations send 
secret tokens and fetch explicit encryption keys, using a custom HTTP library. 
There is no guarantee this library doesn't use strings (most likely, it does). 
Another example - the secret tokens are passed as a Hadoop property from Spark 
or another framework; this is likely to be implemented with strings. Moreover, 
the tokens are built in an access control system, then sent to a user, then 
sent to a Spark driver, then sent to Spark workers (or other framework 
components) - there is no way to control this, except to rely on HTTPS for the 
transport security, and on running framework drivers/workers in a trusted 
environment for the memory security.
   
   In other words, our threat model is simple. We don't trust the storage - 
encrypted Parquet files can be accessed by malicious parties, but they won't be 
able to read them. We do trust the framework hosts (where the JVM runs) - if 
these are breached, the secret data can be stolen from any part of host memory 
/ disc pages; not just the Parquet lib memory, but framework memory, HTTP libs, 
etc. Memory protection is a holy grail in this field, addressed by technologies 
like VMs, containers, hardware enclaves, etc, etc. Parquet encryption is 
focused on data-in-storage protection; data-in-memory protection is covered by 
other technologies.



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