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

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


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

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


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

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




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

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


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

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


> 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

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

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


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

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


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




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

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


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

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


> 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

Re: Arrow 1404: Adding index for Page-level Skipping

2020-06-13 Thread Lekshmi Narayanan, Arun Balajiee
Hi Dev

Thanks Wes for these comments.

As Informed in other threads, I have completed most of it. Will try to 
structure it according to the comments.

I had one question reading a (un)related matter. whenever we make calls to

ReadBatch(int64_t batch_size, int16_t* def_levels,
int16_t* rep_levels, T* values,
int64_t* values_read)

Is there are possibility to keep track of which page we are at to retrieve 
values?

Regards
Arun Balajiee

From: Wes McKinney 
Sent: 02 April 2020 13:16
To: Parquet Dev 
Cc: Deepak Majeti ; Anatoli Shein 

Subject: Re: Arrow 1404: Adding index for Page-level Skipping

I just left comments on the PR. The new APIs (their semantics and what
should be passed as arguments) are still not adequately documented (in
other words, I wouldn't know how to use them just from reading the
header file), so I think we should focus on that for the moment. In
fairness documentation for other functions in these headers in poor,
but they also have the semantics of "read all data in the file from
start to finish". These new APIs appear to do something different, so
we need to write that down in detail in Doxygen-style comments

On Thu, Apr 2, 2020 at 2:23 AM Lekshmi Narayanan, Arun Balajiee
 wrote:
>
> Hi
> Would my pull request be useful for the discussion from here?
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fapache%2Farrow%2Fpull%2F6807&data=02%7C01%7CARL122%40pitt.edu%7Cd36ddd6e18fb44808ef308d7d729b8c8%7C9ef9f489e0a04eeb87cc3a526112fd0d%7C1%7C1%7C637214446509861845&sdata=PQAIxpTPm87qRb%2FmZoHXfLCsdcCiyC%2Biqui40tqEd9U%3D&reserved=0
>
> Regards,
> Arun Balajiee
>
> From: Wes McKinney
> Sent: Tuesday, February 18, 2020 3:34 AM
> To: Parquet Dev
> Cc: Deepak Majeti; Anatoli 
> Shein
> Subject: Re: Arrow 1404: Adding index for Page-level Skipping
>
> That's helpful, but I think it would be a good idea to have enough
> information in the header files to determine what the new APIs do
> without reading example code.
>
> On Mon, Feb 17, 2020 at 10:59 AM Lekshmi Narayanan, Arun Balajiee
>  wrote:
> >
> > I also made changes in the low-level-api folder, couldn’t capture in that 
> > link I think
> > https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fa2un%2Farrow%2Fblob%2FPARQUET-1404-Add-index-pages-to-the-format-to-support-efficient-page-skipping-to-parquet-cpp%2Fcpp%2Fexamples%2Fparquet%2Flow-level-api%2Freader-writer-with-index.cc&data=02%7C01%7CARL122%40pitt.edu%7Cd36ddd6e18fb44808ef308d7d729b8c8%7C9ef9f489e0a04eeb87cc3a526112fd0d%7C1%7C1%7C637214446509861845&sdata=vxuK%2BvZRtwhLcGepda6T5i3r6HDk0JLS3vh9leIcBlo%3D&reserved=0
> >
> > Regards,
> > Arun Balajiee
> >
> > 
> > From: Wes McKinney 
> > Sent: Monday, February 17, 2020 8:11:09 AM
> > To: Parquet Dev 
> > Cc: Deepak Majeti ; Anatoli Shein 
> > 
> > Subject: Re: Arrow 1404: Adding index for Page-level Skipping
> >
> > hi Arun,
> >
> > By "public APIs" I was referring to changes in the public header
> > files. I see there are some changes to parquet/file_reader.h and
> > metadata.h
> >
> > https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fapache%2Farrow%2Fcompare%2Fmaster...a2un%3APARQUET-1404-Add-index-pages-to-the-format-to-support-efficient-page-skipping-to-parquet-cpp&data=02%7C01%7CARL122%40pitt.edu%7Cd36ddd6e18fb44808ef308d7d729b8c8%7C9ef9f489e0a04eeb87cc3a526112fd0d%7C1%7C1%7C637214446509871841&sdata=rBl3pY6bRFuSzWg2QT2Ca6aui2HZJjSoh1mbzDq%2F93M%3D&reserved=0
> >
> > Can you add some Doxygen comments to the new APIs that explain how
> > these APIs are to be used (and what the parameters mean)? The hope
> > would be that a user could make use of the column index functionality
> > by reading the .h files only.
> >
> > Thanks
> > Wes
> >
> > On Fri, Feb 14, 2020 at 2:57 PM Lekshmi Narayanan, Arun Balajiee
> >  wrote:
> > >
> > > Hi
> > > I have made my changes for api here, does it look good and is this what 
> > > you were seeking from me? The writer- api is still in the works and I 
> > > need to make the reader more generic to support all class data types.
> > >
> > > https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fa2un%2Farrow%2Fblob%2FPARQUET-1404-Add-index-pages-to-the-format-to-support-efficient-page-skipping-to-parquet-cpp%2Fcpp%2Fexamples%2Fparquet%2Flow-level-api%2Freader-writer-with-index.cc&data=02%7C01%7CARL122%40pitt.edu%7Cd36ddd6e18fb44808ef308d7d729b8c8%7C9ef9f489e0a04eeb87cc3a526112fd0d%7C1%7C1%7C637214446509871841&sdata=TB29CbqF3MlD0v9d%2BOTH%2FD4NAF%2BqGJvpMpJZIeWd2P4%3D&reserved=0
> > >
> > >
> > > Regards,
> > > Arun Balajiee
> > >
> > > From: Wes McKinney
> > > Sent: Tuesday, February 4, 202