[ 
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<String, 
ExpiringCacheEntry<ConcurrentMap<String, KeyEncryptionKey>>> 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<String, KeyEncryptionKey> 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<ConcurrentMap<String, KeyEncryptionKey>> 
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<String, KeyEncryptionKey>(), 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




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

Reply via email to