This is an automated email from the ASF dual-hosted git repository.

gabor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new d291d05  PARQUET-1938: Key rotation - option to get KMS details from 
key material (#842)
d291d05 is described below

commit d291d05f1c9e3ee5ab87403d94cf13a5ab9cb04d
Author: andersonm-ibm <[email protected]>
AuthorDate: Thu Nov 12 18:03:43 2020 +0200

    PARQUET-1938: Key rotation - option to get KMS details from key material 
(#842)
---
 .../parquet/crypto/keytools/FileKeyUnwrapper.java  | 15 ++++++----
 .../parquet/crypto/keytools/FileKeyWrapper.java    | 32 ++++++++++++++--------
 .../crypto/keytools/HadoopFSKeyMaterialStore.java  |  6 ++--
 .../apache/parquet/crypto/keytools/KeyToolkit.java | 32 ++++++++++++++++++++--
 4 files changed, 64 insertions(+), 21 deletions(-)

diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyUnwrapper.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyUnwrapper.java
index 60bc77f..02ff2b8 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyUnwrapper.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyUnwrapper.java
@@ -42,7 +42,7 @@ public class FileKeyUnwrapper implements 
DecryptionKeyRetriever {
   //A map of KEK_ID -> KEK bytes, for the current token
   private final ConcurrentMap<String,byte[]> kekPerKekID;
 
-  private KmsClient kmsClient = null;
+  private KeyToolkit.KmsClientAndDetails kmsClientAndDetails = null;
   private FileKeyMaterialStore keyMaterialStore = null;
   private boolean checkedKeyMaterialInternalStorage = false;
   private final Configuration hadoopConfiguration;
@@ -112,8 +112,8 @@ public class FileKeyUnwrapper implements 
DecryptionKeyRetriever {
 
 
   KeyWithMasterID getDEKandMasterID(KeyMaterial keyMaterial)  {
-    if (null == kmsClient) {
-      kmsClient = getKmsClientFromConfigOrKeyMaterial(keyMaterial);
+    if (null == kmsClientAndDetails) {
+      kmsClientAndDetails = getKmsClientFromConfigOrKeyMaterial(keyMaterial);
     }
 
     boolean doubleWrapping = keyMaterial.isDoubleWrapped();
@@ -121,6 +121,7 @@ public class FileKeyUnwrapper implements 
DecryptionKeyRetriever {
     String encodedWrappedDEK = keyMaterial.getWrappedDEK();
 
     byte[] dataKey;
+    KmsClient kmsClient = kmsClientAndDetails.getKmsClient();
     if (!doubleWrapping) {
       dataKey = kmsClient.unwrapKey(encodedWrappedDEK, masterKeyID);
     } else {
@@ -143,7 +144,7 @@ public class FileKeyUnwrapper implements 
DecryptionKeyRetriever {
     return new KeyWithMasterID(dataKey, masterKeyID);
   }
 
-  private KmsClient getKmsClientFromConfigOrKeyMaterial(KeyMaterial 
keyMaterial) {
+  KeyToolkit.KmsClientAndDetails 
getKmsClientFromConfigOrKeyMaterial(KeyMaterial keyMaterial) {
     String kmsInstanceID = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_ID_PROPERTY_NAME);
     if (stringIsEmpty(kmsInstanceID)) {
       kmsInstanceID = keyMaterial.getKmsInstanceID();
@@ -168,6 +169,10 @@ public class FileKeyUnwrapper implements 
DecryptionKeyRetriever {
     if (LOG.isDebugEnabled()) {
       LOG.debug("File unwrapper - KmsClient: {}; InstanceId: {}; InstanceURL: 
{}", kmsClient, kmsInstanceID, kmsInstanceURL);
     }
-    return kmsClient;
+    return new KeyToolkit.KmsClientAndDetails(kmsClient, kmsInstanceID, 
kmsInstanceURL);
   }
+
+  KeyToolkit.KmsClientAndDetails getKmsClientAndDetails() {
+    return kmsClientAndDetails;
+  }
 }
\ No newline at end of file
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyWrapper.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyWrapper.java
index d6d6d77..7551fb4 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyWrapper.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/FileKeyWrapper.java
@@ -55,28 +55,34 @@ public class FileKeyWrapper {
   private short keyCounter;
   private String accessToken;
 
-  FileKeyWrapper(Configuration configuration, FileKeyMaterialStore 
keyMaterialStore) {
+  FileKeyWrapper(Configuration configuration, FileKeyMaterialStore 
keyMaterialStore,
+                 KeyToolkit.KmsClientAndDetails kmsClientAndDetails) {
     this.hadoopConfiguration = configuration;
     this.keyMaterialStore = keyMaterialStore;
-    
+
     random = new SecureRandom();
     keyCounter = 0;
 
-    cacheEntryLifetime = 1000L * 
hadoopConfiguration.getLong(KeyToolkit.CACHE_LIFETIME_PROPERTY_NAME, 
-        KeyToolkit.CACHE_LIFETIME_DEFAULT_SECONDS); 
-
-    kmsInstanceID = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_ID_PROPERTY_NAME, 
-        KmsClient.KMS_INSTANCE_ID_DEFAULT);
+    cacheEntryLifetime = 1000L * 
hadoopConfiguration.getLong(KeyToolkit.CACHE_LIFETIME_PROPERTY_NAME,
+      KeyToolkit.CACHE_LIFETIME_DEFAULT_SECONDS);
 
     doubleWrapping =  
hadoopConfiguration.getBoolean(KeyToolkit.DOUBLE_WRAPPING_PROPERTY_NAME, 
KeyToolkit.DOUBLE_WRAPPING_DEFAULT);
     accessToken = 
hadoopConfiguration.getTrimmed(KeyToolkit.KEY_ACCESS_TOKEN_PROPERTY_NAME, 
KmsClient.KEY_ACCESS_TOKEN_DEFAULT);
 
-    kmsInstanceURL = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_URL_PROPERTY_NAME, 
-        KmsClient.KMS_INSTANCE_URL_DEFAULT);
-
     // Check caches upon each file writing (clean once in cacheEntryLifetime)
     KMS_CLIENT_CACHE_PER_TOKEN.checkCacheForExpiredTokens(cacheEntryLifetime);
-    kmsClient = KeyToolkit.getKmsClient(kmsInstanceID, kmsInstanceURL, 
configuration, accessToken, cacheEntryLifetime);
+    
+    if (null == kmsClientAndDetails) {
+      kmsInstanceID = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_ID_PROPERTY_NAME,
+        KmsClient.KMS_INSTANCE_ID_DEFAULT);
+      kmsInstanceURL = 
hadoopConfiguration.getTrimmed(KeyToolkit.KMS_INSTANCE_URL_PROPERTY_NAME,
+        KmsClient.KMS_INSTANCE_URL_DEFAULT);
+      kmsClient = KeyToolkit.getKmsClient(kmsInstanceID, kmsInstanceURL, 
configuration, accessToken, cacheEntryLifetime);
+    } else {
+      kmsInstanceID = kmsClientAndDetails.getKmsInstanceID();
+      kmsInstanceURL = kmsClientAndDetails.getKmsInstanceURL();
+      kmsClient = kmsClientAndDetails.getKmsClient();
+    }
 
     if (doubleWrapping) {
       KEK_WRITE_CACHE_PER_TOKEN.checkCacheForExpiredTokens(cacheEntryLifetime);
@@ -99,6 +105,10 @@ public class FileKeyWrapper {
     }
   }
 
+  FileKeyWrapper(Configuration configuration, FileKeyMaterialStore 
keyMaterialStore) {
+    this(configuration, keyMaterialStore, null/*kmsClientAndDetails*/);
+  }
+
   byte[] getEncryptionKeyMetadata(byte[] dataKey, String masterKeyID, boolean 
isFooterKey) {
     return getEncryptionKeyMetadata(dataKey, masterKeyID, isFooterKey, null);
   }
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java
index fae700a..c569911 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java
@@ -28,13 +28,13 @@ import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 
+import java.io.FileNotFoundException;
 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";
@@ -77,6 +77,8 @@ public class HadoopFSKeyMaterialStore implements 
FileKeyMaterialStore {
       JsonNode keyMaterialJson = objectMapper.readTree(keyMaterialStream);
       keyMaterialMap = objectMapper.readValue(keyMaterialJson,
         new TypeReference<Map<String, String>>() { });
+    } catch (FileNotFoundException e) {
+      throw new ParquetCryptoRuntimeException("External key material not found 
at " + keyMaterialFile, e);
     } catch (IOException e) {
       throw new ParquetCryptoRuntimeException("Failed to get key material from 
" + keyMaterialFile, e);
     }
@@ -105,7 +107,7 @@ public class HadoopFSKeyMaterialStore implements 
FileKeyMaterialStore {
     try {
       hadoopFileSystem.delete(keyMaterialFile, false);
     } catch (IOException e) {
-      throw new ParquetCryptoRuntimeException("Failed to delete file " + 
keyMaterialFile, e);
+      throw new ParquetCryptoRuntimeException("Failed to delete key material 
file " + keyMaterialFile, e);
     }
   }
 
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyToolkit.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyToolkit.java
index ea8a026..0b5a47b 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyToolkit.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyToolkit.java
@@ -206,7 +206,7 @@ public class KeyToolkit {
   public static void rotateMasterKeys(String folderPath, Configuration 
hadoopConfig)
     throws IOException, ParquetCryptoRuntimeException, 
KeyAccessDeniedException, UnsupportedOperationException {
 
-    if (hadoopConfig.getBoolean(KEY_MATERIAL_INTERNAL_PROPERTY_NAME, 
KEY_MATERIAL_INTERNAL_DEFAULT)) {
+    if (hadoopConfig.getBoolean(KEY_MATERIAL_INTERNAL_PROPERTY_NAME, false)) {
       throw new UnsupportedOperationException("Key rotation is not supported 
for internal key material");
     }
 
@@ -237,17 +237,19 @@ public class KeyToolkit {
 
       FileKeyMaterialStore keyMaterialStore = new 
HadoopFSKeyMaterialStore(hadoopFileSystem);
       keyMaterialStore.initialize(parquetFile, hadoopConfig, false);
-      FileKeyUnwrapper fileKeyUnwrapper = new FileKeyUnwrapper(hadoopConfig, 
parquetFile, keyMaterialStore);
 
       FileKeyMaterialStore tempKeyMaterialStore = new 
HadoopFSKeyMaterialStore(hadoopFileSystem);
       tempKeyMaterialStore.initialize(parquetFile, hadoopConfig, true);
-      FileKeyWrapper fileKeyWrapper = new FileKeyWrapper(hadoopConfig, 
tempKeyMaterialStore);
 
       Set<String> fileKeyIdSet = keyMaterialStore.getKeyIDSet();
 
       // Start with footer key (to get KMS ID, URL, if needed)
+      FileKeyUnwrapper fileKeyUnwrapper = new FileKeyUnwrapper(hadoopConfig, 
parquetFile, keyMaterialStore);
       String keyMaterialString = 
keyMaterialStore.getKeyMaterial(KeyMaterial.FOOTER_KEY_ID_IN_FILE);
       KeyWithMasterID key = 
fileKeyUnwrapper.getDEKandMasterID(KeyMaterial.parse(keyMaterialString));
+      KmsClientAndDetails kmsClientAndDetails = 
fileKeyUnwrapper.getKmsClientAndDetails();
+
+      FileKeyWrapper fileKeyWrapper = new FileKeyWrapper(hadoopConfig, 
tempKeyMaterialStore, kmsClientAndDetails);
       fileKeyWrapper.getEncryptionKeyMetadata(key.getDataKey(), 
key.getMasterID(), true,
         KeyMaterial.FOOTER_KEY_ID_IN_FILE);
 
@@ -365,4 +367,28 @@ public class KeyToolkit {
   static boolean stringIsEmpty(String str) {
     return (null == str) || str.isEmpty();
   }
+
+  static class KmsClientAndDetails {
+    private KmsClient kmsClient;
+    private String kmsInstanceID;
+    private String kmsInstanceURL;
+
+    public KmsClientAndDetails(KmsClient kmsClient, String kmsInstanceID, 
String kmsInstanceURL) {
+      this.kmsClient = kmsClient;
+      this.kmsInstanceID = kmsInstanceID;
+      this.kmsInstanceURL = kmsInstanceURL;
+    }
+
+    public KmsClient getKmsClient() {
+      return kmsClient;
+    }
+
+    public String getKmsInstanceID() {
+      return kmsInstanceID;
+    }
+
+    public String getKmsInstanceURL() {
+      return kmsInstanceURL;
+    }
+  }
 }

Reply via email to