accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mmil...@apache.org
Subject [accumulo] branch master updated: Rename some Crypto terms for easier configuration (#640)
Date Mon, 10 Sep 2018 22:42:05 GMT
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/master by this push:
     new c36e741  Rename some Crypto terms for easier configuration (#640)
c36e741 is described below

commit c36e741ff8cb316467f70c3650fe0380ad410655
Author: Mike Miller <mmiller@apache.org>
AuthorDate: Mon Sep 10 18:42:00 2018 -0400

    Rename some Crypto terms for easier configuration (#640)
    
    * Changed instance.crypto.opts.kekId to instance.crypto.opts.key.location
    * Changed instance.crypto.opts.keyManager to instance.crypto.opts.key.provider
---
 .../security/crypto/impl/AESCryptoService.java     | 74 ++++++++++------------
 .../impl/{KeyManager.java => AESKeyUtils.java}     |  8 ++-
 core/src/main/spotbugs/exclude-filter.xml          |  4 +-
 .../accumulo/core/security/crypto/CryptoTest.java  | 20 +++---
 .../test/resources/crypto-on-accumulo.properties   |  4 +-
 .../test/functional/WriteAheadLogEncryptedIT.java  |  4 +-
 6 files changed, 55 insertions(+), 59 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
b/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
index 54d5f59..d10dbff 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
@@ -62,25 +62,27 @@ public class AESCryptoService implements CryptoService {
   private static final String NO_CRYPTO_VERSION = "U+1F47B";
 
   private Key encryptingKek = null;
-  private String encryptingKekId = null;
-  private String encryptingKeyManager = null;
+  private String keyLocation = null;
+  private String keyManager = null;
   // Lets just load keks for reading once
   private HashMap<String,Key> decryptingKeys = null;
   private SecureRandom sr = null;
 
   @Override
   public void init(Map<String,String> conf) throws CryptoException {
-    String kekId = conf.get("instance.crypto.opts.kekId");
-    String keyMgr = conf.get("instance.crypto.opts.keyManager");
-    Objects.requireNonNull(kekId, "Config property instance.crypto.opts.kekId is required.");
-    Objects.requireNonNull(keyMgr, "Config property instance.crypto.opts.keyManager is required.");
+    String keyLocation = conf.get("instance.crypto.opts.key.location");
+    String keyMgr = conf.get("instance.crypto.opts.key.provider");
+    Objects.requireNonNull(keyLocation,
+        "Config property instance.crypto.opts.key.location is required.");
+    Objects.requireNonNull(keyMgr,
+        "Config property instance.crypto.opts.key.provider is required.");
     this.sr = CryptoUtils.newSha1SecureRandom();
     this.decryptingKeys = new HashMap<>();
     switch (keyMgr) {
-      case KeyManager.URI:
-        this.encryptingKeyManager = keyMgr;
-        this.encryptingKekId = kekId;
-        this.encryptingKek = KeyManager.loadKekFromUri(kekId);
+      case AESKeyUtils.URI:
+        this.keyManager = keyMgr;
+        this.keyLocation = keyLocation;
+        this.encryptingKek = AESKeyUtils.loadKekFromUri(keyLocation);
         break;
       default:
         throw new CryptoException("Unrecognized key manager");
@@ -93,13 +95,11 @@ public class AESCryptoService implements CryptoService {
     CryptoModule cm;
     switch (environment.getScope()) {
       case WAL:
-        cm = new AESCBCCryptoModule(this.encryptingKek, this.encryptingKekId,
-            this.encryptingKeyManager);
+        cm = new AESCBCCryptoModule(this.encryptingKek, this.keyLocation, this.keyManager);
         return cm.getEncrypter();
 
       case RFILE:
-        cm = new AESGCMCryptoModule(this.encryptingKek, this.encryptingKekId,
-            this.encryptingKeyManager);
+        cm = new AESGCMCryptoModule(this.encryptingKek, this.keyLocation, this.keyManager);
         return cm.getEncrypter();
 
       default:
@@ -116,15 +116,13 @@ public class AESCryptoService implements CryptoService {
 
     ParsedCryptoParameters parsed = parseCryptoParameters(decryptionParams);
     Key kek = loadDecryptionKek(parsed);
-    Key fek = KeyManager.unwrapKey(parsed.getEncFek(), kek);
+    Key fek = AESKeyUtils.unwrapKey(parsed.getEncFek(), kek);
     switch (parsed.getCryptoServiceVersion()) {
       case AESCBCCryptoModule.VERSION:
-        cm = new AESCBCCryptoModule(this.encryptingKek, this.encryptingKekId,
-            this.encryptingKeyManager);
+        cm = new AESCBCCryptoModule(this.encryptingKek, this.keyLocation, this.keyManager);
         return (cm.getDecrypter(fek));
       case AESGCMCryptoModule.VERSION:
-        cm = new AESGCMCryptoModule(this.encryptingKek, this.encryptingKekId,
-            this.encryptingKeyManager);
+        cm = new AESGCMCryptoModule(this.encryptingKek, this.keyLocation, this.keyManager);
         return (cm.getDecrypter(fek));
       default:
         throw new CryptoException(
@@ -196,7 +194,7 @@ public class AESCryptoService implements CryptoService {
       params.writeUTF(version);
       params.writeUTF(encryptingKeyManager);
       params.writeUTF(encryptingKekId);
-      byte[] wrappedFek = KeyManager.wrapKey(fek, encryptingKek);
+      byte[] wrappedFek = AESKeyUtils.wrapKey(fek, encryptingKek);
       params.writeInt(wrappedFek.length);
       params.write(wrappedFek);
 
@@ -235,8 +233,8 @@ public class AESCryptoService implements CryptoService {
     }
 
     switch (params.keyManagerVersion) {
-      case KeyManager.URI:
-        ret = KeyManager.loadKekFromUri(params.kekId);
+      case AESKeyUtils.URI:
+        ret = AESKeyUtils.loadKekFromUri(params.kekId);
         break;
       default:
         throw new CryptoException("Unable to load kek: " + params.kekId);
@@ -271,14 +269,13 @@ public class AESCryptoService implements CryptoService {
     private final String transformation = "AES/GCM/NoPadding";
     private boolean ivReused = false;
     private Key encryptingKek;
-    private String encryptingKekId;
-    private String encryptingKeyManager;
+    private String keyLocation;
+    private String keyManager;
 
-    public AESGCMCryptoModule(Key encryptingKek, String encryptingKekId,
-        String encryptingKeyManager) {
+    public AESGCMCryptoModule(Key encryptingKek, String keyLocation, String keyManager) {
       this.encryptingKek = encryptingKek;
-      this.encryptingKekId = encryptingKekId;
-      this.encryptingKeyManager = encryptingKeyManager;
+      this.keyLocation = keyLocation;
+      this.keyManager = keyManager;
     }
 
     @Override
@@ -298,7 +295,7 @@ public class AESCryptoService implements CryptoService {
       private byte[] initVector = new byte[GCM_IV_LENGTH_IN_BYTES];
 
       AESGCMFileEncrypter() {
-        fek = KeyManager.generateKey(sr, KEY_LENGTH_IN_BYTES);
+        fek = AESKeyUtils.generateKey(sr, KEY_LENGTH_IN_BYTES);
         sr.nextBytes(initVector);
         firstInitVector = Arrays.copyOf(initVector, initVector.length);
       }
@@ -365,8 +362,7 @@ public class AESCryptoService implements CryptoService {
 
       @Override
       public byte[] getDecryptionParameters() {
-        return createCryptoParameters(VERSION, encryptingKek, encryptingKekId, encryptingKeyManager,
-            fek);
+        return createCryptoParameters(VERSION, encryptingKek, keyLocation, keyManager, fek);
       }
     }
 
@@ -409,14 +405,13 @@ public class AESCryptoService implements CryptoService {
     private final Integer KEY_LENGTH_IN_BYTES = 16;
     private final String transformation = "AES/CBC/NoPadding";
     private Key encryptingKek;
-    private String encryptingKekId;
-    private String encryptingKeyManager;
+    private String keyLocation;
+    private String keyManager;
 
-    public AESCBCCryptoModule(Key encryptingKek, String encryptingKekId,
-        String encryptingKeyManager) {
+    public AESCBCCryptoModule(Key encryptingKek, String keyLocation, String keyManager) {
       this.encryptingKek = encryptingKek;
-      this.encryptingKekId = encryptingKekId;
-      this.encryptingKeyManager = encryptingKeyManager;
+      this.keyLocation = keyLocation;
+      this.keyManager = keyManager;
     }
 
     @Override
@@ -431,7 +426,7 @@ public class AESCryptoService implements CryptoService {
 
     public class AESCBCFileEncrypter implements FileEncrypter {
 
-      private Key fek = KeyManager.generateKey(sr, KEY_LENGTH_IN_BYTES);
+      private Key fek = AESKeyUtils.generateKey(sr, KEY_LENGTH_IN_BYTES);
       private byte[] initVector = new byte[IV_LENGTH_IN_BYTES];
 
       @Override
@@ -465,8 +460,7 @@ public class AESCryptoService implements CryptoService {
 
       @Override
       public byte[] getDecryptionParameters() {
-        return createCryptoParameters(VERSION, encryptingKek, encryptingKekId, encryptingKeyManager,
-            fek);
+        return createCryptoParameters(VERSION, encryptingKek, keyLocation, keyManager, fek);
       }
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/KeyManager.java
b/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESKeyUtils.java
similarity index 90%
rename from core/src/main/java/org/apache/accumulo/core/security/crypto/impl/KeyManager.java
rename to core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESKeyUtils.java
index f24d24e..50b9eea 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/KeyManager.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESKeyUtils.java
@@ -34,9 +34,11 @@ import javax.crypto.spec.SecretKeySpec;
 
 import org.apache.accumulo.core.spi.crypto.CryptoService.CryptoException;
 
-public class KeyManager {
+public class AESKeyUtils {
 
   public static final String URI = "uri";
+  public static final String KEY_WRAP_TRANSFORM = "AESWrap";
+  public static final String KEY_PROVIDER = "SunJCE";
 
   public static Key generateKey(SecureRandom sr, int size) {
     byte[] bytes = new byte[size];
@@ -47,7 +49,7 @@ public class KeyManager {
   public static Key unwrapKey(byte[] fek, Key kek) {
     Key result = null;
     try {
-      Cipher c = Cipher.getInstance("AESWrap", "SunJCE");
+      Cipher c = Cipher.getInstance(KEY_WRAP_TRANSFORM, KEY_PROVIDER);
       c.init(Cipher.UNWRAP_MODE, kek);
       result = c.unwrap(fek, "AES", Cipher.SECRET_KEY);
     } catch (InvalidKeyException | NoSuchAlgorithmException | NoSuchProviderException
@@ -60,7 +62,7 @@ public class KeyManager {
   public static byte[] wrapKey(Key fek, Key kek) {
     byte[] result = null;
     try {
-      Cipher c = Cipher.getInstance("AESWrap", "SunJCE");
+      Cipher c = Cipher.getInstance(KEY_WRAP_TRANSFORM, KEY_PROVIDER);
       c.init(Cipher.WRAP_MODE, kek);
       result = c.wrap(fek);
     } catch (InvalidKeyException | NoSuchAlgorithmException | NoSuchProviderException
diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml
index 5aa5421..2be74ba 100644
--- a/core/src/main/spotbugs/exclude-filter.xml
+++ b/core/src/main/spotbugs/exclude-filter.xml
@@ -38,7 +38,7 @@
       <Class name="org.apache.accumulo.core.security.crypto.CryptoTest"/>
       <Class name="org.apache.accumulo.core.security.crypto.impl.AESCryptoService$AESCBCCryptoModule$AESCBCFileEncrypter"/>
       <Class name="org.apache.accumulo.core.security.crypto.impl.AESCryptoService$AESCBCCryptoModule$AESCBCFileDecrypter"/>
-      <Class name="org.apache.accumulo.core.security.crypto.impl.KeyManager"/>
+      <Class name="org.apache.accumulo.core.security.crypto.impl.AESKeyUtils"/>
     </Or>
     <Bug code="CIPHER" pattern="CIPHER_INTEGRITY" />
   </Match>
@@ -95,7 +95,7 @@
   </Match>
   <Match>
     <!-- Calling new File on input can be dangerous, but OK here -->
-    <Class name="org.apache.accumulo.core.security.crypto.impl.KeyManager" />
+    <Class name="org.apache.accumulo.core.security.crypto.impl.AESKeyUtils" />
     <Method name="loadKekFromUri" params="java.lang.String"/>
     <Bug code="PATH" pattern="PATH_TRAVERSAL_IN" />
   </Match>
diff --git a/core/src/test/java/org/apache/accumulo/core/security/crypto/CryptoTest.java b/core/src/test/java/org/apache/accumulo/core/security/crypto/CryptoTest.java
index d5496fb..e5e3e53 100644
--- a/core/src/test/java/org/apache/accumulo/core/security/crypto/CryptoTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/crypto/CryptoTest.java
@@ -51,8 +51,8 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.crypto.impl.AESCryptoService;
+import org.apache.accumulo.core.security.crypto.impl.AESKeyUtils;
 import org.apache.accumulo.core.security.crypto.impl.CryptoEnvironmentImpl;
-import org.apache.accumulo.core.security.crypto.impl.KeyManager;
 import org.apache.accumulo.core.security.crypto.impl.NoCryptoService;
 import org.apache.accumulo.core.security.crypto.streams.NoFlushOutputStream;
 import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
@@ -272,12 +272,12 @@ public class CryptoTest {
       NoSuchPaddingException, InvalidKeyException {
     SecureRandom sr = SecureRandom.getInstance("SHA1PRNG", "SUN");
     java.security.Key key;
-    key = KeyManager.generateKey(sr, 16);
+    key = AESKeyUtils.generateKey(sr, 16);
     Cipher.getInstance("AES/CBC/NoPadding").init(Cipher.ENCRYPT_MODE, key);
 
-    key = KeyManager.generateKey(sr, 24);
-    key = KeyManager.generateKey(sr, 32);
-    key = KeyManager.generateKey(sr, 11);
+    key = AESKeyUtils.generateKey(sr, 24);
+    key = AESKeyUtils.generateKey(sr, 32);
+    key = AESKeyUtils.generateKey(sr, 11);
 
     exception.expect(InvalidKeyException.class);
     Cipher.getInstance("AES/CBC/NoPadding").init(Cipher.ENCRYPT_MODE, key);
@@ -287,17 +287,17 @@ public class CryptoTest {
   public void testKeyManagerWrapAndUnwrap()
       throws NoSuchAlgorithmException, NoSuchProviderException {
     SecureRandom sr = SecureRandom.getInstance("SHA1PRNG", "SUN");
-    java.security.Key kek = KeyManager.generateKey(sr, 16);
-    java.security.Key fek = KeyManager.generateKey(sr, 16);
-    byte[] wrapped = KeyManager.wrapKey(fek, kek);
+    java.security.Key kek = AESKeyUtils.generateKey(sr, 16);
+    java.security.Key fek = AESKeyUtils.generateKey(sr, 16);
+    byte[] wrapped = AESKeyUtils.wrapKey(fek, kek);
     assertFalse(Arrays.equals(fek.getEncoded(), wrapped));
-    java.security.Key unwrapped = KeyManager.unwrapKey(wrapped, kek);
+    java.security.Key unwrapped = AESKeyUtils.unwrapKey(wrapped, kek);
     assertEquals(unwrapped, fek);
   }
 
   @Test
   public void testKeyManagerLoadKekFromUri() throws IOException {
-    SecretKeySpec fileKey = KeyManager.loadKekFromUri("file:///tmp/testAESFile");
+    SecretKeySpec fileKey = AESKeyUtils.loadKekFromUri("file:///tmp/testAESFile");
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
     dos.writeUTF("sixteenbytekey");
diff --git a/core/src/test/resources/crypto-on-accumulo.properties b/core/src/test/resources/crypto-on-accumulo.properties
index 6623b94..52100c7 100644
--- a/core/src/test/resources/crypto-on-accumulo.properties
+++ b/core/src/test/resources/crypto-on-accumulo.properties
@@ -14,5 +14,5 @@
 # limitations under the License.
 
 instance.crypto.service=org.apache.accumulo.core.security.crypto.impl.AESCryptoService
-instance.crypto.opts.kekId=file:///tmp/testAESFile
-instance.crypto.opts.keyManager=uri
+instance.crypto.opts.key.location=file:///tmp/testAESFile
+instance.crypto.opts.key.provider=uri
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
index 1b263de..0d1e519 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
@@ -47,8 +47,8 @@ public class WriteAheadLogEncryptedIT extends AccumuloClusterHarness {
         + "/target/mini-tests/WriteAheadLogEncryptedIT-testkeyfile";
     cfg.setProperty(Property.INSTANCE_CRYPTO_SERVICE,
         "org.apache.accumulo.core.security.crypto.impl.AESCryptoService");
-    cfg.setProperty(INSTANCE_CRYPTO_PREFIX.getKey() + "kekId", keyPath);
-    cfg.setProperty(INSTANCE_CRYPTO_PREFIX.getKey() + "keyManager", "uri");
+    cfg.setProperty(INSTANCE_CRYPTO_PREFIX.getKey() + "key.location", keyPath);
+    cfg.setProperty(INSTANCE_CRYPTO_PREFIX.getKey() + "key.provider", "uri");
 
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "2M");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");


Mime
View raw message