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

mmiller pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new b454e0c1e4 Improvements to AESCryptoService (#2895)
b454e0c1e4 is described below

commit b454e0c1e4ff0bde55ad9e7a6d9ced517d1c3943
Author: Mike Miller <mmil...@apache.org>
AuthorDate: Fri Aug 26 13:47:55 2022 +0000

    Improvements to AESCryptoService (#2895)
    
    * Reduce Cipher.getInstance calls to once per file vs once per block
    * Move wrap cipher to ThreadLocal objects
    * Add logger and init checks
---
 .../accumulo/core/spi/crypto/AESCryptoService.java | 148 ++++++++++++++-------
 .../apache/accumulo/core/crypto/CryptoTest.java    |  12 +-
 2 files changed, 108 insertions(+), 52 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java 
b/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
index cfe8484b33..bc223f5557 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
@@ -55,6 +55,8 @@ import 
org.apache.accumulo.core.crypto.streams.BlockedOutputStream;
 import org.apache.accumulo.core.crypto.streams.DiscardCloseOutputStream;
 import org.apache.accumulo.core.crypto.streams.RFileCipherOutputStream;
 import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -62,6 +64,9 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * Example implementation of AES encryption for Accumulo
  */
 public class AESCryptoService implements CryptoService {
+  private static final Logger log = 
LoggerFactory.getLogger(AESCryptoService.class);
+  private volatile boolean initialized = false;
+
   // properties required for using this service
   private static final String CRYPTO_PREFIX = "instance.crypto.opts.";
   private static final String KEY_URI = CRYPTO_PREFIX + "key.uri";
@@ -85,8 +90,35 @@ public class AESCryptoService implements CryptoService {
 
   private static final FileEncrypter DISABLED = new NoFileEncrypter();
 
+  private static final ThreadLocal<Cipher> KEY_WRAP_CIPHER = new 
ThreadLocal<Cipher>() {
+    @SuppressFBWarnings(value = "CIPHER_INTEGRITY",
+        justification = "integrity not needed for key wrap")
+    @Override
+    protected Cipher initialValue() {
+      try {
+        return Cipher.getInstance(KEY_WRAP_TRANSFORM);
+      } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
+        throw new CryptoException("Error creating Cipher for AESWrap", e);
+      }
+    }
+  };
+
+  private static final ThreadLocal<Cipher> KEY_UNWRAP_CIPHER = new 
ThreadLocal<Cipher>() {
+    @SuppressFBWarnings(value = "CIPHER_INTEGRITY",
+        justification = "integrity not needed for key wrap")
+    @Override
+    protected Cipher initialValue() {
+      try {
+        return Cipher.getInstance(KEY_WRAP_TRANSFORM);
+      } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
+        throw new CryptoException("Error creating Cipher for AESWrap", e);
+      }
+    }
+  };
+
   @Override
   public void init(Map<String,String> conf) throws CryptoException {
+    ensureNotInit();
     String keyLocation =
         Objects.requireNonNull(conf.get(KEY_URI), "Config property " + KEY_URI 
+ " is required.");
     String enabledProp = conf.get(ENCRYPT_ENABLED);
@@ -107,10 +139,13 @@ public class AESCryptoService implements CryptoService {
     }
     Objects.requireNonNull(this.encryptingKek,
         "Encrypting Key Encryption Key was null, init failed");
+    log.debug("Successfully initialized crypto Key from {}", keyLocation);
+    initialized = true;
   }
 
   @Override
   public FileEncrypter getFileEncrypter(CryptoEnvironment environment) {
+    ensureInit();
     if (!encryptEnabled) {
       return DISABLED;
     }
@@ -131,6 +166,7 @@ public class AESCryptoService implements CryptoService {
 
   @Override
   public FileDecrypter getFileDecrypter(CryptoEnvironment environment) {
+    ensureInit();
     CryptoModule cm;
     byte[] decryptionParams = environment.getDecryptionParams();
     if (decryptionParams == null || checkNoCrypto(decryptionParams))
@@ -205,7 +241,6 @@ public class AESCryptoService implements CryptoService {
   private static byte[] createCryptoParameters(String version, Key 
encryptingKek,
       String encryptingKekId, String encryptingKeyManager, Key fek) {
 
-    byte[] bytes;
     try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream params = new DataOutputStream(baos)) {
       params.writeUTF(AESCryptoService.class.getName());
@@ -216,11 +251,11 @@ public class AESCryptoService implements CryptoService {
       params.writeInt(wrappedFek.length);
       params.write(wrappedFek);
 
-      bytes = baos.toByteArray();
+      params.flush();
+      return baos.toByteArray();
     } catch (IOException e) {
       throw new CryptoException("Error creating crypto params", e);
     }
-    return bytes;
   }
 
   private static ParsedCryptoParameters parseCryptoParameters(byte[] 
parameters) {
@@ -311,11 +346,20 @@ public class AESCryptoService implements CryptoService {
       private final byte[] firstInitVector;
       private final Key fek;
       private final byte[] initVector = new byte[GCM_IV_LENGTH_IN_BYTES];
+      private final Cipher cipher;
+      private final byte[] decryptionParameters;
 
       AESGCMFileEncrypter() {
+        try {
+          cipher = Cipher.getInstance(transformation);
+        } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
+          throw new CryptoException("Error obtaining cipher for transform " + 
transformation, e);
+        }
         this.fek = generateKey(random, KEY_LENGTH_IN_BYTES);
         random.nextBytes(this.initVector);
         this.firstInitVector = Arrays.copyOf(this.initVector, 
this.initVector.length);
+        this.decryptionParameters =
+            createCryptoParameters(VERSION, encryptingKek, keyLocation, 
keyManager, fek);
       }
 
       @Override
@@ -327,8 +371,7 @@ public class AESCryptoService implements CryptoService {
         incrementIV(initVector, initVector.length - 1);
         if (Arrays.equals(initVector, firstInitVector)) {
           ivReused = true; // This will allow us to write the final block, 
since the
-          // initialization vector
-          // is always incremented before use.
+          // initialization vector is always incremented before use.
         }
 
         // write IV before encrypting
@@ -338,13 +381,10 @@ public class AESCryptoService implements CryptoService {
           throw new CryptoException("Unable to write IV to stream", e);
         }
 
-        Cipher cipher;
         try {
-          cipher = Cipher.getInstance(transformation);
           cipher.init(Cipher.ENCRYPT_MODE, fek,
               new GCMParameterSpec(GCM_TAG_LENGTH_IN_BITS, initVector));
-        } catch (NoSuchAlgorithmException | NoSuchPaddingException | 
InvalidKeyException
-            | InvalidAlgorithmParameterException e) {
+        } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
           throw new CryptoException("Unable to initialize cipher", e);
         }
 
@@ -353,8 +393,7 @@ public class AESCryptoService implements CryptoService {
         // Prevent underlying stream from being closed with 
DiscardCloseOutputStream
         // Without this, when the crypto stream is closed (in order to flush 
its last bytes)
         // the underlying RFile stream will *also* be closed, and that's 
undesirable as the
-        // cipher
-        // stream is closed for every block written.
+        // cipher stream is closed for every block written.
         return new BlockedOutputStream(cos, cipher.getBlockSize(), 1024);
       }
 
@@ -381,14 +420,20 @@ public class AESCryptoService implements CryptoService {
 
       @Override
       public byte[] getDecryptionParameters() {
-        return createCryptoParameters(VERSION, encryptingKek, keyLocation, 
keyManager, fek);
+        return decryptionParameters;
       }
     }
 
     public class AESGCMFileDecrypter implements FileDecrypter {
+      private final Cipher cipher;
       private final Key fek;
 
       AESGCMFileDecrypter(Key fek) {
+        try {
+          cipher = Cipher.getInstance(transformation);
+        } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
+          throw new CryptoException("Error obtaining cipher for transform " + 
transformation, e);
+        }
         this.fek = fek;
       }
 
@@ -401,14 +446,11 @@ public class AESCryptoService implements CryptoService {
           throw new CryptoException("Unable to read IV from stream", e);
         }
 
-        Cipher cipher;
         try {
-          cipher = Cipher.getInstance(transformation);
           cipher.init(Cipher.DECRYPT_MODE, fek,
               new GCMParameterSpec(GCM_TAG_LENGTH_IN_BITS, initVector));
 
-        } catch (NoSuchAlgorithmException | NoSuchPaddingException | 
InvalidKeyException
-            | InvalidAlgorithmParameterException e) {
+        } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
           throw new CryptoException("Unable to initialize cipher", e);
         }
 
@@ -445,13 +487,24 @@ public class AESCryptoService implements CryptoService {
 
     @SuppressFBWarnings(value = "CIPHER_INTEGRITY", justification = "CBC is 
provided for WALs")
     public class AESCBCFileEncrypter implements FileEncrypter {
+      private final Cipher cipher;
+      private final Key fek;
+      private final byte[] initVector = new byte[IV_LENGTH_IN_BYTES];
+      private final byte[] decryptionParameters;
 
-      private Key fek = generateKey(random, KEY_LENGTH_IN_BYTES);
-      private byte[] initVector = new byte[IV_LENGTH_IN_BYTES];
+      AESCBCFileEncrypter() {
+        try {
+          cipher = Cipher.getInstance(transformation);
+        } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
+          throw new CryptoException("Error obtaining cipher for transform " + 
transformation, e);
+        }
+        this.fek = generateKey(random, KEY_LENGTH_IN_BYTES);
+        this.decryptionParameters =
+            createCryptoParameters(VERSION, encryptingKek, keyLocation, 
keyManager, fek);
+      }
 
       @Override
       public OutputStream encryptStream(OutputStream outputStream) throws 
CryptoException {
-
         random.nextBytes(initVector);
         try {
           outputStream.write(initVector);
@@ -459,12 +512,9 @@ public class AESCryptoService implements CryptoService {
           throw new CryptoException("Unable to write IV to stream", e);
         }
 
-        Cipher cipher;
         try {
-          cipher = Cipher.getInstance(transformation);
           cipher.init(Cipher.ENCRYPT_MODE, fek, new 
IvParameterSpec(initVector));
-        } catch (NoSuchAlgorithmException | NoSuchPaddingException | 
InvalidKeyException
-            | InvalidAlgorithmParameterException e) {
+        } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
           throw new CryptoException("Unable to initialize cipher", e);
         }
 
@@ -474,15 +524,21 @@ public class AESCryptoService implements CryptoService {
 
       @Override
       public byte[] getDecryptionParameters() {
-        return createCryptoParameters(VERSION, encryptingKek, keyLocation, 
keyManager, fek);
+        return decryptionParameters;
       }
     }
 
     @SuppressFBWarnings(value = "CIPHER_INTEGRITY", justification = "CBC is 
provided for WALs")
     public class AESCBCFileDecrypter implements FileDecrypter {
+      private final Cipher cipher;
       private final Key fek;
 
       AESCBCFileDecrypter(Key fek) {
+        try {
+          cipher = Cipher.getInstance(transformation);
+        } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
+          throw new CryptoException("Error obtaining cipher for transform " + 
transformation, e);
+        }
         this.fek = fek;
       }
 
@@ -495,12 +551,9 @@ public class AESCryptoService implements CryptoService {
           throw new CryptoException("Unable to read IV from stream", e);
         }
 
-        Cipher cipher;
         try {
-          cipher = Cipher.getInstance(transformation);
           cipher.init(Cipher.DECRYPT_MODE, fek, new 
IvParameterSpec(initVector));
-        } catch (NoSuchAlgorithmException | NoSuchPaddingException | 
InvalidKeyException
-            | InvalidAlgorithmParameterException e) {
+        } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
           throw new CryptoException("Unable to initialize cipher", e);
         }
 
@@ -516,48 +569,45 @@ public class AESCryptoService implements CryptoService {
     return new SecretKeySpec(bytes, "AES");
   }
 
-  @SuppressFBWarnings(value = "CIPHER_INTEGRITY",
-      justification = "integrity not needed for key wrap")
   public static Key unwrapKey(byte[] fek, Key kek) {
-    Key result = null;
     try {
-      Cipher c = Cipher.getInstance(KEY_WRAP_TRANSFORM);
+      final Cipher c = KEY_UNWRAP_CIPHER.get();
       c.init(Cipher.UNWRAP_MODE, kek);
-      result = c.unwrap(fek, "AES", Cipher.SECRET_KEY);
-    } catch (InvalidKeyException | NoSuchAlgorithmException | 
NoSuchPaddingException e) {
+      return c.unwrap(fek, "AES", Cipher.SECRET_KEY);
+    } catch (InvalidKeyException | NoSuchAlgorithmException e) {
       throw new CryptoException("Unable to unwrap file encryption key", e);
     }
-    return result;
   }
 
-  @SuppressFBWarnings(value = "CIPHER_INTEGRITY",
-      justification = "integrity not needed for key wrap")
   public static byte[] wrapKey(Key fek, Key kek) {
-    byte[] result = null;
     try {
-      Cipher c = Cipher.getInstance(KEY_WRAP_TRANSFORM);
+      final Cipher c = KEY_WRAP_CIPHER.get();
       c.init(Cipher.WRAP_MODE, kek);
-      result = c.wrap(fek);
-    } catch (InvalidKeyException | NoSuchAlgorithmException | 
NoSuchPaddingException
-        | IllegalBlockSizeException e) {
+      return c.wrap(fek);
+    } catch (InvalidKeyException | IllegalBlockSizeException e) {
       throw new CryptoException("Unable to wrap file encryption key", e);
     }
-
-    return result;
   }
 
   @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "keyId 
specified by admin")
   public static Key loadKekFromUri(String keyId) {
-    java.net.URI uri;
-    SecretKeySpec key = null;
     try {
-      uri = new URI(keyId);
-      key = new SecretKeySpec(Files.readAllBytes(Paths.get(uri.getPath())), 
"AES");
+      final java.net.URI uri = new URI(keyId);
+      return new SecretKeySpec(Files.readAllBytes(Paths.get(uri.getPath())), 
"AES");
     } catch (URISyntaxException | IOException | IllegalArgumentException e) {
       throw new CryptoException("Unable to load key encryption key.", e);
     }
+  }
 
-    return key;
+  private void ensureInit() {
+    if (!initialized) {
+      throw new IllegalStateException("This Crypto Service has not been 
initialized.");
+    }
+  }
 
+  private void ensureNotInit() {
+    if (initialized) {
+      throw new IllegalStateException("This Crypto Service has already been 
initialized.");
+    }
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java 
b/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
index ce1699ae80..05dd902470 100644
--- a/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
@@ -203,7 +203,7 @@ public class CryptoTest {
     decrypt(encryptedBytes, Scope.WAL, ConfigMode.CRYPTO_ON_DISABLED);
 
     // make sure we don't encrypt when disabled
-    byte[] plainBytes = encrypt(cs, Scope.WAL, ConfigMode.CRYPTO_ON_DISABLED);
+    byte[] plainBytes = encrypt(cs, Scope.WAL, ConfigMode.CRYPTO_ON_DISABLED, 
false);
     String stringPlainBytes = Arrays.toString(plainBytes);
     assertNotEquals(stringEncryptedBytes, stringPlainBytes);
     decrypt(plainBytes, Scope.WAL, ConfigMode.CRYPTO_ON_DISABLED);
@@ -236,7 +236,7 @@ public class CryptoTest {
     decrypt(encryptedBytes, Scope.RFILE, ConfigMode.CRYPTO_ON_DISABLED);
 
     // make sure we don't encrypt when disabled
-    byte[] plainBytes = encrypt(cs, Scope.RFILE, 
ConfigMode.CRYPTO_ON_DISABLED);
+    byte[] plainBytes = encrypt(cs, Scope.RFILE, 
ConfigMode.CRYPTO_ON_DISABLED, false);
     String stringPlainBytes = Arrays.toString(plainBytes);
     assertNotEquals(stringEncryptedBytes, stringPlainBytes);
     decrypt(plainBytes, Scope.RFILE, ConfigMode.CRYPTO_ON_DISABLED);
@@ -428,8 +428,14 @@ public class CryptoTest {
 
   private <C extends CryptoService> byte[] encrypt(C cs, Scope scope, 
ConfigMode configMode)
       throws Exception {
+    return encrypt(cs, scope, configMode, true);
+  }
+
+  private <C extends CryptoService> byte[] encrypt(C cs, Scope scope, 
ConfigMode configMode,
+      boolean init) throws Exception {
     AccumuloConfiguration conf = getAccumuloConfig(configMode);
-    cs.init(conf.getAllPropertiesWithPrefix(Property.INSTANCE_CRYPTO_PREFIX));
+    if (init)
+      
cs.init(conf.getAllPropertiesWithPrefix(Property.INSTANCE_CRYPTO_PREFIX));
     CryptoEnvironmentImpl env = new CryptoEnvironmentImpl(scope, null);
     FileEncrypter encrypter = cs.getFileEncrypter(env);
     byte[] params = encrypter.getDecryptionParameters();

Reply via email to