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

2020-06-14 Thread GitBox


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


   > There are a couple of concurrent code parts. We shall test somehow that 
the code is thread-safe. I know, it is not easy but if you execute the related 
code paths with several parallel threads (e.g. encrypting/decrypting several 
files in multiple threads) we might find some issues.
   
   
   we'll make the PR test to run multiple threads, writing a number of files in 
parallel (same for reading).
   



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-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1373:
-

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


   > There are a couple of concurrent code parts. We shall test somehow that 
the code is thread-safe. I know, it is not easy but if you execute the related 
code paths with several parallel threads (e.g. encrypting/decrypting several 
files in multiple threads) we might find some issues.
   
   
   we'll make the PR test to run multiple threads, writing a number of files in 
parallel (same for reading).
   



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




--

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

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


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

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

[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




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

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


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

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


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


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

2020-06-14 Thread Xinli Shang (Jira)
Xinli Shang created PARQUET-1876:


 Summary: 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
 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] shangxinli opened a new pull request #797: PARQUET-1866: Replace Hadoop ZSTD with JNI-ZSTD (#793)

2020-06-14 Thread GitBox


shangxinli opened a new pull request #797:
URL: https://github.com/apache/parquet-mr/pull/797


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1866:
-

shangxinli opened a new pull request #797:
URL: https://github.com/apache/parquet-mr/pull/797


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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)


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

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


[ 
https://issues.apache.org/jira/browse/PARQUET-1866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-14 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




[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_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-1373) Encryption key management tools

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


[ 
https://issues.apache.org/jira/browse/PARQUET-1373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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)