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 2017/11/30 19:05:25 UTC

[accumulo] branch master updated: ACCUMULO-4740 Enable GCM mode for crypto

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

kturner 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 03139a3  ACCUMULO-4740 Enable GCM mode for crypto
03139a3 is described below

commit 03139a35f39b54f4bb57a4c26fba105d1dfae1f5
Author: Nick Felts <31...@users.noreply.github.com>
AuthorDate: Wed Nov 15 09:38:15 2017 -0500

    ACCUMULO-4740 Enable GCM mode for crypto
    
    Introduced the GCMParameterSpec constructor required by cipher
    Updated IV management for AES-GCM (see Appendix A of NIST SP 800-38D)
---
 .../accumulo/core/conf/ConfigSanityCheck.java      |   2 +
 .../org/apache/accumulo/core/conf/Property.java    |  13 +-
 .../accumulo/core/file/rfile/bcfile/BCFile.java    |   5 +-
 .../security/crypto/CryptoModuleParameters.java    |  48 ++++-
 .../core/security/crypto/DefaultCryptoModule.java  |  73 +++++--
 .../accumulo/core/security/crypto/CryptoTest.java  | 221 ++++++++++++++++++++-
 .../src/test/resources/crypto-on-accumulo-site.xml |   8 +-
 .../crypto-on-no-key-encryption-accumulo-site.xml  |   4 +
 .../org/apache/accumulo/tserver/log/DfsLogger.java |   3 +-
 9 files changed, 333 insertions(+), 44 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
index baf1818..f787d5e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
@@ -81,6 +81,8 @@ public class ConfigSanityCheck {
 
       if (key.equals(Property.CRYPTO_CIPHER_SUITE.getKey())) {
         cipherSuite = Objects.requireNonNull(value);
+        Preconditions.checkArgument(cipherSuite.equals("NullCipher") || cipherSuite.split("/").length == 3,
+            "Cipher suite must be NullCipher or in the form algorithm/mode/padding. Suite: " + cipherSuite + " is invalid.");
       }
 
       if (key.equals(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey())) {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index f09bd95..79e7b17 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -49,14 +49,13 @@ public enum Property {
           + "(future) other parts of the code."),
   @Experimental
   CRYPTO_CIPHER_SUITE("crypto.cipher.suite", "NullCipher", PropertyType.STRING,
-      "Describes the cipher suite to use for rfile encryption. If a WAL cipher suite is not set, it will default to this value. The suite should be in the "
-          + "form of algorithm/mode/padding, e.g. AES/CBC/NoPadding"),
+      "Describes the cipher suite to use for rfile encryption. The value must be either NullCipher or in the form of algorithm/mode/padding, "
+          + "e.g. AES/CBC/NoPadding"),
   @Experimental
-  CRYPTO_WAL_CIPHER_SUITE(
-      "crypto.wal.cipher.suite",
-      "NullCipher",
-      PropertyType.STRING,
-      "Describes the cipher suite to use for the write-ahead log. Defaults to 'cyrpto.cipher.suite' and will use that value for WAL encryption unless otherwise specified."),
+  CRYPTO_WAL_CIPHER_SUITE("crypto.wal.cipher.suite", "", PropertyType.STRING,
+      "Describes the cipher suite to use for the write-ahead log. Defaults to 'cyrpto.cipher.suite' "
+          + "and will use that value for WAL encryption unless otherwise specified. Valid suite values include: an empty string, NullCipher, or a string the "
+          + "form of algorithm/mode/padding, e.g. AES/CBC/NOPadding"),
   @Experimental
   CRYPTO_CIPHER_KEY_ALGORITHM_NAME("crypto.cipher.key.algorithm.name", "NullCipher", PropertyType.STRING,
       "States the name of the algorithm used for the key for the corresponding cipher suite. The key type must be compatible with the cipher suite."),
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 f9a61a7..e745583 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
@@ -160,9 +160,8 @@ public final class BCFile {
         // *This* is also very important. We don't want the underlying stream messed with.
         cryptoParams.setRecordParametersToStream(false);
 
-        // It is also important to make sure we get a new initialization vector on every call in here,
-        // so set any existing one to null, in case we're reusing a parameters object for its RNG or other bits
-        cryptoParams.setInitializationVector(null);
+        // Create a new IV for the block or update an existing one in the case of GCM
+        cryptoParams.updateInitializationVector();
 
         // Initialize the cipher including generating a new IV
         cryptoParams = cryptoModule.initializeCipher(cryptoParams);
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleParameters.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleParameters.java
index 10535e8..fd210b1 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleParameters.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModuleParameters.java
@@ -570,6 +570,52 @@ public class CryptoModuleParameters {
   }
 
   /**
+   * Returns the mode from the cipher suite. Assumes the suite is in the form of algorithm/mode/padding, returns null if the cipher suite is malformed or
+   * NullCipher.
+   *
+   * @return the encryption mode from the cipher suite
+   */
+  public String getCipherSuiteEncryptionMode() {
+    String[] parts = this.cipherSuite.split("/");
+    if (parts.length == 3) {
+      return parts[1];
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Updates the initialization vector for use when the encryption mode is GCM. If the IV is not currently null, and the encryption mode is GCM, it will
+   * increment the IV instead of letting the CryptoModule decide what to do.
+   */
+  public void updateInitializationVector() {
+    if (this.initializationVector != null && getCipherSuiteEncryptionMode().equals(DefaultCryptoModule.ALGORITHM_PARAMETER_SPEC_GCM)) {
+      incrementIV(this.initializationVector, this.initializationVector.length - 1);
+    } else {
+      this.initializationVector = null;
+    }
+  }
+
+  /**
+   * Because IVs can be longer than longs, this increments arbitrarily sized byte arrays by 1, with a roll over to 0 after the max value is reached.
+   *
+   * @param iv
+   *          The iv to be incremented
+   * @param i
+   *          The current byte being incremented
+   */
+  static void incrementIV(byte[] iv, int i) {
+    iv[i]++;
+    if (iv[i] == 0) {
+      if (i != 0) {
+        incrementIV(iv, i - 1);
+      } else
+        return;
+    }
+
+  }
+
+  /**
    * Gets the overall set of options for the {@link CryptoModule}.
    *
    * @see CryptoModuleParameters#setAllOptions(Map)
@@ -617,7 +663,7 @@ public class CryptoModuleParameters {
 
   private Cipher cipher;
   private SecureRandom secureRandom;
-  private byte[] initializationVector;
+  private byte[] initializationVector = null;
 
   private Map<String,String> allOptions;
   private int blockStreamSize;
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 c97e7f5..f0d0a60 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
@@ -30,6 +30,7 @@ import java.util.Map;
 
 import javax.crypto.Cipher;
 import javax.crypto.CipherInputStream;
+import javax.crypto.spec.GCMParameterSpec;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 
@@ -50,6 +51,17 @@ public class DefaultCryptoModule implements CryptoModule {
 
   private static final String ENCRYPTION_HEADER_MARKER_V1 = "---Log File Encrypted (v1)---";
   private static final String ENCRYPTION_HEADER_MARKER_V2 = "---Log File Encrypted (v2)---";
+  public static final String ALGORITHM_PARAMETER_SPEC_GCM = "GCM";
+
+  // 128-bit tags are the longest available for GCM
+  private static final Integer GCM_TAG_LENGTH_IN_BYTES = 16;
+
+  /*
+   * According to NIST Special Publication 800-38D, Section 5.2.1.1: "For IVs, it is recommended that implementations restrict support to the length of 96 bits,
+   * to promote interoperability, efficiency, and simplicity of design"
+   */
+  private static final Integer GCM_IV_LENGTH_IN_BYTES = 12;
+
   private static final Logger log = LoggerFactory.getLogger(DefaultCryptoModule.class);
 
   public DefaultCryptoModule() {}
@@ -69,26 +81,21 @@ public class DefaultCryptoModule implements CryptoModule {
     Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite(), params.getSecurityProvider());
 
     if (params.getInitializationVector() == null) {
-      try {
-        cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()), params.getSecureRandom());
-      } catch (InvalidKeyException e) {
-        log.error("Accumulo encountered an unknown error in generating the secret key object (SecretKeySpec) for an encrypted stream");
-        throw new RuntimeException(e);
+      if (params.getCipherSuiteEncryptionMode().equals(ALGORITHM_PARAMETER_SPEC_GCM)) {
+        byte[] gcmIV = new byte[GCM_IV_LENGTH_IN_BYTES];
+        params.getSecureRandom().nextBytes(gcmIV);
+        params.setInitializationVector(gcmIV);
       }
+    }
 
-      params.setInitializationVector(cipher.getIV());
-
-    } else {
-      try {
-        cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()),
-            new IvParameterSpec(params.getInitializationVector()));
-      } catch (InvalidKeyException e) {
-        log.error("Accumulo encountered an unknown error in generating the secret key object (SecretKeySpec) for an encrypted stream");
-        throw new RuntimeException(e);
-      } catch (InvalidAlgorithmParameterException e) {
-        log.error("Accumulo encountered an unknown error in setting up the initialization vector for an encrypted stream");
-        throw new RuntimeException(e);
-      }
+    try {
+      initCipher(params, cipher, Cipher.ENCRYPT_MODE);
+    } catch (InvalidKeyException e) {
+      log.error("Accumulo encountered an unknown error in generating the secret key object (SecretKeySpec) for an encrypted stream");
+      throw new RuntimeException(e);
+    } catch (InvalidAlgorithmParameterException e) {
+      log.error("Accumulo encountered an unknown error in setting up the initialization vector for an encrypted stream");
+      throw new RuntimeException(e);
     }
 
     params.setCipher(cipher);
@@ -370,8 +377,7 @@ public class DefaultCryptoModule implements CryptoModule {
     Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite(), params.getSecurityProvider());
 
     try {
-      cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()),
-          new IvParameterSpec(params.getInitializationVector()));
+      initCipher(params, cipher, Cipher.DECRYPT_MODE);
     } catch (InvalidKeyException e) {
       log.error("Error when trying to initialize cipher with secret key");
       throw new RuntimeException(e);
@@ -392,6 +398,33 @@ public class DefaultCryptoModule implements CryptoModule {
     return params;
   }
 
+  /**
+   *
+   * @param params
+   *          the crypto parameters
+   * @param cipher
+   *          the Java Cipher object to be init'd
+   * @param opMode
+   *          encrypt or decrypt
+   * @throws InvalidKeyException
+   *           if the crypto params are missing necessary values
+   * @throws InvalidAlgorithmParameterException
+   *           if an invalid algorithm is chosen
+   */
+  private void initCipher(CryptoModuleParameters params, Cipher cipher, int opMode) throws InvalidKeyException, InvalidAlgorithmParameterException {
+    if (params.getCipherSuiteEncryptionMode().equals(ALGORITHM_PARAMETER_SPEC_GCM)) {
+      cipher.init(opMode, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()),
+          new GCMParameterSpec(GCM_TAG_LENGTH_IN_BYTES * 8, params.getInitializationVector()));
+    } else {
+      if (params.getInitializationVector() == null) {
+        cipher.init(opMode, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()), params.getSecureRandom());
+        params.setInitializationVector(cipher.getIV());
+      } else {
+        cipher.init(opMode, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()), new IvParameterSpec(params.getInitializationVector()));
+      }
+    }
+  }
+
   @Override
   public CryptoModuleParameters generateNewRandomSessionKey(CryptoModuleParameters params) {
 
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 1e29757..a403f69 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
@@ -17,8 +17,11 @@
 
 package org.apache.accumulo.core.security.crypto;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayInputStream;
@@ -35,6 +38,7 @@ import java.security.SecureRandom;
 import java.util.Arrays;
 import java.util.Map.Entry;
 
+import javax.crypto.AEADBadTagException;
 import javax.crypto.BadPaddingException;
 import javax.crypto.Cipher;
 import javax.crypto.IllegalBlockSizeException;
@@ -92,8 +96,9 @@ public class CryptoTest {
     CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 
     assertNotNull(params);
-    assertEquals("AES/CFB/NoPadding", params.getCipherSuite());
+    assertEquals("AES/GCM/NoPadding", params.getCipherSuite());
     assertEquals("AES/CBC/NoPadding", params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()));
+    assertEquals("GCM", params.getCipherSuiteEncryptionMode());
     assertEquals("AES", params.getKeyAlgorithmName());
     assertEquals(128, params.getKeyLength());
     assertEquals("SHA1PRNG", params.getRandomNumberGenerator());
@@ -196,7 +201,7 @@ public class CryptoTest {
     InputStream plaintextIn = params.getPlaintextInputStream();
 
     assertNotNull(plaintextIn);
-    assertTrue(plaintextIn != in);
+    assertNotSame(plaintextIn, in);
     DataInputStream dataIn = new DataInputStream(plaintextIn);
     String markerString = dataIn.readUTF();
     int markerInt = dataIn.readInt();
@@ -213,7 +218,7 @@ public class CryptoTest {
     params = cryptoModule.getEncryptingOutputStream(params);
 
     assertNotNull(params.getEncryptedOutputStream());
-    assertTrue(params.getEncryptedOutputStream() != out);
+    assertNotSame(params.getEncryptedOutputStream(), out);
 
     DataOutputStream dataOut = new DataOutputStream(params.getEncryptedOutputStream());
     dataOut.writeUTF(MARKER_STRING);
@@ -227,8 +232,8 @@ public class CryptoTest {
     String stringifiedOtherBytes = getStringifiedBytes(MARKER_INT);
 
     // OK, let's make sure it's encrypted
-    assertTrue(!stringifiedBytes.contains(stringifiedMarkerBytes));
-    assertTrue(!stringifiedBytes.contains(stringifiedOtherBytes));
+    assertFalse(stringifiedBytes.contains(stringifiedMarkerBytes));
+    assertFalse(stringifiedBytes.contains(stringifiedOtherBytes));
     return resultingBytes;
   }
 
@@ -239,6 +244,9 @@ public class CryptoTest {
     CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(conf);
     CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 
+    // CRYPTO_ON_CONF uses AESWrap which produces wrapped keys that are too large and require a change to
+    // JCE Unlimited Strength Jurisdiction. Using AES/ECB/NoPadding should avoid this problem.
+    params.getAllOptions().put(Property.CRYPTO_DEFAULT_KEY_STRATEGY_CIPHER_SUITE.getKey(), "AES/ECB/NoPadding");
     assertTrue(cryptoModule instanceof DefaultCryptoModule);
     assertNotNull(params.getKeyEncryptionStrategyClass());
     assertEquals("org.apache.accumulo.core.security.crypto.CachingHDFSSecretKeyEncryptionStrategy", params.getKeyEncryptionStrategyClass());
@@ -342,7 +350,7 @@ public class CryptoTest {
   @Test
   public void testKeyWrapAndUnwrap() throws NoSuchAlgorithmException, NoSuchPaddingException, NoSuchProviderException, InvalidKeyException,
       IllegalBlockSizeException, BadPaddingException {
-    Cipher keyWrapCipher = Cipher.getInstance("AES/ECB/NoPadding");
+    Cipher keyWrapCipher = Cipher.getInstance("AESWrap/ECB/NoPadding");
     SecureRandom random = SecureRandom.getInstance("SHA1PRNG", "SUN");
 
     byte[] kek = new byte[16];
@@ -356,15 +364,208 @@ public class CryptoTest {
 
     byte[] wrappedKey = keyWrapCipher.wrap(randKey);
 
-    assert (wrappedKey != null);
-    assert (wrappedKey.length == randomKey.length);
+    assertNotNull(wrappedKey);
+    // AESWrap will produce 24 bytes given 128 bits of key data with a 128-bit KEK
+    assertEquals(wrappedKey.length, randomKey.length + 8);
 
-    Cipher keyUnwrapCipher = Cipher.getInstance("AES/ECB/NoPadding");
+    Cipher keyUnwrapCipher = Cipher.getInstance("AESWrap/ECB/NoPadding");
     keyUnwrapCipher.init(Cipher.UNWRAP_MODE, new SecretKeySpec(kek, "AES"));
     Key unwrappedKey = keyUnwrapCipher.unwrap(wrappedKey, "AES", Cipher.SECRET_KEY);
 
     byte[] unwrappedKeyBytes = unwrappedKey.getEncoded();
-    assert (Arrays.equals(unwrappedKeyBytes, randomKey));
+    assertArrayEquals(unwrappedKeyBytes, randomKey);
 
   }
+
+  @Test
+  public void AESGCM_Encryption_Test_Correct_Encryption_And_Decryption() throws IOException, AEADBadTagException {
+    AccumuloConfiguration conf = setAndGetAccumuloConfig(CRYPTO_ON_CONF);
+    byte[] encryptedBytes = testEncryption(conf, new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 20});
+    Integer result = testDecryption(conf, encryptedBytes);
+    assertEquals(result, Integer.valueOf(1));
+  }
+
+  @Test
+  public void AESGCM_Encryption_Test_Tag_Integrity_Compromised() throws IOException, AEADBadTagException {
+    AccumuloConfiguration conf = setAndGetAccumuloConfig(CRYPTO_ON_CONF);
+    byte[] encryptedBytes = testEncryption(conf, new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 20});
+
+    encryptedBytes[encryptedBytes.length - 1]++; // modify the tag
+    exception.expect(AEADBadTagException.class);
+    testDecryption(conf, encryptedBytes);
+    encryptedBytes[encryptedBytes.length - 1]--;
+    encryptedBytes[1486]++; // modify the data
+    exception.expect(AEADBadTagException.class);
+    testDecryption(conf, encryptedBytes);
+  }
+
+  @Test
+  public void testIVIncrements() {
+    // One byte
+    byte[] testIv1 = new byte[1];
+    // 11111110
+    testIv1[0] = (byte) 0xFE;
+
+    // 11111111
+    CryptoModuleParameters.incrementIV(testIv1, testIv1.length - 1);
+    assertArrayEquals(testIv1, new byte[] {(byte) 0xff});
+
+    // 00000000
+    CryptoModuleParameters.incrementIV(testIv1, testIv1.length - 1);
+    assertArrayEquals(testIv1, new byte[] {(byte) 0x00});
+
+    // Two bytes
+    byte[] testIv2 = new byte[2];
+    // 00000000 11111110
+    testIv2[0] = (byte) 0x00;
+    testIv2[1] = (byte) 0xFE;
+
+    // 00000000 11111111
+    CryptoModuleParameters.incrementIV(testIv2, testIv2.length - 1);
+    assertArrayEquals(testIv2, new byte[] {(byte) 0x00, (byte) 0xFF});
+
+    // 00000001 00000000
+    CryptoModuleParameters.incrementIV(testIv2, testIv2.length - 1);
+    assertArrayEquals(testIv2, new byte[] {(byte) 0x01, (byte) 0x00});
+
+    // 00000001 00000001
+    CryptoModuleParameters.incrementIV(testIv2, testIv2.length - 1);
+    assertArrayEquals(testIv2, new byte[] {(byte) 0x01, (byte) 0x01});
+
+    // 11111111 11111111
+    testIv2[0] = (byte) 0xFF;
+    testIv2[1] = (byte) 0xFF;
+
+    // 00000000 00000000
+    CryptoModuleParameters.incrementIV(testIv2, testIv2.length - 1);
+    assertArrayEquals(testIv2, new byte[] {(byte) 0x00, (byte) 0x00});
+
+    // Three bytes
+    byte[] testIv3 = new byte[3];
+    // 00000000 00000000 11111110
+    testIv3[0] = (byte) 0x00;
+    testIv3[1] = (byte) 0x00;
+    testIv3[2] = (byte) 0xFE;
+
+    // 00000000 00000000 11111111
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x00, (byte) 0x00, (byte) 0xFF});
+
+    // 00000000 00000001 00000000
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x00, (byte) 0x01, (byte) 0x00});
+
+    // 00000000 00000001 00000001
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x00, (byte) 0x01, (byte) 0x01});
+
+    // 00000000 11111111 11111110
+    testIv3[0] = (byte) 0x00;
+    testIv3[1] = (byte) 0xFF;
+    testIv3[2] = (byte) 0xFE;
+
+    // 00000000 11111111 11111111
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x00, (byte) 0xFF, (byte) 0xFF});
+
+    // 00000001 00000000 00000000
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x01, (byte) 0x00, (byte) 0x00});
+
+    // 00000001 00000000 00000001
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x01, (byte) 0x00, (byte) 0x01});
+
+    // 11111111 11111111 11111110
+    testIv3[0] = (byte) 0xFF;
+    testIv3[1] = (byte) 0xFF;
+    testIv3[2] = (byte) 0xFE;
+
+    // 11111111 11111111 11111111
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF});
+
+    // 00000000 00000000 00000000
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x00, (byte) 0x00, (byte) 0x00});
+
+    // 00000000 00000000 00000001
+    CryptoModuleParameters.incrementIV(testIv3, testIv3.length - 1);
+    assertArrayEquals(testIv3, new byte[] {(byte) 0x00, (byte) 0x00, (byte) 0x01});
+
+  }
+
+  /**
+   * Used in AESGCM unit tests to encrypt data. Uses MARKER_STRING and MARKER_INT
+   *
+   * @param conf
+   *          The accumulo configuration
+   * @param initVector
+   *          The IV to be used in encryption
+   * @return the encrypted string
+   * @throws IOException
+   *           if DataOutputStream fails
+   */
+  private static byte[] testEncryption(AccumuloConfiguration conf, byte[] initVector) throws IOException {
+    CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
+    CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(conf);
+    params.getAllOptions().put(Property.CRYPTO_WAL_CIPHER_SUITE.getKey(), "AES/GCM/NoPadding");
+    params.setInitializationVector(initVector);
+
+    /*
+     * Now lets encrypt this data!
+     */
+    ByteArrayOutputStream encryptedByteStream = new ByteArrayOutputStream();
+    params.setPlaintextOutputStream(new NoFlushOutputStream(encryptedByteStream));
+    params = cryptoModule.getEncryptingOutputStream(params);
+    DataOutputStream encryptedDataStream = new DataOutputStream(params.getEncryptedOutputStream());
+    encryptedDataStream.writeUTF(MARKER_STRING);
+    encryptedDataStream.writeInt(MARKER_INT);
+    encryptedDataStream.close();
+    byte[] encryptedBytes = encryptedByteStream.toByteArray();
+    return (encryptedBytes);
+  }
+
+  /**
+   * Used in AESGCM unit tests to decrypt data. Uses MARKER_STRING and MARKER_INT
+   *
+   * @param conf
+   *          The accumulo configuration
+   * @param encryptedBytes
+   *          The encrypted bytes
+   * @return 0 if data is incorrectly decrypted, 1 if decrypted data matches input
+   * @throws IOException
+   *           if DataInputStream fails
+   * @throws AEADBadTagException
+   *           if the encrypted stream has been modified
+   */
+  private static Integer testDecryption(AccumuloConfiguration conf, byte[] encryptedBytes) throws IOException, AEADBadTagException {
+    CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
+    CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(conf);
+    ByteArrayInputStream decryptedByteStream = new ByteArrayInputStream(encryptedBytes);
+    params.setEncryptedInputStream(decryptedByteStream);
+    params = cryptoModule.getDecryptingInputStream(params);
+    DataInputStream decryptedDataStream = new DataInputStream(params.getPlaintextInputStream());
+
+    String utf;
+    Integer in;
+    try {
+      utf = decryptedDataStream.readUTF();
+      in = decryptedDataStream.readInt();
+    } catch (IOException e) {
+      if (e.getCause().getClass().equals(AEADBadTagException.class)) {
+        throw new AEADBadTagException();
+      } else {
+        throw e;
+      }
+    }
+
+    decryptedDataStream.close();
+    if (utf.equals(MARKER_STRING) && in.equals(MARKER_INT)) {
+      return 1;
+    } else {
+      return 0;
+    }
+  }
+
 }
diff --git a/core/src/test/resources/crypto-on-accumulo-site.xml b/core/src/test/resources/crypto-on-accumulo-site.xml
index 6d9ef13..953e26a 100644
--- a/core/src/test/resources/crypto-on-accumulo-site.xml
+++ b/core/src/test/resources/crypto-on-accumulo-site.xml
@@ -76,8 +76,12 @@
       <value>org.apache.accumulo.core.security.crypto.DefaultCryptoModule</value>
     </property>
     <property>
+      <name>crypto.security.provider</name>
+      <value>SunJCE</value>
+    </property>
+    <property>
       <name>crypto.cipher.suite</name>
-      <value>AES/CFB/NoPadding</value>
+      <value>AES/GCM/NoPadding</value>
     </property>
     <property>
       <name>crypto.wal.cipher.suite</name>
@@ -127,7 +131,7 @@
     
     <property>
     	<name>crypto.default.key.strategy.cipher.suite</name>
-    	<value>AES/ECB/NoPadding</value>
+    	<value>AESWrap/ECB/NoPadding</value>
     </property>
 
 </configuration>
diff --git a/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml b/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml
index a82f9f9..f477682 100644
--- a/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml
+++ b/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml
@@ -80,6 +80,10 @@
       <value>AES/CFB/NoPadding</value>
     </property>
     <property>
+      <name>crypto.wal.cipher.suite</name>
+      <value>AES/CBC/NoPadding</value>
+    </property>
+    <property>
       <name>crypto.cipher.key.algorithm.name</name>
       <value>AES</value>
     </property>
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index d48907f..5fe8d56 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -468,7 +468,8 @@ public class DfsLogger implements Comparable<DfsLogger> {
 
       CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf.getConfiguration());
       // Immediately update to the correct cipher. Doing this here keeps the CryptoModule independent of the writers using it
-      if (params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()) != null) {
+      if (params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()) != null
+          && !params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()).equals("")) {
         params.setCipherSuite(params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()));
       }
 

-- 
To stop receiving notification emails like this one, please contact
['"commits@accumulo.apache.org" <co...@accumulo.apache.org>'].