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/09 21:54:50 UTC

[accumulo] branch master updated (492d573 -> 314f9f2)

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

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


    from 492d573  Merge branch 'no-value-equals-bytes'
     new e875f01  ACCUMULO-4708 Limit RFile block size to 2GB
     new daeffd8  ACCUMULO-4737 Clean up cipher algorithm configuration
     new 314f9f2  ACCUMULO-4733 Enabled configuring crypto security provider

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../accumulo/core/conf/ConfigSanityCheck.java      | 28 +++++++
 .../org/apache/accumulo/core/conf/Property.java    | 17 +++-
 .../org/apache/accumulo/core/file/rfile/RFile.java | 18 +++--
 .../accumulo/core/file/rfile/RFileOperations.java  |  6 ++
 .../accumulo/core/file/rfile/bcfile/BCFile.java    |  9 ++-
 .../CachingHDFSSecretKeyEncryptionStrategy.java    |  9 ++-
 .../core/security/crypto/CryptoModule.java         |  1 +
 .../core/security/crypto/CryptoModuleFactory.java  | 22 ++---
 .../security/crypto/CryptoModuleParameters.java    | 94 ++++++++++------------
 .../core/security/crypto/DefaultCryptoModule.java  | 49 ++++-------
 .../security/crypto/DefaultCryptoModuleUtils.java  | 11 ++-
 .../NonCachingSecretKeyEncryptionStrategy.java     |  9 ++-
 .../security/crypto/RFileCipherOutputStream.java   | 85 +++++++++++++++++++
 .../accumulo/core/conf/ConfigSanityCheckTest.java  | 16 ++++
 .../accumulo/core/security/crypto/CryptoTest.java  | 14 ++--
 .../src/test/resources/crypto-on-accumulo-site.xml | 10 ++-
 .../crypto-on-no-key-encryption-accumulo-site.xml  |  2 +-
 pom.xml                                            |  3 +-
 .../org/apache/accumulo/tserver/log/DfsLogger.java |  4 +
 .../org/apache/accumulo/test/ShellConfigIT.java    |  4 +-
 20 files changed, 275 insertions(+), 136 deletions(-)
 create mode 100644 core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java

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

[accumulo] 03/03: ACCUMULO-4733 Enabled configuring crypto security provider

Posted by kt...@apache.org.
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

commit 314f9f2b2f70d28a52cb699e1c65b85e4d6be7e3
Author: Nick <31...@users.noreply.github.com>
AuthorDate: Tue Oct 31 14:34:04 2017 -0400

    ACCUMULO-4733 Enabled configuring crypto security provider
    
    By default uses system provider unless one is specified
---
 .../org/apache/accumulo/core/conf/Property.java     |  3 +++
 .../CachingHDFSSecretKeyEncryptionStrategy.java     |  3 ++-
 .../core/security/crypto/CryptoModuleFactory.java   |  1 +
 .../security/crypto/CryptoModuleParameters.java     | 21 +++++++++++++++++++++
 .../core/security/crypto/DefaultCryptoModule.java   |  4 ++--
 .../security/crypto/DefaultCryptoModuleUtils.java   | 11 +++++++++--
 .../NonCachingSecretKeyEncryptionStrategy.java      |  3 ++-
 .../accumulo/core/security/crypto/CryptoTest.java   |  1 +
 core/src/test/resources/crypto-on-accumulo-site.xml |  4 ++++
 9 files changed, 45 insertions(+), 6 deletions(-)

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 f0e2338..48a4796 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
@@ -67,6 +67,9 @@ public enum Property {
   CRYPTO_CIPHER_KEY_LENGTH("crypto.cipher.key.length", "128", PropertyType.STRING,
       "Specifies the key length *in bits* to use for the symmetric key, should probably be 128 or 256 unless you really know what you're doing"),
   @Experimental
+  CRYPTO_SECURITY_PROVIDER("crypto.security.provider", "", PropertyType.STRING,
+      "States the security provider to use, and defaults to the system configured provider"),
+  @Experimental
   CRYPTO_SECURE_RNG("crypto.secure.rng", "SHA1PRNG", PropertyType.STRING,
       "States the secure random number generator to use, and defaults to the built-in Sun SHA1PRNG"),
   @Experimental
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
index ed6d6a5..1fa659a 100644
--- 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
@@ -69,7 +69,8 @@ public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncrypti
   }
 
   private void doKeyEncryptionOperation(int encryptionMode, CryptoModuleParameters params) throws IOException {
-    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_CIPHER_SUITE.getKey()));
+    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_CIPHER_SUITE.getKey()),
+        params.getSecurityProvider());
 
     try {
       cipher.init(encryptionMode, new SecretKeySpec(secretKeyCache.getKeyEncryptionKey(), params.getKeyAlgorithmName()));
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 e56ee92..251a605 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
@@ -273,6 +273,7 @@ public class CryptoModuleFactory {
         .getKey())));
     params.setRandomNumberGenerator(cryptoOpts.get(Property.CRYPTO_SECURE_RNG.getKey()));
     params.setRandomNumberGeneratorProvider(cryptoOpts.get(Property.CRYPTO_SECURE_RNG_PROVIDER.getKey()));
+    params.setSecurityProvider(cryptoOpts.get(Property.CRYPTO_SECURITY_PROVIDER.getKey()));
     String blockStreamSize = cryptoOpts.get(Property.CRYPTO_BLOCK_STREAM_SIZE.getKey());
     if (blockStreamSize != null)
       params.setBlockStreamSize(Integer.parseInt(blockStreamSize));
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 b5a1ae4..04d8107 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
@@ -204,6 +204,26 @@ public class CryptoModuleParameters {
   }
 
   /**
+   * Gets the security provider name.
+   *
+   * @see CryptoParameters#setSecurityProvider(String)
+   * @return the security provider name
+   */
+  public String getSecurityProvider() {
+    return securityProvider;
+  }
+
+  /**
+   * Sets the name of the security provider to use for crypto.
+   *
+   * @param securityProvider
+   *          the name of the provider to use
+   */
+  public void setSecurityProvider(String securityProvider) {
+    this.securityProvider = securityProvider;
+  }
+
+  /**
    * Gets the key encryption strategy class.
    *
    * @see CryptoModuleParameters#setKeyEncryptionStrategyClass(String)
@@ -580,6 +600,7 @@ public class CryptoModuleParameters {
   private int keyLength = 0;
   private String randomNumberGenerator = null;
   private String randomNumberGeneratorProvider = null;
+  private String securityProvider = null;
 
   private String keyEncryptionStrategyClass;
   private byte[] encryptedKey;
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 1798e89..c97e7f5 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
@@ -66,7 +66,7 @@ public class DefaultCryptoModule implements CryptoModule {
       params.setSecureRandom(secureRandom);
     }
 
-    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite());
+    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite(), params.getSecurityProvider());
 
     if (params.getInitializationVector() == null) {
       try {
@@ -367,7 +367,7 @@ public class DefaultCryptoModule implements CryptoModule {
       throw new RuntimeException("CryptoModuleParameters object failed validation for decrypt");
     }
 
-    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite());
+    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite(), params.getSecurityProvider());
 
     try {
       cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()),
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModuleUtils.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModuleUtils.java
index 1a33915..015ffca 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModuleUtils.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/DefaultCryptoModuleUtils.java
@@ -50,20 +50,27 @@ public class DefaultCryptoModuleUtils {
     return secureRandom;
   }
 
-  public static Cipher getCipher(String cipherSuite) {
+  public static Cipher getCipher(String cipherSuite, String securityProvider) {
     Cipher cipher = null;
 
     if (cipherSuite.equals("NullCipher")) {
       cipher = new NullCipher();
     } else {
       try {
-        cipher = Cipher.getInstance(cipherSuite);
+        if (securityProvider == null || securityProvider.equals("")) {
+          cipher = Cipher.getInstance(cipherSuite);
+        } else {
+          cipher = Cipher.getInstance(cipherSuite, securityProvider);
+        }
       } catch (NoSuchAlgorithmException e) {
         log.error(String.format("Accumulo configuration file contained a cipher suite \"%s\" that was not recognized by any providers", cipherSuite));
         throw new RuntimeException(e);
       } catch (NoSuchPaddingException e) {
         log.error(String.format("Accumulo configuration file contained a cipher, \"%s\" with a padding that was not recognized by any providers", cipherSuite));
         throw new RuntimeException(e);
+      } catch (NoSuchProviderException e) {
+        log.error(String.format("Accumulo configuration file contained a provider, \"%s\" an unrecognized provider", securityProvider));
+        throw new RuntimeException(e);
       }
     }
     return cipher;
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java
index 2d2d2f7..fa2ee70 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java
@@ -78,7 +78,8 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio
       byte[] keyEncryptionKey = new byte[keyEncryptionKeyLength];
       int bytesRead = in.read(keyEncryptionKey);
 
-      Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_CIPHER_SUITE.getKey()));
+      Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_CIPHER_SUITE.getKey()),
+          params.getSecurityProvider());
 
       // check if the number of bytes read into the array is the same as the value of the length field,
       if (bytesRead == keyEncryptionKeyLength) {
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 4467828..1e29757 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
@@ -98,6 +98,7 @@ public class CryptoTest {
     assertEquals(128, params.getKeyLength());
     assertEquals("SHA1PRNG", params.getRandomNumberGenerator());
     assertEquals("SUN", params.getRandomNumberGeneratorProvider());
+    assertEquals("SunJCE", params.getSecurityProvider());
     assertEquals("org.apache.accumulo.core.security.crypto.CachingHDFSSecretKeyEncryptionStrategy", params.getKeyEncryptionStrategyClass());
   }
 
diff --git a/core/src/test/resources/crypto-on-accumulo-site.xml b/core/src/test/resources/crypto-on-accumulo-site.xml
index f5824c4..6d9ef13 100644
--- a/core/src/test/resources/crypto-on-accumulo-site.xml
+++ b/core/src/test/resources/crypto-on-accumulo-site.xml
@@ -100,6 +100,10 @@
       <value>SUN</value>
     </property>
     <property>
+      <name>crypto.security.provider</name>
+      <value>SunJCE</value>
+    </property>
+    <property>
       <name>crypto.secret.key.encryption.strategy.class</name>
       <value>org.apache.accumulo.core.security.crypto.CachingHDFSSecretKeyEncryptionStrategy</value>
     </property>

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

[accumulo] 01/03: ACCUMULO-4708 Limit RFile block size to 2GB

Posted by kt...@apache.org.
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

commit e875f01ed519100225bdf3a1550c40caccd8521f
Author: Nick <31...@users.noreply.github.com>
AuthorDate: Fri Sep 15 14:42:29 2017 -0400

    ACCUMULO-4708 Limit RFile block size to 2GB
---
 .../accumulo/core/conf/ConfigSanityCheck.java      |  9 +++
 .../org/apache/accumulo/core/file/rfile/RFile.java | 18 +++--
 .../accumulo/core/file/rfile/RFileOperations.java  |  6 ++
 .../accumulo/core/file/rfile/bcfile/BCFile.java    |  7 +-
 .../core/security/crypto/CryptoModule.java         |  1 +
 .../core/security/crypto/DefaultCryptoModule.java  |  3 +-
 .../security/crypto/RFileCipherOutputStream.java   | 86 ++++++++++++++++++++++
 pom.xml                                            |  3 +-
 8 files changed, 122 insertions(+), 11 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 130863c..9c2ed6c 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
@@ -21,6 +21,8 @@ import java.util.Map.Entry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * A utility class for validating {@link AccumuloConfiguration} instances.
  */
@@ -66,6 +68,13 @@ public class ConfigSanityCheck {
       if (key.equals(Property.INSTANCE_VOLUMES.getKey())) {
         usingVolumes = value != null && !value.isEmpty();
       }
+
+      // If the block size or block size index is configured to be too large, we throw an exception to avoid potentially corrupting RFiles later
+      if (key.equals(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey()) || key.equals(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey())) {
+        long bsize = ConfigurationTypeHelper.getFixedMemoryAsBytes(value);
+        Preconditions.checkArgument(bsize > 0 && bsize < Integer.MAX_VALUE, key + " must be greater than 0 and less than " + Integer.MAX_VALUE + " but was: "
+            + bsize);
+      }
     }
 
     if (instanceZkTimeoutValue != null) {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
index c1931da..c399a22 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
@@ -404,7 +404,7 @@ public class RFile {
     private SampleLocalityGroupWriter sample;
 
     private SummaryStatistics keyLenStats = new SummaryStatistics();
-    private double avergageKeySize = 0;
+    private double averageKeySize = 0;
 
     LocalityGroupWriter(BlockFileWriter fileWriter, long blockSize, long maxBlockSize, LocalityGroupMetadata currentLocalityGroup,
         SampleLocalityGroupWriter sample) {
@@ -441,19 +441,27 @@ public class RFile {
       } else if (blockWriter.getRawSize() > blockSize) {
 
         // Look for a key thats short to put in the index, defining short as average or below.
-        if (avergageKeySize == 0) {
+        if (averageKeySize == 0) {
           // use the same average for the search for a below average key for a block
-          avergageKeySize = keyLenStats.getMean();
+          averageKeySize = keyLenStats.getMean();
         }
 
         // Possibly produce a shorter key that does not exist in data. Even if a key can be shortened, it may not be below average.
         Key closeKey = KeyShortener.shorten(prevKey, key);
 
-        if ((closeKey.getSize() <= avergageKeySize || blockWriter.getRawSize() > maxBlockSize) && !isGiantKey(closeKey)) {
+        if ((closeKey.getSize() <= averageKeySize || blockWriter.getRawSize() > maxBlockSize) && !isGiantKey(closeKey)) {
           closeBlock(closeKey, false);
           blockWriter = fileWriter.prepareDataBlock();
           // set average to zero so its recomputed for the next block
-          avergageKeySize = 0;
+          averageKeySize = 0;
+          // To constrain the growth of data blocks, we limit our worst case scenarios to closing
+          // blocks if they reach the maximum configurable block size of Integer.MAX_VALUE.
+          // 128 bytes added for metadata overhead
+        } else if (((long) key.getSize() + (long) value.getSize() + blockWriter.getRawSize() + 128L) >= Integer.MAX_VALUE) {
+          closeBlock(closeKey, false);
+          blockWriter = fileWriter.prepareDataBlock();
+          averageKeySize = 0;
+
         }
       }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
index 4d1af7e..195da93 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import com.google.common.base.Preconditions;
+
 public class RFileOperations extends FileOperations {
 
   private static final Collection<ByteSequence> EMPTY_CF_SET = Collections.emptySet();
@@ -82,7 +84,11 @@ public class RFileOperations extends FileOperations {
     AccumuloConfiguration acuconf = options.getTableConfiguration();
 
     long blockSize = acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);
+    Preconditions.checkArgument((blockSize < Integer.MAX_VALUE && blockSize > 0), "table.file.compress.blocksize must be greater than 0 and less than "
+        + Integer.MAX_VALUE);
     long indexBlockSize = acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX);
+    Preconditions.checkArgument((indexBlockSize < Integer.MAX_VALUE && indexBlockSize > 0),
+        "table.file.compress.blocksize.index must be greater than 0 and less than " + Integer.MAX_VALUE);
 
     SamplerConfigurationImpl samplerConfig = SamplerConfigurationImpl.newSamplerConfig(acuconf);
     Sampler sampler = null;
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 5cfe824..a169619 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
@@ -283,12 +283,13 @@ public final class BCFile {
       /**
        * Get the raw size of the block.
        *
+       * Caution: size() comes from DataOutputStream which returns Integer.MAX_VALUE on an overflow. This results in a value of 2GiB meaning that
+       * an unknown amount of data, at least 2GiB large, has been written. RFiles handle this issue by keeping track of the position of blocks
+       * instead of relying on blocks to provide this information.
+       *
        * @return the number of uncompressed bytes written through the BlockAppender so far.
        */
       public long getRawSize() throws IOException {
-        /**
-         * Expecting the size() of a block not exceeding 4GB. Assuming the size() will wrap to negative integer if it exceeds 2GB.
-         */
         return size() & 0x00000000ffffffffL;
       }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModule.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModule.java
index 44531dc..c1962f7 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModule.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoModule.java
@@ -28,6 +28,7 @@ import javax.crypto.CipherOutputStream;
  * Classes that obey this interface may be used to provide encrypting and decrypting streams to the rest of Accumulo. Classes that obey this interface may be
  * configured as the crypto module by setting the property crypto.module.class in the accumulo-site.xml file.
  *
+ * When implementing CryptoModule, it is recommended that any {@link CipherOutputStreams} uses {@link RFileCipherOutputStream} instead.
  *
  */
 public interface CryptoModule {
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 7609bb0..c5c41cd 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,7 +30,6 @@ import java.util.Map;
 
 import javax.crypto.Cipher;
 import javax.crypto.CipherInputStream;
-import javax.crypto.CipherOutputStream;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 
@@ -251,7 +250,7 @@ public class DefaultCryptoModule implements CryptoModule {
       throw new RuntimeException("Encryption cipher must be a block cipher");
     }
 
-    CipherOutputStream cipherOutputStream = new CipherOutputStream(params.getPlaintextOutputStream(), cipher);
+    RFileCipherOutputStream cipherOutputStream = new RFileCipherOutputStream(params.getPlaintextOutputStream(), cipher);
     BlockedOutputStream blockedOutputStream = new BlockedOutputStream(cipherOutputStream, cipher.getBlockSize(), params.getBlockStreamSize());
 
     params.setEncryptedOutputStream(blockedOutputStream);
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java
new file mode 100644
index 0000000..7dad802
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java
@@ -0,0 +1,86 @@
+/*
+ * 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.IOException;
+import java.io.OutputStream;
+
+import javax.crypto.Cipher;
+import javax.crypto.CipherOutputStream;
+
+/**
+ *
+ * This class extends {@link CipherOutputStream} to include a way to track the number of bytes that have
+ * been encrypted by the stream. The write method also includes a mechanism to stop writing and
+ * throw an exception if exceeding a maximum number of bytes is attempted.
+ *
+ */
+public class RFileCipherOutputStream extends CipherOutputStream {
+
+  // This is the maximum size encrypted stream that can be written. Attempting to write anything larger
+  // will cause an exception. Given that each block in an rfile is encrypted separately, and blocks
+  // should be written such that a block cannot ever reach 16GiB, this is believed to be a safe number.
+  // If this does cause an exception, it is an issue best addressed elsewhere.
+  private final long maxOutputSize = 1L << 34; //16GiB
+
+  // The total number of bytes that have been written out
+  private long count = 0;
+
+  /**
+   *
+   * Constructs a RFileCipherOutputStream
+   *
+   * @param os
+   *          the OutputStream object
+   * @param c
+   *          an initialized Cipher object
+   */
+  public RFileCipherOutputStream(OutputStream os, Cipher c) {
+    super(os, c);
+  }
+
+  /**
+   * Override of CipherOutputStream's write to count the number of bytes that have been encrypted.
+   * This method now throws an exception if an attempt to write bytes beyond a maximum is made.
+   */
+  @Override
+  public void write(byte b[], int off, int len) throws IOException {
+    count += len;
+    if (count > maxOutputSize) {
+      throw new IOException("Attempt to write " + count + " bytes was made. A maximum of " + maxOutputSize + " is allowed for an encryption stream.");
+    }
+    super.write(b, off, len);
+  }
+
+  @Override
+  public void write(byte b[]) throws IOException {
+    write(b, 0, b.length);
+  }
+
+  /**
+   * Override of CipherOutputStream's write for a single byte to count it. This method now throws
+   * an exception if an attempt to write bytes beyond a maximum is made.
+   */
+  @Override
+  public void write(int b) throws IOException {
+    count++;
+    if (count > maxOutputSize) {
+      throw new IOException("Attempt to write " + count + " bytes was made. A maximum of " + maxOutputSize + " is allowed for an encryption stream.");
+    }
+    super.write(b);
+  }
+}
diff --git a/pom.xml b/pom.xml
index deaccf0..2cc3abf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -151,6 +151,7 @@
     <surefire.groups />
     <!-- Thrift version -->
     <thrift.version>0.10.0</thrift.version>
+    <unitTestMemSize>-Xmx1G</unitTestMemSize>
     <!-- ZooKeeper version -->
     <zookeeper.version>3.4.10</zookeeper.version>
   </properties>
@@ -868,7 +869,7 @@
             <systemPropertyVariables>
               <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
             </systemPropertyVariables>
-            <argLine>-Xmx1G</argLine>
+            <argLine>${unitTestMemSize}</argLine>
           </configuration>
         </plugin>
         <plugin>

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

[accumulo] 02/03: ACCUMULO-4737 Clean up cipher algorithm configuration

Posted by kt...@apache.org.
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

commit daeffd8d9cf980814fb7d131a0d89cbdfd856298
Author: Nick <31...@users.noreply.github.com>
AuthorDate: Mon Nov 6 11:56:59 2017 -0500

    ACCUMULO-4737 Clean up cipher algorithm configuration
    
    Renamed crypto.cipher.algorithm.name to crypto.cipher.key.algorithm.name
    Removed the unused mode/padding code in favor of passing around the crypto suite (instead of splitting it up and rebuilding it)
    Accumulo will now use crypto.cipher.key.algorithm.name to build the encryption key
    Accumulo will now use crypto.cipher.suite to build the Java Cipher object
    Unit test was updated to reflect the change
    
    Additionally, this was stumbled upon when implementing a separate cipher algorithm option for WAL files, so that change has been included.
    
    Added sanity check for the config file along with tests
---
 .../accumulo/core/conf/ConfigSanityCheck.java      | 19 ++++++
 .../org/apache/accumulo/core/conf/Property.java    | 14 ++++-
 .../accumulo/core/file/rfile/bcfile/BCFile.java    |  8 +--
 .../CachingHDFSSecretKeyEncryptionStrategy.java    |  6 +-
 .../core/security/crypto/CryptoModuleFactory.java  | 21 ++-----
 .../security/crypto/CryptoModuleParameters.java    | 73 +++++++---------------
 .../core/security/crypto/DefaultCryptoModule.java  | 46 +++++---------
 .../NonCachingSecretKeyEncryptionStrategy.java     |  6 +-
 .../security/crypto/RFileCipherOutputStream.java   | 15 +++--
 .../accumulo/core/conf/ConfigSanityCheckTest.java  | 16 +++++
 .../accumulo/core/security/crypto/CryptoTest.java  | 13 ++--
 .../src/test/resources/crypto-on-accumulo-site.xml |  6 +-
 .../crypto-on-no-key-encryption-accumulo-site.xml  |  2 +-
 .../org/apache/accumulo/tserver/log/DfsLogger.java |  4 ++
 .../org/apache/accumulo/test/ShellConfigIT.java    |  4 +-
 15 files changed, 121 insertions(+), 132 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 9c2ed6c..baf1818 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
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.conf;
 
 import java.util.Map.Entry;
+import java.util.Objects;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,6 +49,8 @@ public class ConfigSanityCheck {
   public static void validate(Iterable<Entry<String,String>> entries) {
     String instanceZkTimeoutValue = null;
     boolean usingVolumes = false;
+    String cipherSuite = null;
+    String keyAlgorithm = null;
     for (Entry<String,String> entry : entries) {
       String key = entry.getKey();
       String value = entry.getValue();
@@ -75,6 +78,14 @@ public class ConfigSanityCheck {
         Preconditions.checkArgument(bsize > 0 && bsize < Integer.MAX_VALUE, key + " must be greater than 0 and less than " + Integer.MAX_VALUE + " but was: "
             + bsize);
       }
+
+      if (key.equals(Property.CRYPTO_CIPHER_SUITE.getKey())) {
+        cipherSuite = Objects.requireNonNull(value);
+      }
+
+      if (key.equals(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey())) {
+        keyAlgorithm = Objects.requireNonNull(value);
+      }
     }
 
     if (instanceZkTimeoutValue != null) {
@@ -84,6 +95,14 @@ public class ConfigSanityCheck {
     if (!usingVolumes) {
       log.warn("Use of {} and {} are deprecated. Consider using {} instead.", INSTANCE_DFS_URI, INSTANCE_DFS_DIR, Property.INSTANCE_VOLUMES);
     }
+
+    if (cipherSuite.equals("NullCipher") && !keyAlgorithm.equals("NullCipher")) {
+      fatal(Property.CRYPTO_CIPHER_SUITE.getKey() + " should be configured when " + Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey() + " is set.");
+    }
+
+    if (!cipherSuite.equals("NullCipher") && keyAlgorithm.equals("NullCipher")) {
+      fatal(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey() + " should be configured when " + Property.CRYPTO_CIPHER_SUITE.getKey() + " is set.");
+    }
   }
 
   private interface CheckTimeDuration {
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 e08df9e..f0e2338 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
@@ -48,10 +48,18 @@ public enum Property {
       "Fully qualified class name of the class that implements the CryptoModule interface, to be used in setting up encryption at rest for the WAL and "
           + "(future) other parts of the code."),
   @Experimental
-  CRYPTO_CIPHER_SUITE("crypto.cipher.suite", "NullCipher", PropertyType.STRING, "Describes the cipher suite to use for the write-ahead log"),
+  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"),
   @Experimental
-  CRYPTO_CIPHER_ALGORITHM_NAME("crypto.cipher.algorithm.name", "NullCipher", PropertyType.STRING,
-      "States the name of the algorithm used in the corresponding cipher suite. Do not make these different, unless you enjoy mysterious exceptions and bugs."),
+  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."),
+  @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."),
   @Experimental
   CRYPTO_BLOCK_STREAM_SIZE("crypto.block.stream.size", "1K", PropertyType.BYTES,
       "The size of the buffer above the cipher stream. Used for reading files and padding walog entries."),
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 a169619..f9a61a7 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
@@ -283,9 +283,9 @@ public final class BCFile {
       /**
        * Get the raw size of the block.
        *
-       * Caution: size() comes from DataOutputStream which returns Integer.MAX_VALUE on an overflow. This results in a value of 2GiB meaning that
-       * an unknown amount of data, at least 2GiB large, has been written. RFiles handle this issue by keeping track of the position of blocks
-       * instead of relying on blocks to provide this information.
+       * Caution: size() comes from DataOutputStream which returns Integer.MAX_VALUE on an overflow. This results in a value of 2GiB meaning that an unknown
+       * amount of data, at least 2GiB large, has been written. RFiles handle this issue by keeping track of the position of blocks instead of relying on blocks
+       * to provide this information.
        *
        * @return the number of uncompressed bytes written through the BlockAppender so far.
        */
@@ -395,7 +395,7 @@ public final class BCFile {
           long offsetIndexMeta = out.position();
           metaIndex.write(out);
 
-          if (cryptoParams.getAlgorithmName() == null || cryptoParams.getAlgorithmName().equals(Property.CRYPTO_CIPHER_SUITE.getDefaultValue())) {
+          if (cryptoParams.getCipherSuite() == null || cryptoParams.getCipherSuite().equals(Property.CRYPTO_CIPHER_SUITE.getDefaultValue())) {
             out.writeLong(offsetIndexMeta);
             API_VERSION_1.write(out);
           } else {
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
index 7b79d99..ed6d6a5 100644
--- 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
@@ -72,7 +72,7 @@ public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncrypti
     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()));
+      cipher.init(encryptionMode, new SecretKeySpec(secretKeyCache.getKeyEncryptionKey(), params.getKeyAlgorithmName()));
     } catch (InvalidKeyException e) {
       log.error("{}", e.getMessage(), e);
       throw new RuntimeException(e);
@@ -80,7 +80,7 @@ public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncrypti
 
     if (Cipher.UNWRAP_MODE == encryptionMode) {
       try {
-        Key plaintextKey = cipher.unwrap(params.getEncryptedKey(), params.getAlgorithmName(), Cipher.SECRET_KEY);
+        Key plaintextKey = cipher.unwrap(params.getEncryptedKey(), params.getKeyAlgorithmName(), Cipher.SECRET_KEY);
         params.setPlaintextKey(plaintextKey.getEncoded());
       } catch (InvalidKeyException e) {
         log.error("{}", e.getMessage(), e);
@@ -90,7 +90,7 @@ public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncrypti
         throw new RuntimeException(e);
       }
     } else {
-      Key plaintextKey = new SecretKeySpec(params.getPlaintextKey(), params.getAlgorithmName());
+      Key plaintextKey = new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName());
       try {
         byte[] encryptedSecretKey = cipher.wrap(plaintextKey);
         params.setEncryptedKey(encryptedSecretKey);
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 6a295ad..e56ee92 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
@@ -239,14 +239,6 @@ public class CryptoModuleFactory {
 
   }
 
-  public static String[] parseCipherTransform(String cipherTransform) {
-    if (cipherTransform == null) {
-      return new String[3];
-    }
-
-    return cipherTransform.split("/");
-  }
-
   public static CryptoModuleParameters createParamsObjectFromAccumuloConfiguration(AccumuloConfiguration conf) {
     CryptoModuleParameters params = new CryptoModuleParameters();
 
@@ -264,26 +256,21 @@ public class CryptoModuleFactory {
   }
 
   public static CryptoModuleParameters fillParamsObjectFromStringMap(CryptoModuleParameters params, Map<String,String> cryptoOpts) {
-
-    // Parse the cipher suite for the mode and padding options
-    String[] cipherTransformParts = parseCipherTransform(cryptoOpts.get(Property.CRYPTO_CIPHER_SUITE.getKey()));
-
+    params.setCipherSuite(cryptoOpts.get(Property.CRYPTO_CIPHER_SUITE.getKey()));
     // If no encryption has been specified, then we abort here.
-    if (cipherTransformParts[0] == null || cipherTransformParts[0].equals("NullCipher")) {
+    if (params.getCipherSuite() == null || params.getCipherSuite().equals("NullCipher")) {
       params.setAllOptions(cryptoOpts);
-      params.setAlgorithmName("NullCipher");
+
       return params;
     }
 
     params.setAllOptions(cryptoOpts);
 
-    params.setAlgorithmName(cryptoOpts.get(Property.CRYPTO_CIPHER_ALGORITHM_NAME.getKey()));
-    params.setEncryptionMode(cipherTransformParts[1]);
+    params.setKeyAlgorithmName(cryptoOpts.get(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey()));
     params.setKeyEncryptionStrategyClass(cryptoOpts.get(Property.CRYPTO_SECRET_KEY_ENCRYPTION_STRATEGY_CLASS.getKey()));
     params.setKeyLength(Integer.parseInt(cryptoOpts.get(Property.CRYPTO_CIPHER_KEY_LENGTH.getKey())));
     params.setOverrideStreamsSecretKeyEncryptionStrategy(Boolean.parseBoolean(cryptoOpts.get(Property.CRYPTO_OVERRIDE_KEY_STRATEGY_WITH_CONFIGURED_STRATEGY
         .getKey())));
-    params.setPadding(cipherTransformParts[2]);
     params.setRandomNumberGenerator(cryptoOpts.get(Property.CRYPTO_SECURE_RNG.getKey()));
     params.setRandomNumberGeneratorProvider(cryptoOpts.get(Property.CRYPTO_SECURE_RNG_PROVIDER.getKey()));
     String blockStreamSize = cryptoOpts.get(Property.CRYPTO_BLOCK_STREAM_SIZE.getKey());
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 a7bb93d..b5a1ae4 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
@@ -37,17 +37,17 @@ import javax.crypto.CipherOutputStream;
 public class CryptoModuleParameters {
 
   /**
-   * Gets the name of the symmetric algorithm to use for encryption.
+   * Gets the name of the symmetric algorithm to use for the creation of encryption keys.
    *
-   * @see CryptoModuleParameters#setAlgorithmName(String)
+   * @see CryptoModuleParameters#setKeyAlgorithmName(String)
    */
 
-  public String getAlgorithmName() {
-    return algorithmName;
+  public String getKeyAlgorithmName() {
+    return keyAlgorithmName;
   }
 
   /**
-   * Sets the name of the symmetric algorithm to use for an encryption stream.
+   * Sets the name of the symmetric algorithm to use for the creation of encryption keys.
    * <p>
    * Valid names are names recognized by your cryptographic engine provider. For the default Java provider, valid names would include things like "AES", "RC4",
    * "DESede", etc.
@@ -56,73 +56,45 @@ public class CryptoModuleParameters {
    * decryption. <br>
    * For <b>decryption</b>, this value is often disregarded in favor of the value encoded with the ciphertext.
    *
-   * @param algorithmName
+   * @param keyAlgorithmName
    *          the name of the cryptographic algorithm to use.
    * @see <a href="http://docs.oracle.com/javase/1.5.0/docs/guide/security/jce/JCERefGuide.html#AppA">Standard Algorithm Names in JCE</a>
    *
    */
 
-  public void setAlgorithmName(String algorithmName) {
-    this.algorithmName = algorithmName;
+  public void setKeyAlgorithmName(String keyAlgorithmName) {
+    this.keyAlgorithmName = keyAlgorithmName;
   }
 
   /**
-   * Gets the name of the encryption mode to use for encryption.
+   * Gets the name of the cipher suite used for encryption
    *
-   * @see CryptoModuleParameters#setEncryptionMode(String)
+   * @see CryptoModuleParameters#setCipherSuite(String)
    */
 
-  public String getEncryptionMode() {
-    return encryptionMode;
+  public String getCipherSuite() {
+    return cipherSuite;
   }
 
   /**
-   * Sets the name of the encryption mode to use for an encryption stream.
+   * Sets the name of the crypto suite to use for an encryption stream.
    * <p>
-   * Valid names are names recognized by your cryptographic engine provider. For the default Java provider, valid names would include things like "EBC", "CBC",
-   * "CFB", etc.
-   * <p>
-   * For <b>encryption</b>, this value is <b>required</b> and is always used. Its value should be prepended or otherwise included with the ciphertext for future
-   * decryption. <br>
-   * For <b>decryption</b>, this value is often disregarded in favor of the value encoded with the ciphertext.
+   * Valid names are names recognized by your cryptographic engine provider.
    *
-   * @param encryptionMode
-   *          the name of the encryption mode to use.
-   * @see <a href="http://docs.oracle.com/javase/1.5.0/docs/guide/security/jce/JCERefGuide.html#AppA">Standard Mode Names in JCE</a>
-   *
-   */
-
-  public void setEncryptionMode(String encryptionMode) {
-    this.encryptionMode = encryptionMode;
-  }
-
-  /**
-   * Gets the name of the padding type to use for encryption.
+   * The format for input should be: algorithm/mode/padding
    *
-   * @see CryptoModuleParameters#setPadding(String)
-   */
-
-  public String getPadding() {
-    return padding;
-  }
-
-  /**
-   * Sets the name of the padding type to use for an encryption stream.
-   * <p>
-   * Valid names are names recognized by your cryptographic engine provider. For the default Java provider, valid names would include things like "NoPadding",
-   * "None", etc.
+   * For the default Java provider, valid names would include things like "AES/CBC/NoPadding".
    * <p>
    * For <b>encryption</b>, this value is <b>required</b> and is always used. Its value should be prepended or otherwise included with the ciphertext for future
    * decryption. <br>
    * For <b>decryption</b>, this value is often disregarded in favor of the value encoded with the ciphertext.
    *
-   * @param padding
-   *          the name of the padding type to use.
-   * @see <a href="http://docs.oracle.com/javase/1.5.0/docs/guide/security/jce/JCERefGuide.html#AppA">Standard Padding Names in JCE</a>
+   * @param cipherSuite
+   *          the cipher suite to use.
    *
    */
-  public void setPadding(String padding) {
-    this.padding = padding;
+  public void setCipherSuite(String cipherSuite) {
+    this.cipherSuite = cipherSuite;
   }
 
   /**
@@ -602,9 +574,8 @@ public class CryptoModuleParameters {
     this.allOptions = allOptions;
   }
 
-  private String algorithmName = null;
-  private String encryptionMode = null;
-  private String padding = null;
+  private String cipherSuite = null;
+  private String keyAlgorithmName = null;
   private byte[] plaintextKey;
   private int keyLength = 0;
   private String randomNumberGenerator = null;
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 c5c41cd..1798e89 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
@@ -56,10 +56,9 @@ public class DefaultCryptoModule implements CryptoModule {
 
   @Override
   public CryptoModuleParameters initializeCipher(CryptoModuleParameters params) {
-    String cipherTransformation = getCipherTransformation(params);
 
     log.trace(String.format("Using cipher suite \"%s\" with key length %d with RNG \"%s\" and RNG provider \"%s\" and key encryption strategy \"%s\"",
-        cipherTransformation, params.getKeyLength(), params.getRandomNumberGenerator(), params.getRandomNumberGeneratorProvider(),
+        params.getCipherSuite(), params.getKeyLength(), params.getRandomNumberGenerator(), params.getRandomNumberGeneratorProvider(),
         params.getKeyEncryptionStrategyClass()));
 
     if (params.getSecureRandom() == null) {
@@ -67,11 +66,11 @@ public class DefaultCryptoModule implements CryptoModule {
       params.setSecureRandom(secureRandom);
     }
 
-    Cipher cipher = DefaultCryptoModuleUtils.getCipher(cipherTransformation);
+    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite());
 
     if (params.getInitializationVector() == null) {
       try {
-        cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getAlgorithmName()), params.getSecureRandom());
+        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);
@@ -81,7 +80,7 @@ public class DefaultCryptoModule implements CryptoModule {
 
     } else {
       try {
-        cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getAlgorithmName()),
+        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");
@@ -98,15 +97,6 @@ public class DefaultCryptoModule implements CryptoModule {
 
   }
 
-  private String getCipherTransformation(CryptoModuleParameters params) {
-    String cipherSuite = params.getAlgorithmName() + "/" + params.getEncryptionMode() + "/" + params.getPadding();
-    return cipherSuite;
-  }
-
-  private String[] parseCipherSuite(String cipherSuite) {
-    return cipherSuite.split("/");
-  }
-
   private boolean validateNotEmpty(String givenValue, boolean allIsWell, StringBuilder buf, String errorMessage) {
     if (givenValue == null || givenValue.equals("")) {
       buf.append(errorMessage);
@@ -145,15 +135,14 @@ public class DefaultCryptoModule implements CryptoModule {
           "The following problems were found with the CryptoModuleParameters object you provided for an encrypt operation:\n");
       boolean allIsWell = true;
 
-      allIsWell = validateNotEmpty(params.getAlgorithmName(), allIsWell, errorBuf, "No algorithm name was specified.");
+      allIsWell = validateNotEmpty(params.getCipherSuite(), allIsWell, errorBuf, "No cipher suite was specified.");
 
-      if (allIsWell && params.getAlgorithmName().equals("NullCipher")) {
+      if (allIsWell && params.getCipherSuite().equals("NullCipher")) {
         return true;
       }
 
-      allIsWell = validateNotEmpty(params.getPadding(), allIsWell, errorBuf, "No padding was specified.");
       allIsWell = validateNotZero(params.getKeyLength(), allIsWell, errorBuf, "No key length was specified.");
-      allIsWell = validateNotEmpty(params.getEncryptionMode(), allIsWell, errorBuf, "No encryption mode was specified.");
+      allIsWell = validateNotEmpty(params.getKeyAlgorithmName(), allIsWell, errorBuf, "No key algorithm name was specified.");
       allIsWell = validateNotEmpty(params.getRandomNumberGenerator(), allIsWell, errorBuf, "No random number generator was specified.");
       allIsWell = validateNotEmpty(params.getRandomNumberGeneratorProvider(), allIsWell, errorBuf, "No random number generate provider was specified.");
       allIsWell = validateNotNull(params.getPlaintextOutputStream(), allIsWell, errorBuf, "No plaintext output stream was specified.");
@@ -171,9 +160,7 @@ public class DefaultCryptoModule implements CryptoModule {
           "The following problems were found with the CryptoModuleParameters object you provided for a decrypt operation:\n");
       boolean allIsWell = true;
 
-      allIsWell = validateNotEmpty(params.getPadding(), allIsWell, errorBuf, "No padding was specified.");
       allIsWell = validateNotZero(params.getKeyLength(), allIsWell, errorBuf, "No key length was specified.");
-      allIsWell = validateNotEmpty(params.getEncryptionMode(), allIsWell, errorBuf, "No encryption mode was specified.");
       allIsWell = validateNotEmpty(params.getRandomNumberGenerator(), allIsWell, errorBuf, "No random number generator was specified.");
       allIsWell = validateNotEmpty(params.getRandomNumberGeneratorProvider(), allIsWell, errorBuf, "No random number generate provider was specified.");
       allIsWell = validateNotNull(params.getEncryptedInputStream(), allIsWell, errorBuf, "No encrypted input stream was specified.");
@@ -211,7 +198,7 @@ public class DefaultCryptoModule implements CryptoModule {
     }
 
     // If they want a null output stream, just return their plaintext stream as the encrypted stream
-    if (params.getAlgorithmName().equals("NullCipher")) {
+    if (params.getCipherSuite().equals("NullCipher")) {
       params.setEncryptedOutputStream(params.getPlaintextOutputStream());
       return params;
     }
@@ -271,8 +258,8 @@ public class DefaultCryptoModule implements CryptoModule {
 
       // Write out the cipher suite and algorithm used to encrypt this file. In case the admin changes, we want to still
       // decode the old format.
-      dataOut.writeUTF(getCipherTransformation(params));
-      dataOut.writeUTF(params.getAlgorithmName());
+      dataOut.writeUTF(params.getCipherSuite());
+      dataOut.writeUTF(params.getKeyAlgorithmName());
 
       // Write the init vector to the log file
       dataOut.writeInt(params.getInitializationVector().length);
@@ -313,10 +300,8 @@ public class DefaultCryptoModule implements CryptoModule {
         // Set the cipher parameters
         String cipherSuiteFromFile = dataIn.readUTF();
         String algorithmNameFromFile = dataIn.readUTF();
-        String[] cipherSuiteParts = parseCipherSuite(cipherSuiteFromFile);
-        params.setAlgorithmName(algorithmNameFromFile);
-        params.setEncryptionMode(cipherSuiteParts[1]);
-        params.setPadding(cipherSuiteParts[2]);
+        params.setCipherSuite(cipherSuiteFromFile);
+        params.setKeyAlgorithmName(algorithmNameFromFile);
 
         // Read the secret key and initialization vector from the file
         int initVectorLength = dataIn.readInt();
@@ -382,10 +367,10 @@ public class DefaultCryptoModule implements CryptoModule {
       throw new RuntimeException("CryptoModuleParameters object failed validation for decrypt");
     }
 
-    Cipher cipher = DefaultCryptoModuleUtils.getCipher(getCipherTransformation(params));
+    Cipher cipher = DefaultCryptoModuleUtils.getCipher(params.getCipherSuite());
 
     try {
-      cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getAlgorithmName()),
+      cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName()),
           new IvParameterSpec(params.getInitializationVector()));
     } catch (InvalidKeyException e) {
       log.error("Error when trying to initialize cipher with secret key");
@@ -400,7 +385,7 @@ public class DefaultCryptoModule implements CryptoModule {
     if (params.getBlockStreamSize() > 0)
       blockedDecryptingInputStream = new BlockedInputStream(blockedDecryptingInputStream, cipher.getBlockSize(), params.getBlockStreamSize());
 
-    log.trace("Initialized cipher input stream with transformation [{}]", getCipherTransformation(params));
+    log.trace("Initialized cipher input stream with transformation [{}]", params.getCipherSuite());
 
     params.setPlaintextInputStream(blockedDecryptingInputStream);
 
@@ -420,5 +405,4 @@ public class DefaultCryptoModule implements CryptoModule {
 
     return params;
   }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java
index 5c9ca8c..2d2d2f7 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java
@@ -83,7 +83,7 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio
       // check if the number of bytes read into the array is the same as the value of the length field,
       if (bytesRead == keyEncryptionKeyLength) {
         try {
-          cipher.init(encryptionMode, new SecretKeySpec(keyEncryptionKey, params.getAlgorithmName()));
+          cipher.init(encryptionMode, new SecretKeySpec(keyEncryptionKey, params.getKeyAlgorithmName()));
         } catch (InvalidKeyException e) {
           log.error("{}", e.getMessage(), e);
           throw new RuntimeException(e);
@@ -91,7 +91,7 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio
 
         if (Cipher.UNWRAP_MODE == encryptionMode) {
           try {
-            Key plaintextKey = cipher.unwrap(params.getEncryptedKey(), params.getAlgorithmName(), Cipher.SECRET_KEY);
+            Key plaintextKey = cipher.unwrap(params.getEncryptedKey(), params.getKeyAlgorithmName(), Cipher.SECRET_KEY);
             params.setPlaintextKey(plaintextKey.getEncoded());
           } catch (InvalidKeyException e) {
             log.error("{}", e.getMessage(), e);
@@ -101,7 +101,7 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio
             throw new RuntimeException(e);
           }
         } else {
-          Key plaintextKey = new SecretKeySpec(params.getPlaintextKey(), params.getAlgorithmName());
+          Key plaintextKey = new SecretKeySpec(params.getPlaintextKey(), params.getKeyAlgorithmName());
           try {
             byte[] encryptedSecretKey = cipher.wrap(plaintextKey);
             params.setEncryptedKey(encryptedSecretKey);
diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java
index 7dad802..cb85d45 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/RFileCipherOutputStream.java
@@ -24,9 +24,8 @@ import javax.crypto.CipherOutputStream;
 
 /**
  *
- * This class extends {@link CipherOutputStream} to include a way to track the number of bytes that have
- * been encrypted by the stream. The write method also includes a mechanism to stop writing and
- * throw an exception if exceeding a maximum number of bytes is attempted.
+ * This class extends {@link CipherOutputStream} to include a way to track the number of bytes that have been encrypted by the stream. The write method also
+ * includes a mechanism to stop writing and throw an exception if exceeding a maximum number of bytes is attempted.
  *
  */
 public class RFileCipherOutputStream extends CipherOutputStream {
@@ -35,7 +34,7 @@ public class RFileCipherOutputStream extends CipherOutputStream {
   // will cause an exception. Given that each block in an rfile is encrypted separately, and blocks
   // should be written such that a block cannot ever reach 16GiB, this is believed to be a safe number.
   // If this does cause an exception, it is an issue best addressed elsewhere.
-  private final long maxOutputSize = 1L << 34; //16GiB
+  private final long maxOutputSize = 1L << 34; // 16GiB
 
   // The total number of bytes that have been written out
   private long count = 0;
@@ -54,8 +53,8 @@ public class RFileCipherOutputStream extends CipherOutputStream {
   }
 
   /**
-   * Override of CipherOutputStream's write to count the number of bytes that have been encrypted.
-   * This method now throws an exception if an attempt to write bytes beyond a maximum is made.
+   * Override of CipherOutputStream's write to count the number of bytes that have been encrypted. This method now throws an exception if an attempt to write
+   * bytes beyond a maximum is made.
    */
   @Override
   public void write(byte b[], int off, int len) throws IOException {
@@ -72,8 +71,8 @@ public class RFileCipherOutputStream extends CipherOutputStream {
   }
 
   /**
-   * Override of CipherOutputStream's write for a single byte to count it. This method now throws
-   * an exception if an attempt to write bytes beyond a maximum is made.
+   * Override of CipherOutputStream's write for a single byte to count it. This method now throws an exception if an attempt to write bytes beyond a maximum is
+   * made.
    */
   @Override
   public void write(int b) throws IOException {
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java b/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java
index 7ac2113..9f2ff8e 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java
@@ -28,6 +28,8 @@ public class ConfigSanityCheckTest {
   @Before
   public void setUp() {
     m = new java.util.HashMap<>();
+    m.put(Property.CRYPTO_CIPHER_SUITE.getKey(), "NullCipher");
+    m.put(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey(), "NullCipher");
   }
 
   @Test
@@ -77,4 +79,18 @@ public class ConfigSanityCheckTest {
     m.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "10ms");
     ConfigSanityCheck.validate(m.entrySet());
   }
+
+  @Test(expected = SanityCheckException.class)
+  public void testFail_cipherSuiteSetKeyAlgorithmNotSet() {
+    m.put(Property.CRYPTO_CIPHER_SUITE.getKey(), "AES/CBC/NoPadding");
+    m.put(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey(), "NullCipher");
+    ConfigSanityCheck.validate(m.entrySet());
+  }
+
+  @Test(expected = SanityCheckException.class)
+  public void testFail_cipherSuiteNotSetKeyAlgorithmSet() {
+    m.put(Property.CRYPTO_CIPHER_SUITE.getKey(), "NullCipher");
+    m.put(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey(), "AES");
+    ConfigSanityCheck.validate(m.entrySet());
+  }
 }
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 a32a465..4467828 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
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.security.crypto;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayInputStream;
@@ -71,9 +70,7 @@ public class CryptoTest {
     CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 
     assertNotNull(params);
-    assertEquals("NullCipher", params.getAlgorithmName());
-    assertNull(params.getEncryptionMode());
-    assertNull(params.getPadding());
+    assertEquals("NullCipher", params.getCipherSuite());
 
     CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(conf);
     assertNotNull(cryptoModule);
@@ -95,9 +92,9 @@ public class CryptoTest {
     CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 
     assertNotNull(params);
-    assertEquals("AES", params.getAlgorithmName());
-    assertEquals("CFB", params.getEncryptionMode());
-    assertEquals("NoPadding", params.getPadding());
+    assertEquals("AES/CFB/NoPadding", params.getCipherSuite());
+    assertEquals("AES/CBC/NoPadding", params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()));
+    assertEquals("AES", params.getKeyAlgorithmName());
     assertEquals(128, params.getKeyLength());
     assertEquals("SHA1PRNG", params.getRandomNumberGenerator());
     assertEquals("SUN", params.getRandomNumberGeneratorProvider());
@@ -314,7 +311,7 @@ public class CryptoTest {
     // from those configured within the site configuration. After doing this, we should
     // still be able to read the file that was created with a different set of parameters.
     params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
-    params.setAlgorithmName("DESede");
+    params.setKeyAlgorithmName("DESede");
     params.setKeyLength(24 * 8);
 
     ByteArrayInputStream in = new ByteArrayInputStream(resultingBytes);
diff --git a/core/src/test/resources/crypto-on-accumulo-site.xml b/core/src/test/resources/crypto-on-accumulo-site.xml
index ebfc9ae..f5824c4 100644
--- a/core/src/test/resources/crypto-on-accumulo-site.xml
+++ b/core/src/test/resources/crypto-on-accumulo-site.xml
@@ -80,7 +80,11 @@
       <value>AES/CFB/NoPadding</value>
     </property>
     <property>
-      <name>crypto.cipher.algorithm.name</name>
+      <name>crypto.wal.cipher.suite</name>
+      <value>AES/CBC/NoPadding</value>
+    </property>
+    <property>
+      <name>crypto.cipher.key.algorithm.name</name>
       <value>AES</value>
     </property>
     <property>
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 d980783..a82f9f9 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,7 +80,7 @@
       <value>AES/CFB/NoPadding</value>
     </property>
     <property>
-      <name>crypto.cipher.algorithm.name</name>
+      <name>crypto.cipher.key.algorithm.name</name>
       <value>AES</value>
     </property>
     <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 d336c3c..d48907f 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
@@ -467,6 +467,10 @@ public class DfsLogger implements Comparable<DfsLogger> {
       logFile.write(LOG_FILE_HEADER_V3.getBytes(UTF_8));
 
       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) {
+        params.setCipherSuite(params.getAllOptions().get(Property.CRYPTO_WAL_CIPHER_SUITE.getKey()));
+      }
 
       NoFlushOutputStream nfos = new NoFlushOutputStream(logFile);
       params.setPlaintextOutputStream(nfos);
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java b/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
index 977cc36..f90435d 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
@@ -93,11 +93,11 @@ public class ShellConfigIT extends AccumuloClusterHarness {
       Assert.fail("Unknown token type");
     }
 
-    assertTrue(Property.CRYPTO_CIPHER_ALGORITHM_NAME.isExperimental());
+    assertTrue(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.isExperimental());
 
     String configOutput = ts.exec("config");
 
     assertTrue(configOutput.contains(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER));
-    assertFalse(configOutput.contains(Property.CRYPTO_CIPHER_ALGORITHM_NAME.getKey()));
+    assertFalse(configOutput.contains(Property.CRYPTO_CIPHER_KEY_ALGORITHM_NAME.getKey()));
   }
 }

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