You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/08/16 18:01:30 UTC

git commit: ACCUMULO-1654 Fixed a bug where keys and IVs from encrypted files were not always entirely read before being used. Found this bug in longevity testing. Also adds a key management strategy that caches keys retrieved from HDFS for better perfor

Updated Branches:
  refs/heads/master e0a37f808 -> 2d36f885a


ACCUMULO-1654 Fixed a bug where keys and IVs from encrypted files were not always entirely read before being used. Found this bug in longevity testing. Also adds a key management strategy that caches keys retrieved from HDFS for better performance.

Signed-off-by: Keith Turner <kt...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/2d36f885
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/2d36f885
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/2d36f885

Branch: refs/heads/master
Commit: 2d36f885a117e529195ca9176dc288d85618bfc1
Parents: e0a37f8
Author: Michael Allen <mi...@sqrrl.com>
Authored: Thu Aug 15 16:02:20 2013 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Fri Aug 16 11:54:20 2013 -0400

----------------------------------------------------------------------
 .../accumulo/core/file/rfile/bcfile/BCFile.java |   2 +-
 .../CachingHDFSSecretKeyEncryptionStrategy.java | 205 +++++++++++++++++++
 .../security/crypto/CryptoModuleFactory.java    |  48 ++++-
 .../security/crypto/DefaultCryptoModule.java    |   4 +-
 4 files changed, 249 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2d36f885/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
index 35cd82e..2d9f6af 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
@@ -613,7 +613,7 @@ public final class BCFile {
           
           int ivLength = tempDataInputStream.readInt();
           byte[] initVector = new byte[ivLength];
-          tempDataInputStream.read(initVector);
+          tempDataInputStream.readFully(initVector);
           
           cryptoParams.setInitializationVector(initVector);
           cryptoParams.setEncryptedInputStream(boundedRangeFileInputStream);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2d36f885/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java
new file mode 100644
index 0000000..4a8e0ed
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java
@@ -0,0 +1,205 @@
+/*
+ * 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.accumulo.core.security.crypto;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.security.InvalidKeyException;
+import java.security.Key;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+
+import javax.crypto.Cipher;
+import javax.crypto.IllegalBlockSizeException;
+import javax.crypto.spec.SecretKeySpec;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+
+public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncryptionStrategy {
+  
+  private static final Logger log = Logger.getLogger(CachingHDFSSecretKeyEncryptionStrategy.class);
+  private SecretKeyCache secretKeyCache = new SecretKeyCache();
+  
+  
+  @Override
+  public CryptoModuleParameters encryptSecretKey(CryptoModuleParameters context) {
+    try {
+      secretKeyCache.ensureSecretKeyCacheInitialized(context);
+      doKeyEncryptionOperation(Cipher.WRAP_MODE, context);
+    } catch (IOException e) {
+      log.error(e);
+      throw new RuntimeException(e);
+    }
+    return context;
+  }
+  
+  @Override
+  public CryptoModuleParameters decryptSecretKey(CryptoModuleParameters context) {
+    try {
+      secretKeyCache.ensureSecretKeyCacheInitialized(context);
+      doKeyEncryptionOperation(Cipher.UNWRAP_MODE, context);
+    } catch (IOException e) {
+      log.error(e);
+      throw new RuntimeException(e);
+    }
+    return context;
+  }
+  
+  private void doKeyEncryptionOperation(int encryptionMode, CryptoModuleParameters params) throws IOException {
+    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_CIPHER_SUITE.getKey()));
+
+    try {
+      cipher.init(encryptionMode, new SecretKeySpec(secretKeyCache.getKeyEncryptionKey(), params.getAlgorithmName()));
+    } catch (InvalidKeyException e) {
+      log.error(e);
+      throw new RuntimeException(e);
+    }      
+    
+    if (Cipher.UNWRAP_MODE == encryptionMode) {
+      try {
+        Key plaintextKey = cipher.unwrap(params.getEncryptedKey(), params.getAlgorithmName(), Cipher.SECRET_KEY);
+        params.setPlaintextKey(plaintextKey.getEncoded());
+      } catch (InvalidKeyException e) {
+        log.error(e);
+        throw new RuntimeException(e);
+      } catch (NoSuchAlgorithmException e) {
+        log.error(e);
+        throw new RuntimeException(e);
+      }
+    } else {
+      Key plaintextKey = new SecretKeySpec(params.getPlaintextKey(), params.getAlgorithmName());
+      try {
+        byte[] encryptedSecretKey = cipher.wrap(plaintextKey);
+        params.setEncryptedKey(encryptedSecretKey);
+        params.setOpaqueKeyEncryptionKeyID(secretKeyCache.getPathToKeyName());
+      } catch (InvalidKeyException e) {
+        log.error(e);
+        throw new RuntimeException(e);
+      } catch (IllegalBlockSizeException e) {
+        log.error(e);
+        throw new RuntimeException(e);
+      }
+      
+    } 
+  }
+  
+  private static class SecretKeyCache {
+
+    private boolean initialized = false;
+    private byte[] keyEncryptionKey;
+    private String pathToKeyName;
+    
+    
+    public SecretKeyCache() {
+    };
+    
+    public synchronized void ensureSecretKeyCacheInitialized(CryptoModuleParameters context) throws IOException {
+      
+      if (initialized) {
+        return;
+      }
+      
+      // First identify if the KEK already exists
+      pathToKeyName = getFullPathToKey(context);
+      
+      if (pathToKeyName == null || pathToKeyName.equals("")) {
+        pathToKeyName = Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getDefaultValue();
+      }
+      
+      Path pathToKey = new Path(pathToKeyName);  
+      FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());   
+
+      DataInputStream in = null;
+      try {
+        if (!fs.exists(pathToKey)) {
+          initializeKeyEncryptionKey(fs, pathToKey, context);
+        }
+        
+        in = fs.open(pathToKey);
+
+        int keyEncryptionKeyLength = in.readInt();
+        keyEncryptionKey = new byte[keyEncryptionKeyLength];
+        in.read(keyEncryptionKey);
+
+        initialized = true;
+        
+      } catch (IOException e) {
+        log.error("Could not initialize key encryption cache", e);
+      } finally {
+        IOUtils.closeQuietly(in);
+      }
+    }
+    
+    private void initializeKeyEncryptionKey(FileSystem fs, Path pathToKey, CryptoModuleParameters params) throws IOException {
+      DataOutputStream out = null;
+      try {
+        out = fs.create(pathToKey);
+        // Very important, lets hedge our bets
+        fs.setReplication(pathToKey, (short) 5);
+        SecureRandom random = DefaultCryptoModuleUtils.getSecureRandom(params.getRandomNumberGenerator(), params.getRandomNumberGeneratorProvider());
+        int keyLength = params.getKeyLength();
+        byte[] newRandomKeyEncryptionKey = new byte[keyLength / 8];
+        random.nextBytes(newRandomKeyEncryptionKey);
+        out.writeInt(newRandomKeyEncryptionKey.length);
+        out.write(newRandomKeyEncryptionKey);
+        out.flush();
+      } finally {
+        if (out != null) {
+          out.close();        
+        }
+      }
+      
+    }
+
+    private String getFullPathToKey(CryptoModuleParameters params) {
+      String pathToKeyName = params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getKey());
+      String instanceDirectory = params.getAllOptions().get(Property.INSTANCE_DFS_DIR.getKey());
+      
+      
+      if (pathToKeyName == null) {
+        pathToKeyName = Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getDefaultValue();
+      }
+      
+      if (instanceDirectory == null) {
+        instanceDirectory = Property.INSTANCE_DFS_DIR.getDefaultValue();
+      }
+      
+      if (!pathToKeyName.startsWith("/")) {
+        pathToKeyName = "/" + pathToKeyName;
+      }
+      
+      String fullPath = instanceDirectory + pathToKeyName;
+      return fullPath;
+    }
+
+    public byte[] getKeyEncryptionKey() {
+      return keyEncryptionKey;
+    }
+
+    public String getPathToKeyName() {
+      return pathToKeyName;
+    }
+  }
+  
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2d36f885/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleFactory.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleFactory.java
index 6b7b193..259c13d 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleFactory.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.security.crypto;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -32,6 +33,8 @@ import org.apache.log4j.Logger;
 public class CryptoModuleFactory {
   
   private static Logger log = Logger.getLogger(CryptoModuleFactory.class);
+  private static Map<String,CryptoModule> cryptoModulesCache = new HashMap<String,CryptoModule>();
+  private static Map<String,SecretKeyEncryptionStrategy> secretKeyEncryptionStrategyCache = new HashMap<String,SecretKeyEncryptionStrategy>();
   
   /**
    * This method returns a crypto module based on settings in the given configuration parameter.
@@ -52,12 +55,27 @@ public class CryptoModuleFactory {
       cryptoModuleClassname = cryptoModuleClassname.trim();
     }
     
-    log.trace(String.format("About to instantiate crypto module %s", cryptoModuleClassname));
-    
     if (cryptoModuleClassname == null || cryptoModuleClassname.equals("NullCryptoModule")) {
       return new NullCryptoModule();
+    }   
+    
+    CryptoModule cryptoModule = null;
+    synchronized (cryptoModulesCache) {
+      if (cryptoModulesCache.containsKey(cryptoModuleClassname)) {
+        cryptoModule = cryptoModulesCache.get(cryptoModuleClassname);
+      } else {
+        cryptoModule = instantiateCryptoModule(cryptoModuleClassname);
+        cryptoModulesCache.put(cryptoModuleClassname, cryptoModule);
+      }
     }
     
+    return cryptoModule;
+  }
+
+  @SuppressWarnings({"rawtypes"})
+  private static CryptoModule instantiateCryptoModule(String cryptoModuleClassname) {
+    log.debug(String.format("About to instantiate crypto module %s", cryptoModuleClassname));
+
     CryptoModule cryptoModule = null;
     Class cryptoModuleClazz = null;
     try {
@@ -79,13 +97,13 @@ public class CryptoModuleFactory {
     }
     
     if (!implementsCryptoModule) {
-      log.warn("Configured Accumulo crypto module \"%s\" does not implement the CryptoModule interface. No encryption will be used.");
+      log.warn("Configured Accumulo crypto module \""+cryptoModuleClassname+"\" does not implement the CryptoModule interface. No encryption will be used.");
       return new NullCryptoModule();
     } else {
       try {
         cryptoModule = (CryptoModule) cryptoModuleClazz.newInstance();
         
-        log.trace("Successfully instantiated crypto module");
+        log.debug("Successfully instantiated crypto module "+cryptoModuleClassname);
         
       } catch (InstantiationException e) {
         log.warn(String.format("Got instantiation exception %s when instantiating crypto module \"%s\".  No encryption will be used.", e.getCause().getClass()
@@ -107,7 +125,6 @@ public class CryptoModuleFactory {
     return getSecretKeyEncryptionStrategy(className);
   }
   
-  @SuppressWarnings("rawtypes")
   public static SecretKeyEncryptionStrategy getSecretKeyEncryptionStrategy(String className) {
     
     if (className != null) {
@@ -119,6 +136,24 @@ public class CryptoModuleFactory {
     }
     
     SecretKeyEncryptionStrategy strategy = null;
+    synchronized (secretKeyEncryptionStrategyCache) {
+      if (secretKeyEncryptionStrategyCache.containsKey(className)) {
+        strategy = secretKeyEncryptionStrategyCache.get(className);
+      } else {
+        strategy = instantiateSecreteKeyEncryptionStrategy(className); 
+        secretKeyEncryptionStrategyCache.put(className, strategy);
+      }
+    }
+    
+    return strategy;
+  }
+
+  @SuppressWarnings("rawtypes")
+  private static SecretKeyEncryptionStrategy instantiateSecreteKeyEncryptionStrategy(String className) {
+    
+    log.debug("About to instantiate secret key encryption strategy "+className);
+    
+    SecretKeyEncryptionStrategy strategy = null;
     Class keyEncryptionStrategyClazz = null;
     try {
       keyEncryptionStrategyClazz = AccumuloVFSClassLoader.loadClass(className);
@@ -145,7 +180,7 @@ public class CryptoModuleFactory {
       try {
         strategy = (SecretKeyEncryptionStrategy) keyEncryptionStrategyClazz.newInstance();
         
-        log.trace("Successfully instantiated secret key encryption strategy");
+        log.debug("Successfully instantiated secret key encryption strategy "+className);
         
       } catch (InstantiationException e) {
         log.warn(String.format("Got instantiation exception %s when instantiating secret key encryption strategy \"%s\".  No encryption will be used.", e
@@ -159,7 +194,6 @@ public class CryptoModuleFactory {
         return new NullSecretKeyEncryptionStrategy();
       }
     }
-    
     return strategy;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2d36f885/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModule.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModule.java
index 905eb6e..d14a424 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModule.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModule.java
@@ -320,7 +320,7 @@ public class DefaultCryptoModule implements CryptoModule {
         // Read the secret key and initialization vector from the file
         int initVectorLength = dataIn.readInt();
         byte[] initVector = new byte[initVectorLength];
-        dataIn.read(initVector, 0, initVectorLength);
+        dataIn.readFully(initVector);
         
         params.setInitializationVector(initVector);
         
@@ -330,7 +330,7 @@ public class DefaultCryptoModule implements CryptoModule {
         
         int encryptedSecretKeyLength = dataIn.readInt();
         byte[] encryptedSecretKey = new byte[encryptedSecretKeyLength]; 
-        dataIn.read(encryptedSecretKey);
+        dataIn.readFully(encryptedSecretKey);
         params.setEncryptedKey(encryptedSecretKey);