You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2017/09/26 14:56:07 UTC

[7/7] nifi git commit: NIFI-3116 This closes #2108. Added initial regression test for StringEncryptor to ensure continued functionality during removal of Jasypt. Added external compatibility regression test for StringEncryptor to ensure continued functio

NIFI-3116 This closes #2108. Added initial regression test for StringEncryptor to ensure continued functionality during removal of Jasypt.
Added external compatibility regression test for StringEncryptor to ensure continued functionality during removal of Jasypt.
Documents custom salt lengths and iteration counts for each encryption method.
Added (ignored) failing tests for keyed encryption (Jasypt does not support keyed encryption).
Changed StringEncryptor to non-final class and added protected default constructor.
Added failing test for initialization status.
Added utility methods in CipherUtility.
Moved PBE cipher providers (and tests) from nifi-standard-processors to nifi-security-utils module.
Implemented PBE and keyed encryption/decryption logic.
Moved Scrypt unit test back into scrypt package.
Resolved test failures in limited strength cryptographic environment.
Implemented keyed encryption/decryption and enabled unit tests.
Removed Jasypt dependency from production scope (kept in test scope for backward compatibility tests).

Signed-off-by: joewitt <jo...@apache.org>


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

Branch: refs/heads/master
Commit: 2c1f5b49e449d34c030080ee46b6b580772c5378
Parents: 3b1b326
Author: Andy LoPresto <al...@apache.org>
Authored: Tue Aug 15 15:33:19 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Tue Sep 26 10:55:29 2017 -0400

----------------------------------------------------------------------
 nifi-assembly/NOTICE                            |   4 -
 nifi-commons/nifi-security-utils/pom.xml        |   3 +
 .../nifi/security/util/EncryptionMethod.java    |   7 +
 .../util/crypto/BcryptCipherProvider.java       | 176 +++++
 .../util/crypto/CipherProviderFactory.java      |  56 ++
 .../security/util/crypto/CipherUtility.java     |  98 ++-
 .../util/crypto/KeyedCipherProvider.java        |   4 +-
 .../util/crypto/NiFiLegacyCipherProvider.java   | 135 ++++
 .../util/crypto/OpenSSLPKCS5CipherProvider.java | 198 +++++
 .../security/util/crypto/PBECipherProvider.java |  72 ++
 .../util/crypto/PBKDF2CipherProvider.java       | 200 +++++
 .../util/crypto/RandomIVPBECipherProvider.java  |  70 ++
 .../util/crypto/ScryptCipherProvider.java       | 285 +++++++
 .../security/util/crypto/bcrypt/BCrypt.java     | 789 +++++++++++++++++++
 .../security/util/crypto/scrypt/Scrypt.java     | 510 ++++++++++++
 .../BcryptCipherProviderGroovyTest.groovy       | 538 +++++++++++++
 .../CipherProviderFactoryGroovyTest.groovy      |  97 +++
 .../NiFiLegacyCipherProviderGroovyTest.groovy   | 299 +++++++
 .../OpenSSLPKCS5CipherProviderGroovyTest.groovy | 323 ++++++++
 .../PBKDF2CipherProviderGroovyTest.groovy       | 545 +++++++++++++
 .../ScryptCipherProviderGroovyTest.groovy       | 597 ++++++++++++++
 .../util/scrypt/ScryptGroovyTest.groovy         | 400 ++++++++++
 .../src/test/resources/openssl_aes.rb           |  46 ++
 .../src/test/resources/openssl_bcrypt.rb        |  62 ++
 .../src/test/resources/openssl_pbkdf2.rb        |  52 ++
 .../src/test/resources/openssl_scrypt.rb        |  58 ++
 .../src/main/resources/META-INF/NOTICE          |   4 -
 .../nifi-framework/nifi-framework-core/pom.xml  |  10 +-
 .../serialization/FlowFromDOMFactory.java       |   3 +-
 .../apache/nifi/encrypt/StringEncryptor.java    | 394 +++++++--
 .../nifi/encrypt/StringEncryptorTest.groovy     | 482 +++++++++++
 .../src/test/resources/logback-test.xml         |   1 +
 .../nifi-standard-processors/pom.xml            |   2 +-
 .../util/crypto/BcryptCipherProvider.java       | 176 -----
 .../util/crypto/CipherProviderFactory.java      |  56 --
 .../util/crypto/NiFiLegacyCipherProvider.java   | 135 ----
 .../util/crypto/OpenSSLPKCS5CipherProvider.java | 198 -----
 .../security/util/crypto/PBECipherProvider.java |  72 --
 .../util/crypto/PBKDF2CipherProvider.java       | 200 -----
 .../util/crypto/RandomIVPBECipherProvider.java  |  70 --
 .../util/crypto/ScryptCipherProvider.java       | 285 -------
 .../security/util/crypto/bcrypt/BCrypt.java     | 789 -------------------
 .../security/util/crypto/scrypt/Scrypt.java     | 510 ------------
 .../BcryptCipherProviderGroovyTest.groovy       | 538 -------------
 .../CipherProviderFactoryGroovyTest.groovy      |  97 ---
 .../NiFiLegacyCipherProviderGroovyTest.groovy   | 299 -------
 .../OpenSSLPKCS5CipherProviderGroovyTest.groovy | 323 --------
 .../PBKDF2CipherProviderGroovyTest.groovy       | 545 -------------
 .../ScryptCipherProviderGroovyTest.groovy       | 597 --------------
 .../util/crypto/scrypt/ScryptGroovyTest.groovy  | 399 ----------
 .../src/test/resources/openssl_aes.rb           |  46 --
 .../src/test/resources/openssl_bcrypt.rb        |  62 --
 .../src/test/resources/openssl_pbkdf2.rb        |  52 --
 .../src/test/resources/openssl_scrypt.rb        |  58 --
 54 files changed, 6452 insertions(+), 5575 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-assembly/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index 83fd65a..73b6778 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -125,10 +125,6 @@ The following binary components are provided under the Apache Software License v
     CurvesAIP is BSD-licensed software (https://github.com/virtuald/curvesapi/)
     Copyright (c) 2005, Graph Builder
 
-  (ASLv2) Jasypt
-    The following NOTICE information applies:
-	  Copyright (c) 2007-2010, The JASYPT team (http://www.jasypt.org)
-
   (ASLv2) Apache Commons Codec
     The following NOTICE information applies:
       Apache Commons Codec

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/pom.xml b/nifi-commons/nifi-security-utils/pom.xml
index 3f4a088..92b5df5 100644
--- a/nifi-commons/nifi-security-utils/pom.xml
+++ b/nifi-commons/nifi-security-utils/pom.xml
@@ -68,6 +68,9 @@
                 <configuration>
                     <excludes combine.children="append">
                         <exclude>src/test/resources/xxe_template.xml</exclude>
+                        <!-- This file is copied from https://github.com/jeremyh/jBCrypt
+because the binary is compiled for Java 8 and we must support Java 7 -->
+                        <exclude>src/main/java/org/apache/nifi/security/util/crypto/bcrypt/BCrypt.java</exclude>
                     </excludes>
                 </configuration>
             </plugin>

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
index a1ef2a4..028c915 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java
@@ -94,6 +94,13 @@ public enum EncryptionMethod {
         return !algorithm.startsWith("PBE") && !algorithm.startsWith("PGP");
     }
 
+    /**
+     * @return true if this algorithm uses its own internal key derivation process from a password
+     */
+    public boolean isPBECipher() {
+        return algorithm.startsWith("PBE");
+    }
+
     @Override
     public String toString() {
         final ToStringBuilder builder = new ToStringBuilder(this);

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/BcryptCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/BcryptCipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/BcryptCipherProvider.java
new file mode 100644
index 0000000..f28cde9
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/BcryptCipherProvider.java
@@ -0,0 +1,176 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.security.util.crypto.bcrypt.BCrypt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BcryptCipherProvider extends RandomIVPBECipherProvider {
+    private static final Logger logger = LoggerFactory.getLogger(BcryptCipherProvider.class);
+
+    private final int workFactor;
+    /**
+     * This can be calculated automatically using the code {@see BcryptCipherProviderGroovyTest#calculateMinimumWorkFactor} or manually updated by a maintainer
+     */
+    private static final int DEFAULT_WORK_FACTOR = 12;
+    private static final int DEFAULT_SALT_LENGTH = 16;
+
+    private static final Pattern BCRYPT_SALT_FORMAT = Pattern.compile("^\\$\\d\\w\\$\\d{2}\\$[\\w\\/\\.]{22}");
+
+    /**
+     * Instantiates a Bcrypt cipher provider with the default work factor 12 (2^12 key expansion rounds).
+     */
+    public BcryptCipherProvider() {
+        this(DEFAULT_WORK_FACTOR);
+    }
+
+    /**
+     * Instantiates a Bcrypt cipher provider with the specified work factor w (2^w key expansion rounds).
+     *
+     * @param workFactor the (log) number of key expansion rounds [4..30]
+     */
+    public BcryptCipherProvider(int workFactor) {
+        this.workFactor = workFactor;
+        if (workFactor < DEFAULT_WORK_FACTOR) {
+            logger.warn("The provided work factor {} is below the recommended minimum {}", workFactor, DEFAULT_WORK_FACTOR);
+        }
+    }
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key is derived by the KDF of the implementation. The IV is provided externally to allow for non-deterministic IVs, as IVs
+     * deterministically derived from the password are a potential vulnerability and compromise semantic security. See
+     * <a href="http://crypto.stackexchange.com/a/3970/12569">Ilmari Karonen's answer on Crypto Stack Exchange</a>
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the complete salt (e.g. {@code "$2a$10$gUVbkVzp79H8YaCOsCVZNu".getBytes(StandardCharsets.UTF_8)})
+     * @param iv               the IV
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception {
+        try {
+            return getInitializedCipher(encryptionMethod, password, salt, iv, keyLength, encryptMode);
+        } catch (IllegalArgumentException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new ProcessException("Error initializing the cipher", e);
+        }
+    }
+
+    @Override
+    Logger getLogger() {
+        return logger;
+    }
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
+     *
+     * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the complete salt (e.g. {@code "$2a$10$gUVbkVzp79H8YaCOsCVZNu".getBytes(StandardCharsets.UTF_8)})
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, int keyLength, boolean encryptMode) throws Exception {
+        return getCipher(encryptionMethod, password, salt, new byte[0], keyLength, encryptMode);
+    }
+
+    protected Cipher getInitializedCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception {
+        if (encryptionMethod == null) {
+            throw new IllegalArgumentException("The encryption method must be specified");
+        }
+        if (!encryptionMethod.isCompatibleWithStrongKDFs()) {
+            throw new IllegalArgumentException(encryptionMethod.name() + " is not compatible with Bcrypt");
+        }
+
+        if (StringUtils.isEmpty(password)) {
+            throw new IllegalArgumentException("Encryption with an empty password is not supported");
+        }
+
+        String algorithm = encryptionMethod.getAlgorithm();
+        String provider = encryptionMethod.getProvider();
+
+        final String cipherName = CipherUtility.parseCipherFromAlgorithm(algorithm);
+        if (!CipherUtility.isValidKeyLength(keyLength, cipherName)) {
+            throw new IllegalArgumentException(String.valueOf(keyLength) + " is not a valid key length for " + cipherName);
+        }
+
+        String bcryptSalt = formatSaltForBcrypt(salt);
+
+        String hash = BCrypt.hashpw(password, bcryptSalt);
+
+        /* The SHA-512 hash is required in order to derive a key longer than 184 bits (the resulting size of the Bcrypt hash) and ensuring the avalanche effect causes higher key entropy (if all
+        derived keys follow a consistent pattern, it weakens the strength of the encryption) */
+        MessageDigest digest = MessageDigest.getInstance("SHA-512", provider);
+        byte[] dk = digest.digest(hash.getBytes(StandardCharsets.UTF_8));
+        dk = Arrays.copyOf(dk, keyLength / 8);
+        SecretKey tempKey = new SecretKeySpec(dk, algorithm);
+
+        KeyedCipherProvider keyedCipherProvider = new AESKeyedCipherProvider();
+        return keyedCipherProvider.getCipher(encryptionMethod, tempKey, iv, encryptMode);
+    }
+
+    private String formatSaltForBcrypt(byte[] salt) {
+        if (salt == null || salt.length == 0) {
+            throw new IllegalArgumentException("The salt cannot be empty. To generate a salt, use BcryptCipherProvider#generateSalt()");
+        }
+
+        String rawSalt = new String(salt, StandardCharsets.UTF_8);
+        Matcher matcher = BCRYPT_SALT_FORMAT.matcher(rawSalt);
+
+        if (matcher.find()) {
+            return rawSalt;
+        } else {
+            throw new IllegalArgumentException("The salt must be of the format $2a$10$gUVbkVzp79H8YaCOsCVZNu. To generate a salt, use BcryptCipherProvider#generateSalt()");
+        }
+    }
+
+    @Override
+    public byte[] generateSalt() {
+        return BCrypt.gensalt(workFactor).getBytes(StandardCharsets.UTF_8);
+    }
+
+    @Override
+    public int getDefaultSaltLength() {
+        return DEFAULT_SALT_LENGTH;
+    }
+
+    protected int getWorkFactor() {
+        return workFactor;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherProviderFactory.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherProviderFactory.java
new file mode 100644
index 0000000..09004bf
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherProviderFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.KeyDerivationFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CipherProviderFactory {
+    private static final Logger logger = LoggerFactory.getLogger(CipherProviderFactory.class);
+
+    private static Map<KeyDerivationFunction, Class<? extends CipherProvider>> registeredCipherProviders;
+
+    static {
+        registeredCipherProviders = new HashMap<>();
+        registeredCipherProviders.put(KeyDerivationFunction.NIFI_LEGACY, NiFiLegacyCipherProvider.class);
+        registeredCipherProviders.put(KeyDerivationFunction.OPENSSL_EVP_BYTES_TO_KEY, OpenSSLPKCS5CipherProvider.class);
+        registeredCipherProviders.put(KeyDerivationFunction.PBKDF2, PBKDF2CipherProvider.class);
+        registeredCipherProviders.put(KeyDerivationFunction.BCRYPT, BcryptCipherProvider.class);
+        registeredCipherProviders.put(KeyDerivationFunction.SCRYPT, ScryptCipherProvider.class);
+        registeredCipherProviders.put(KeyDerivationFunction.NONE, AESKeyedCipherProvider.class);
+    }
+
+    public static CipherProvider getCipherProvider(KeyDerivationFunction kdf) {
+        logger.debug("{} KDFs registered", registeredCipherProviders.size());
+
+        if (registeredCipherProviders.containsKey(kdf)) {
+            Class<? extends CipherProvider> clazz = registeredCipherProviders.get(kdf);
+            try {
+                return clazz.newInstance();
+            } catch (Exception e) {
+               logger.error("Error instantiating new {} with default parameters for {}", clazz.getName(), kdf.getName());
+                throw new ProcessException("Error instantiating cipher provider");
+            }
+        }
+
+        throw new IllegalArgumentException("No cipher provider registered for " + kdf.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
index 2bf952b..369b015 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
@@ -20,6 +20,11 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.spec.InvalidKeySpecException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -29,6 +34,11 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import javax.crypto.Cipher;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.SecretKey;
+import javax.crypto.SecretKeyFactory;
+import javax.crypto.spec.PBEKeySpec;
+import javax.crypto.spec.PBEParameterSpec;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -41,6 +51,7 @@ public class CipherUtility {
     private static final Pattern KEY_LENGTH_PATTERN = Pattern.compile("([\\d]+)BIT");
 
     private static final Map<String, Integer> MAX_PASSWORD_LENGTH_BY_ALGORITHM;
+    private static final int DEFAULT_MAX_ALLOWED_KEY_LENGTH = 128;
 
     static {
         Map<String, Integer> aMap = new HashMap<>();
@@ -187,6 +198,12 @@ public class CipherUtility {
         return getValidKeyLengthsForAlgorithm(algorithm).contains(keyLength);
     }
 
+    /**
+     * Returns a list of valid key lengths in bits for this algorithm. If the algorithm cannot be parsed, an empty list is returned.
+     *
+     * @param algorithm the name of the algorithm
+     * @return a list of valid key lengths
+     */
     public static List<Integer> getValidKeyLengthsForAlgorithm(String algorithm) {
         List<Integer> validKeyLengths = new ArrayList<>();
         if (StringUtils.isEmpty(algorithm)) {
@@ -309,11 +326,84 @@ public class CipherUtility {
         if (encryptionMethod == null) {
             throw new IllegalArgumentException("Cannot evaluate an empty encryption method algorithm");
         }
+        return MAX_PASSWORD_LENGTH_BY_ALGORITHM.getOrDefault(encryptionMethod.getAlgorithm(), -1);
+    }
 
-        if (MAX_PASSWORD_LENGTH_BY_ALGORITHM.containsKey(encryptionMethod.getAlgorithm())) {
-            return MAX_PASSWORD_LENGTH_BY_ALGORITHM.get(encryptionMethod.getAlgorithm());
-        } else {
-            return -1;
+    public static boolean isUnlimitedStrengthCryptoSupported() {
+        try {
+            return (Cipher.getMaxAllowedKeyLength("AES") > DEFAULT_MAX_ALLOWED_KEY_LENGTH);
+        } catch (NoSuchAlgorithmException e) {
+            return false;
+        }
+    }
+
+    public static boolean isPBECipher(String algorithm) {
+        EncryptionMethod em = EncryptionMethod.forAlgorithm(algorithm);
+        return em != null && em.isPBECipher();
+    }
+
+    public static boolean isKeyedCipher(String algorithm) {
+        EncryptionMethod em = EncryptionMethod.forAlgorithm(algorithm);
+        return em != null && em.isKeyedCipher();
+    }
+
+    /**
+     * Initializes a {@link Cipher} object with the given PBE parameters.
+     *
+     * @param algorithm      the algorithm
+     * @param provider       the JCA provider
+     * @param password       the password
+     * @param salt           the salt
+     * @param iterationCount the KDF iteration count
+     * @param encryptMode    true to encrypt; false to decrypt
+     * @return the initialized Cipher
+     * @throws IllegalArgumentException if any parameter is invalid
+     */
+    public static Cipher initPBECipher(String algorithm, String provider, String password, byte[] salt, int iterationCount, boolean encryptMode) throws IllegalArgumentException {
+        try {
+            // Initialize secret key from password
+            final PBEKeySpec pbeKeySpec = new PBEKeySpec(password.toCharArray());
+            final SecretKeyFactory factory = SecretKeyFactory.getInstance(algorithm, provider);
+            SecretKey tempKey = factory.generateSecret(pbeKeySpec);
+
+            final PBEParameterSpec parameterSpec = new PBEParameterSpec(salt, iterationCount);
+            Cipher cipher = Cipher.getInstance(algorithm, provider);
+            cipher.init(encryptMode ? Cipher.ENCRYPT_MODE : Cipher.DECRYPT_MODE, tempKey, parameterSpec);
+            return cipher;
+        } catch (NoSuchAlgorithmException | NoSuchProviderException | InvalidKeySpecException | NoSuchPaddingException | InvalidKeyException | InvalidAlgorithmParameterException e) {
+            throw new IllegalArgumentException("One or more parameters to initialize the PBE cipher were invalid", e);
+        }
+    }
+
+    /**
+     * Returns the KDF iteration count for various PBE algorithms. These values were determined empirically from configured/chosen legacy values from the earlier version of the project.
+     * Code demonstrating this is available at {@link StringEncryptorTest#testPBEncryptionShouldBeExternallyConsistent}.
+     *
+     * @param algorithm the {@link EncryptionMethod#algorithm}
+     * @return the iteration count. Default is 0.
+     */
+    public static int getIterationCountForAlgorithm(String algorithm) {
+        int iterationCount = 0;
+        // DES/RC*/SHA-1/-256 algorithms use custom iteration counts
+        if (algorithm.matches("DES|RC|SHAA|SHA256")) {
+            iterationCount = 1000;
+        }
+        return iterationCount;
+    }
+
+    /**
+     * Returns the salt length for various PBE algorithms. These values were determined empirically from configured/chosen legacy values from the earlier version of the project.
+     * Code demonstrating this is available at {@link StringEncryptorTest#testPBEncryptionShouldBeExternallyConsistent}.
+     *
+     * @param algorithm the {@link EncryptionMethod#algorithm}
+     * @return the salt length in bytes. Default is 16.
+     */
+    public static int getSaltLengthForAlgorithm(String algorithm) {
+        int saltLength = 16;
+        // DES/RC* algorithms use custom iteration counts
+        if (algorithm.contains("DES") || algorithm.contains("RC")) {
+            saltLength = 8;
         }
+        return saltLength;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/KeyedCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/KeyedCipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/KeyedCipherProvider.java
index 719150f..17bfaa0 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/KeyedCipherProvider.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/KeyedCipherProvider.java
@@ -42,7 +42,7 @@ public abstract class KeyedCipherProvider implements CipherProvider {
      * @return the initialized cipher
      * @throws Exception if there is a problem initializing the cipher
      */
-    abstract Cipher getCipher(EncryptionMethod encryptionMethod, SecretKey key, byte[] iv, boolean encryptMode) throws Exception;
+    public abstract Cipher getCipher(EncryptionMethod encryptionMethod, SecretKey key, byte[] iv, boolean encryptMode) throws Exception;
 
     /**
      * Returns an initialized cipher for the specified algorithm. The IV will be generated internally (for encryption). If decryption is requested, it will throw an exception.
@@ -53,7 +53,7 @@ public abstract class KeyedCipherProvider implements CipherProvider {
      * @return the initialized cipher
      * @throws Exception if there is a problem initializing the cipher or if decryption is requested
      */
-    abstract Cipher getCipher(EncryptionMethod encryptionMethod, SecretKey key, boolean encryptMode) throws Exception;
+    public abstract Cipher getCipher(EncryptionMethod encryptionMethod, SecretKey key, boolean encryptMode) throws Exception;
 
     /**
      * Generates a new random IV of the correct length.

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/NiFiLegacyCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/NiFiLegacyCipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/NiFiLegacyCipherProvider.java
new file mode 100644
index 0000000..df8a54d
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/NiFiLegacyCipherProvider.java
@@ -0,0 +1,135 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.SecureRandom;
+import javax.crypto.Cipher;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a cipher initialized with the original NiFi key derivation process for password-based encryption (MD5 @ 1000 iterations). This is not a secure
+ * {@link org.apache.nifi.security.util.KeyDerivationFunction} (KDF) and should no longer be used.
+ * It is provided only for backward-compatibility with legacy data. A strong KDF should be selected for any future use.
+ *
+ * @see BcryptCipherProvider
+ * @see ScryptCipherProvider
+ * @see PBKDF2CipherProvider
+ */
+@Deprecated
+public class NiFiLegacyCipherProvider extends OpenSSLPKCS5CipherProvider implements PBECipherProvider {
+    private static final Logger logger = LoggerFactory.getLogger(NiFiLegacyCipherProvider.class);
+
+    // Legacy magic number value
+    private static final int ITERATION_COUNT = 1000;
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived using the NiFi legacy code, based on @see org.apache.nifi.crypto
+     * .OpenSSLPKCS5CipherProvider#getCipher(java.lang.String, java.lang.String, java.lang.String, byte[], boolean) [essentially {@code MD5(password || salt) * 1000 }].
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param keyLength        the desired key length in bits (ignored because OpenSSL ciphers provide key length in algorithm name)
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, int keyLength, boolean encryptMode) throws Exception {
+        try {
+            // This method is defined in the OpenSSL implementation and just uses a locally-overridden iteration count
+            return getInitializedCipher(encryptionMethod, password, salt, encryptMode);
+        } catch (IllegalArgumentException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new ProcessException("Error initializing the cipher", e);
+        }
+    }
+
+    public byte[] generateSalt(EncryptionMethod encryptionMethod) {
+        byte[] salt = new byte[calculateSaltLength(encryptionMethod)];
+        new SecureRandom().nextBytes(salt);
+        return salt;
+    }
+
+    protected void validateSalt(EncryptionMethod encryptionMethod, byte[] salt) {
+        final int saltLength = calculateSaltLength(encryptionMethod);
+        if (salt.length != saltLength && salt.length != 0) {
+            throw new IllegalArgumentException("Salt must be " + saltLength + " bytes or empty");
+        }
+    }
+
+    private int calculateSaltLength(EncryptionMethod encryptionMethod) {
+        try {
+            Cipher cipher = Cipher.getInstance(encryptionMethod.getAlgorithm(), encryptionMethod.getProvider());
+            return cipher.getBlockSize() > 0 ? cipher.getBlockSize() : getDefaultSaltLength();
+        } catch (Exception e) {
+            logger.warn("Encountered exception determining salt length from encryption method {}", encryptionMethod.getAlgorithm(), e);
+            final int defaultSaltLength = getDefaultSaltLength();
+            logger.warn("Returning default length: {} bytes", defaultSaltLength);
+            return defaultSaltLength;
+        }
+    }
+
+    @Override
+    public byte[] readSalt(InputStream in) throws IOException, ProcessException {
+        return readSalt(EncryptionMethod.AES_CBC, in);
+    }
+
+    /**
+     * Returns the salt provided as part of the cipher stream, or throws an exception if one cannot be detected.
+     * This method is only implemented by {@link NiFiLegacyCipherProvider} because the legacy salt generation was dependent on the cipher block size.
+     *
+     * @param encryptionMethod the encryption method
+     * @param in the cipher InputStream
+     * @return the salt
+     */
+    public byte[] readSalt(EncryptionMethod encryptionMethod, InputStream in) throws IOException {
+        if (in == null) {
+            throw new IllegalArgumentException("Cannot read salt from null InputStream");
+        }
+
+        // The first 8-16 bytes (depending on the cipher blocksize) of the input stream are the salt
+        final int saltLength = calculateSaltLength(encryptionMethod);
+        if (in.available() < saltLength) {
+            throw new ProcessException("The cipher stream is too small to contain the salt");
+        }
+        byte[] salt = new byte[saltLength];
+        StreamUtils.fillBuffer(in, salt);
+        return salt;
+    }
+
+    @Override
+    public void writeSalt(byte[] salt, OutputStream out) throws IOException {
+        if (out == null) {
+            throw new IllegalArgumentException("Cannot write salt to null OutputStream");
+        }
+        out.write(salt);
+    }
+
+    @Override
+    protected int getIterationCount() {
+        return ITERATION_COUNT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/OpenSSLPKCS5CipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/OpenSSLPKCS5CipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/OpenSSLPKCS5CipherProvider.java
new file mode 100644
index 0000000..597e516
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/OpenSSLPKCS5CipherProvider.java
@@ -0,0 +1,198 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.SecureRandom;
+import java.security.spec.InvalidKeySpecException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.SecretKey;
+import javax.crypto.SecretKeyFactory;
+import javax.crypto.spec.PBEKeySpec;
+import javax.crypto.spec.PBEParameterSpec;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OpenSSLPKCS5CipherProvider implements PBECipherProvider {
+    private static final Logger logger = LoggerFactory.getLogger(OpenSSLPKCS5CipherProvider.class);
+
+    // Legacy magic number value
+    private static final int ITERATION_COUNT = 0;
+    private static final int DEFAULT_SALT_LENGTH = 8;
+    private static final byte[] EMPTY_SALT = new byte[8];
+
+    private static final String OPENSSL_EVP_HEADER_MARKER = "Salted__";
+    private static final int OPENSSL_EVP_HEADER_SIZE = 8;
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived using the
+     * <a href="https://www.openssl.org/docs/manmaster/crypto/EVP_BytesToKey.html">OpenSSL EVP_BytesToKey proprietary KDF</a> [essentially {@code MD5(password || salt) }].
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param keyLength        the desired key length in bits (ignored because OpenSSL ciphers provide key length in algorithm name)
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, int keyLength, boolean encryptMode) throws Exception {
+        try {
+            return getInitializedCipher(encryptionMethod, password, salt, encryptMode);
+        } catch (IllegalArgumentException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new ProcessException("Error initializing the cipher", e);
+        }
+    }
+
+    /**
+     * Convenience method without key length parameter. See {@link OpenSSLPKCS5CipherProvider#getCipher(EncryptionMethod, String, int, boolean)}
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, boolean encryptMode) throws Exception {
+        return getCipher(encryptionMethod, password, new byte[0], -1, encryptMode);
+    }
+
+    /**
+     * Convenience method without key length parameter. See {@link OpenSSLPKCS5CipherProvider#getCipher(EncryptionMethod, String, byte[], int, boolean)}
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, boolean encryptMode) throws Exception {
+        return getCipher(encryptionMethod, password, salt, -1, encryptMode);
+    }
+
+    protected Cipher getInitializedCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, boolean encryptMode)
+            throws NoSuchAlgorithmException, NoSuchProviderException, InvalidKeySpecException, NoSuchPaddingException, InvalidKeyException,
+            InvalidAlgorithmParameterException {
+        if (encryptionMethod == null) {
+            throw new IllegalArgumentException("The encryption method must be specified");
+        }
+
+        if (StringUtils.isEmpty(password)) {
+            throw new IllegalArgumentException("Encryption with an empty password is not supported");
+        }
+
+        validateSalt(encryptionMethod, salt);
+
+        String algorithm = encryptionMethod.getAlgorithm();
+        String provider = encryptionMethod.getProvider();
+
+        // Initialize secret key from password
+        final PBEKeySpec pbeKeySpec = new PBEKeySpec(password.toCharArray());
+        final SecretKeyFactory factory = SecretKeyFactory.getInstance(algorithm, provider);
+        SecretKey tempKey = factory.generateSecret(pbeKeySpec);
+
+        final PBEParameterSpec parameterSpec = new PBEParameterSpec(salt, getIterationCount());
+        Cipher cipher = Cipher.getInstance(algorithm, provider);
+        cipher.init(encryptMode ? Cipher.ENCRYPT_MODE : Cipher.DECRYPT_MODE, tempKey, parameterSpec);
+        return cipher;
+    }
+
+    protected void validateSalt(EncryptionMethod encryptionMethod, byte[] salt) {
+        if (salt.length != DEFAULT_SALT_LENGTH && salt.length != 0) {
+            // This does not enforce ASCII encoding, just length
+            throw new IllegalArgumentException("Salt must be 8 bytes US-ASCII encoded or empty");
+        }
+    }
+
+    protected int getIterationCount() {
+        return ITERATION_COUNT;
+    }
+
+    @Override
+    public byte[] generateSalt() {
+        byte[] salt = new byte[getDefaultSaltLength()];
+        new SecureRandom().nextBytes(salt);
+        return salt;
+    }
+
+    @Override
+    public int getDefaultSaltLength() {
+        return DEFAULT_SALT_LENGTH;
+    }
+
+    /**
+     * Returns the salt provided as part of the cipher stream, or throws an exception if one cannot be detected.
+     *
+     * @param in the cipher InputStream
+     * @return the salt
+     */
+    @Override
+    public byte[] readSalt(InputStream in) throws IOException {
+        if (in == null) {
+            throw new IllegalArgumentException("Cannot read salt from null InputStream");
+        }
+
+        // The header and salt format is "Salted__salt x8b" in ASCII
+        byte[] salt = new byte[DEFAULT_SALT_LENGTH];
+
+        // Try to read the header and salt from the input
+        byte[] header = new byte[OPENSSL_EVP_HEADER_SIZE];
+
+        // Mark the stream in case there is no salt
+        in.mark(OPENSSL_EVP_HEADER_SIZE + 1);
+        StreamUtils.fillBuffer(in, header);
+
+        final byte[] headerMarkerBytes = OPENSSL_EVP_HEADER_MARKER.getBytes(StandardCharsets.US_ASCII);
+
+        if (!Arrays.equals(headerMarkerBytes, header)) {
+            // No salt present
+            salt = new byte[0];
+            // Reset the stream because we skipped 8 bytes of cipher text
+            in.reset();
+        }
+
+        StreamUtils.fillBuffer(in, salt);
+        return salt;
+    }
+
+    @Override
+    public void writeSalt(byte[] salt, OutputStream out) throws IOException {
+        if (out == null) {
+            throw new IllegalArgumentException("Cannot write salt to null OutputStream");
+        }
+
+        out.write(OPENSSL_EVP_HEADER_MARKER.getBytes(StandardCharsets.US_ASCII));
+        out.write(salt);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBECipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBECipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBECipherProvider.java
new file mode 100644
index 0000000..235af00
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBECipherProvider.java
@@ -0,0 +1,72 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import javax.crypto.Cipher;
+import org.apache.nifi.security.util.EncryptionMethod;
+
+public interface PBECipherProvider extends CipherProvider {
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
+     * <p/>
+     * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, int keyLength, boolean encryptMode) throws Exception;
+
+    /**
+     * Returns a random salt suitable for this cipher provider.
+     *
+     * @return a random salt
+     * @see PBECipherProvider#getDefaultSaltLength()
+     */
+    byte[] generateSalt();
+
+    /**
+     * Returns the default salt length for this implementation.
+     *
+     * @return the default salt length in bytes
+     */
+    int getDefaultSaltLength();
+
+    /**
+     * Returns the salt provided as part of the cipher stream, or throws an exception if one cannot be detected.
+     *
+     * @param in the cipher InputStream
+     * @return the salt
+     */
+    byte[] readSalt(InputStream in) throws IOException;
+
+    /**
+     * Writes the salt provided as part of the cipher stream, or throws an exception if it cannot be written.
+     *
+     * @param salt the salt
+     * @param out  the cipher OutputStream
+     */
+    void writeSalt(byte[] salt, OutputStream out) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBKDF2CipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBKDF2CipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBKDF2CipherProvider.java
new file mode 100644
index 0000000..fbad3b6
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/PBKDF2CipherProvider.java
@@ -0,0 +1,200 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.crypto.Digest;
+import org.bouncycastle.crypto.digests.MD5Digest;
+import org.bouncycastle.crypto.digests.SHA1Digest;
+import org.bouncycastle.crypto.digests.SHA256Digest;
+import org.bouncycastle.crypto.digests.SHA384Digest;
+import org.bouncycastle.crypto.digests.SHA512Digest;
+import org.bouncycastle.crypto.generators.PKCS5S2ParametersGenerator;
+import org.bouncycastle.crypto.params.KeyParameter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PBKDF2CipherProvider extends RandomIVPBECipherProvider {
+    private static final Logger logger = LoggerFactory.getLogger(PBKDF2CipherProvider.class);
+    private static final int DEFAULT_SALT_LENGTH = 16;
+
+    private final int iterationCount;
+    private final Digest prf;
+
+    private static final String DEFAULT_PRF = "SHA-512";
+    /**
+     * This can be calculated automatically using the code {@see PBKDF2CipherProviderGroovyTest#calculateMinimumIterationCount} or manually updated by a maintainer
+     */
+    private static final int DEFAULT_ITERATION_COUNT = 160_000;
+
+    /**
+     * Instantiates a PBKDF2 cipher provider with the default number of iterations and the default PRF. Currently 128,000 iterations and SHA-512.
+     */
+    public PBKDF2CipherProvider() {
+        this(DEFAULT_PRF, DEFAULT_ITERATION_COUNT);
+    }
+
+    /**
+     * Instantiates a PBKDF2 cipher provider with the specified number of iterations and the specified PRF. Currently supports MD5, SHA1, SHA256, SHA384, and SHA512. Unknown PRFs will default to
+     * SHA512.
+     *
+     * @param prf            a String representation of the PRF name, e.g. "SHA256", "SHA-384" "sha_512"
+     * @param iterationCount the number of iterations
+     */
+    public PBKDF2CipherProvider(String prf, int iterationCount) {
+        this.iterationCount = iterationCount;
+        if (iterationCount < DEFAULT_ITERATION_COUNT) {
+            logger.warn("The provided iteration count {} is below the recommended minimum {}", iterationCount, DEFAULT_ITERATION_COUNT);
+        }
+        this.prf = resolvePRF(prf);
+    }
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key is derived by the KDF of the implementation. The IV is provided externally to allow for non-deterministic IVs, as IVs
+     * deterministically derived from the password are a potential vulnerability and compromise semantic security. See
+     * <a href="http://crypto.stackexchange.com/a/3970/12569">Ilmari Karonen's answer on Crypto Stack Exchange</a>
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param iv               the IV
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception {
+        try {
+            return getInitializedCipher(encryptionMethod, password, salt, iv, keyLength, encryptMode);
+        } catch (IllegalArgumentException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new ProcessException("Error initializing the cipher", e);
+        }
+    }
+
+    @Override
+    Logger getLogger() {
+        return logger;
+    }
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
+     *
+     * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, int keyLength, boolean encryptMode) throws Exception {
+        return getCipher(encryptionMethod, password, salt, new byte[0], keyLength, encryptMode);
+    }
+
+    protected Cipher getInitializedCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception {
+        if (encryptionMethod == null) {
+            throw new IllegalArgumentException("The encryption method must be specified");
+        }
+
+        if (!encryptionMethod.isCompatibleWithStrongKDFs()) {
+            throw new IllegalArgumentException(encryptionMethod.name() + " is not compatible with PBKDF2");
+        }
+
+        String algorithm = encryptionMethod.getAlgorithm();
+
+        final String cipherName = CipherUtility.parseCipherFromAlgorithm(algorithm);
+        if (!CipherUtility.isValidKeyLength(keyLength, cipherName)) {
+            throw new IllegalArgumentException(String.valueOf(keyLength) + " is not a valid key length for " + cipherName);
+        }
+
+        if (StringUtils.isEmpty(password)) {
+            throw new IllegalArgumentException("Encryption with an empty password is not supported");
+        }
+
+        if (salt == null || salt.length < DEFAULT_SALT_LENGTH) {
+            throw new IllegalArgumentException("The salt must be at least " + DEFAULT_SALT_LENGTH + " bytes. To generate a salt, use PBKDF2CipherProvider#generateSalt()");
+        }
+
+        PKCS5S2ParametersGenerator gen = new PKCS5S2ParametersGenerator(this.prf);
+        gen.init(password.getBytes(StandardCharsets.UTF_8), salt, getIterationCount());
+        byte[] dk = ((KeyParameter) gen.generateDerivedParameters(keyLength)).getKey();
+        SecretKey tempKey = new SecretKeySpec(dk, algorithm);
+
+        KeyedCipherProvider keyedCipherProvider = new AESKeyedCipherProvider();
+        return keyedCipherProvider.getCipher(encryptionMethod, tempKey, iv, encryptMode);
+    }
+
+    @Override
+    public byte[] generateSalt() {
+        byte[] salt = new byte[DEFAULT_SALT_LENGTH];
+        new SecureRandom().nextBytes(salt);
+        return salt;
+    }
+
+    @Override
+    public int getDefaultSaltLength() {
+        return DEFAULT_SALT_LENGTH;
+    }
+
+    protected int getIterationCount() {
+        return iterationCount;
+    }
+
+    protected String getPRFName() {
+        if (prf != null) {
+            return prf.getAlgorithmName();
+        } else {
+            return "No PRF enabled";
+        }
+    }
+
+    private Digest resolvePRF(final String prf) {
+        if (StringUtils.isEmpty(prf)) {
+            throw new IllegalArgumentException("Cannot resolve empty PRF");
+        }
+        String formattedPRF = prf.toLowerCase().replaceAll("[\\W]+", "");
+        logger.debug("Resolved PRF {} to {}", prf, formattedPRF);
+        switch (formattedPRF) {
+            case "md5":
+                return new MD5Digest();
+            case "sha1":
+                return new SHA1Digest();
+            case "sha384":
+                return new SHA384Digest();
+            case "sha256":
+                return new SHA256Digest();
+            case "sha512":
+                return new SHA512Digest();
+            default:
+                logger.warn("Could not resolve PRF {}. Using default PRF {} instead", prf, DEFAULT_PRF);
+                return new SHA512Digest();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/RandomIVPBECipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/RandomIVPBECipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/RandomIVPBECipherProvider.java
new file mode 100644
index 0000000..66fd1ba
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/RandomIVPBECipherProvider.java
@@ -0,0 +1,70 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import javax.crypto.Cipher;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.slf4j.Logger;
+
+public abstract class RandomIVPBECipherProvider implements PBECipherProvider {
+    static final byte[] SALT_DELIMITER = "NiFiSALT".getBytes(StandardCharsets.UTF_8);
+    static final int MAX_SALT_LIMIT = 128;
+    static final byte[] IV_DELIMITER = "NiFiIV".getBytes(StandardCharsets.UTF_8);
+    // This is 16 bytes for AES but can vary for other ciphers
+    static final int MAX_IV_LIMIT = 16;
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key is derived by the KDF of the implementation. The IV is provided externally to allow for non-deterministic IVs, as IVs
+     * deterministically derived from the password are a potential vulnerability and compromise semantic security. See
+     * <a href="http://crypto.stackexchange.com/a/3970/12569">Ilmari Karonen's answer on Crypto Stack Exchange</a>
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the salt
+     * @param iv               the IV
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    abstract Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception;
+
+    abstract Logger getLogger();
+
+    @Override
+    public byte[] readSalt(InputStream in) throws IOException, ProcessException {
+       return CipherUtility.readBytesFromInputStream(in, "salt", MAX_SALT_LIMIT, SALT_DELIMITER);
+    }
+
+    @Override
+    public void writeSalt(byte[] salt, OutputStream out) throws IOException {
+        CipherUtility.writeBytesToOutputStream(out, salt, "salt", SALT_DELIMITER);
+    }
+
+    public byte[] readIV(InputStream in) throws IOException, ProcessException {
+        return CipherUtility.readBytesFromInputStream(in, "IV", MAX_IV_LIMIT, IV_DELIMITER);
+    }
+
+    public void writeIV(byte[] iv, OutputStream out) throws IOException {
+        CipherUtility.writeBytesToOutputStream(out, iv, "IV", IV_DELIMITER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/2c1f5b49/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/ScryptCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/ScryptCipherProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/ScryptCipherProvider.java
new file mode 100644
index 0000000..b532d8e
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/ScryptCipherProvider.java
@@ -0,0 +1,285 @@
+/*
+ * 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.nifi.security.util.crypto;
+
+import java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.security.util.crypto.scrypt.Scrypt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ScryptCipherProvider extends RandomIVPBECipherProvider {
+    private static final Logger logger = LoggerFactory.getLogger(ScryptCipherProvider.class);
+
+    private final int n;
+    private final int r;
+    private final int p;
+    /**
+     * These values can be calculated automatically using the code {@see ScryptCipherProviderGroovyTest#calculateMinimumParameters} or manually updated by a maintainer
+     */
+    private static final int DEFAULT_N = Double.valueOf(Math.pow(2, 14)).intValue();
+    private static final int DEFAULT_R = 8;
+    private static final int DEFAULT_P = 1;
+
+    private static final Pattern SCRYPT_SALT_FORMAT = Pattern.compile("^\\$s0\\$[a-f0-9]{5,16}\\$[\\w\\/\\.]{12,44}");
+    private static final Pattern MCRYPT_SALT_FORMAT = Pattern.compile("^\\$\\d+\\$\\d+\\$\\d+\\$[a-f0-9]{16,64}");
+
+    /**
+     * Instantiates a Scrypt cipher provider with the default parameters N=2^14, r=8, p=1.
+     */
+    public ScryptCipherProvider() {
+        this(DEFAULT_N, DEFAULT_R, DEFAULT_P);
+    }
+
+    /**
+     * Instantiates a Scrypt cipher provider with the specified N, r, p values.
+     *
+     * @param n the number of iterations
+     * @param r the block size in bytes
+     * @param p the parallelization factor
+     */
+    public ScryptCipherProvider(int n, int r, int p) {
+        this.n = n;
+        this.r = r;
+        this.p = p;
+        if (n < DEFAULT_N) {
+            logger.warn("The provided iteration count {} is below the recommended minimum {}", n, DEFAULT_N);
+        }
+        if (r < DEFAULT_R) {
+            logger.warn("The provided block size {} is below the recommended minimum {}", r, DEFAULT_R);
+        }
+        if (p < DEFAULT_P) {
+            logger.warn("The provided parallelization factor {} is below the recommended minimum {}", p, DEFAULT_P);
+        }
+    }
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key is derived by the KDF of the implementation. The IV is provided externally to allow for non-deterministic IVs, as IVs
+     * deterministically derived from the password are a potential vulnerability and compromise semantic security. See
+     * <a href="http://crypto.stackexchange.com/a/3970/12569">Ilmari Karonen's answer on Crypto Stack Exchange</a>
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the complete salt (e.g. {@code "$2a$10$gUVbkVzp79H8YaCOsCVZNu".getBytes(StandardCharsets.UTF_8)})
+     * @param iv               the IV
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception {
+        try {
+            return getInitializedCipher(encryptionMethod, password, salt, iv, keyLength, encryptMode);
+        } catch (IllegalArgumentException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new ProcessException("Error initializing the cipher", e);
+        }
+    }
+
+    @Override
+    Logger getLogger() {
+        return logger;
+    }
+
+    /**
+     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
+     *
+     * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
+     *
+     * @param encryptionMethod the {@link EncryptionMethod}
+     * @param password         the secret input
+     * @param salt             the complete salt (e.g. {@code "$s0$20101$gUVbkVzp79H8YaCOsCVZNu".getBytes(StandardCharsets.UTF_8)})
+     * @param keyLength        the desired key length in bits
+     * @param encryptMode      true for encrypt, false for decrypt
+     * @return the initialized cipher
+     * @throws Exception if there is a problem initializing the cipher
+     */
+    @Override
+    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, int keyLength, boolean encryptMode) throws Exception {
+        return getCipher(encryptionMethod, password, salt, new byte[0], keyLength, encryptMode);
+    }
+
+    protected Cipher getInitializedCipher(EncryptionMethod encryptionMethod, String password, byte[] salt, byte[] iv, int keyLength, boolean encryptMode) throws Exception {
+        if (encryptionMethod == null) {
+            throw new IllegalArgumentException("The encryption method must be specified");
+        }
+        if (!encryptionMethod.isCompatibleWithStrongKDFs()) {
+            throw new IllegalArgumentException(encryptionMethod.name() + " is not compatible with Scrypt");
+        }
+
+        if (StringUtils.isEmpty(password)) {
+            throw new IllegalArgumentException("Encryption with an empty password is not supported");
+        }
+
+        String algorithm = encryptionMethod.getAlgorithm();
+
+        final String cipherName = CipherUtility.parseCipherFromAlgorithm(algorithm);
+        if (!CipherUtility.isValidKeyLength(keyLength, cipherName)) {
+            throw new IllegalArgumentException(String.valueOf(keyLength) + " is not a valid key length for " + cipherName);
+        }
+
+        String scryptSalt = formatSaltForScrypt(salt);
+        List<Integer> params = new ArrayList<>(3);
+        byte[] rawSalt = new byte[Scrypt.getDefaultSaltLength()];
+
+        parseSalt(scryptSalt, rawSalt, params);
+
+        String hash = Scrypt.scrypt(password, rawSalt, params.get(0), params.get(1), params.get(2), keyLength);
+
+        // Split out the derived key from the hash and form a key object
+        final String[] hashComponents = hash.split("\\$");
+        final int HASH_INDEX = 4;
+        if (hashComponents.length < HASH_INDEX) {
+            throw new ProcessException("There was an error generating a scrypt hash -- the resulting hash was not properly formatted");
+        }
+        byte[] keyBytes = Base64.decodeBase64(hashComponents[HASH_INDEX]);
+        SecretKey tempKey = new SecretKeySpec(keyBytes, algorithm);
+
+        KeyedCipherProvider keyedCipherProvider = new AESKeyedCipherProvider();
+        return keyedCipherProvider.getCipher(encryptionMethod, tempKey, iv, encryptMode);
+    }
+
+    private void parseSalt(String scryptSalt, byte[] rawSalt, List<Integer> params) {
+        if (StringUtils.isEmpty(scryptSalt)) {
+            throw new IllegalArgumentException("Cannot parse empty salt");
+        }
+
+        /** Salt format is $s0$params$saltB64 where params is encoded according to
+         *  {@link Scrypt#parseParameters(String)}*/
+        final String[] saltComponents = scryptSalt.split("\\$");
+        if (saltComponents.length < 4) {
+            throw new IllegalArgumentException("Could not parse salt");
+        }
+        byte[] salt = Base64.decodeBase64(saltComponents[3]);
+        if (rawSalt.length < salt.length) {
+            byte[] tempBytes = new byte[salt.length];
+            System.arraycopy(rawSalt, 0, tempBytes, 0, rawSalt.length);
+            rawSalt = tempBytes;
+        }
+        System.arraycopy(salt, 0, rawSalt, 0, salt.length);
+
+        if (params == null) {
+            params = new ArrayList<>(3);
+        }
+        params.addAll(Scrypt.parseParameters(saltComponents[2]));
+    }
+
+    /**
+     * Formats the salt into a string which Scrypt can understand containing the N, r, p values along with the salt value. If the provided salt contains all values, the response will be unchanged.
+     * If it only contains the raw salt value, the resulting return value will also include the current instance version, N, r, and p.
+     *
+     * @param salt the provided salt
+     * @return the properly-formatted and complete salt
+     */
+    private String formatSaltForScrypt(byte[] salt) {
+        if (salt == null || salt.length == 0) {
+            throw new IllegalArgumentException("The salt cannot be empty. To generate a salt, use ScryptCipherProvider#generateSalt()");
+        }
+
+        String saltString = new String(salt, StandardCharsets.UTF_8);
+        Matcher matcher = SCRYPT_SALT_FORMAT.matcher(saltString);
+
+        if (matcher.find()) {
+            return saltString;
+        } else {
+            if (saltString.startsWith("$")) {
+                logger.warn("Salt starts with $ but is not valid scrypt salt");
+                matcher = MCRYPT_SALT_FORMAT.matcher(saltString);
+                if (matcher.find()) {
+                    logger.warn("The salt appears to be of the modified mcrypt format. Use ScryptCipherProvider#translateSalt(mcryptSalt) to form a valid salt");
+                    return translateSalt(saltString);
+                }
+
+                logger.info("Salt is not modified mcrypt format");
+            }
+            logger.info("Treating as raw salt bytes");
+
+            // Ensure the length of the salt
+            int saltLength = salt.length;
+            if (saltLength < 8 || saltLength > 32) {
+                throw new IllegalArgumentException("The raw salt must be between 8 and 32 bytes");
+            }
+            return Scrypt.formatSalt(salt, n, r, p);
+        }
+    }
+
+    /**
+     * Translates a salt from the mcrypt format {@code $n$r$p$salt_hex} to the Java scrypt format {@code $s0$params$saltBase64}.
+     *
+     * @param mcryptSalt the mcrypt-formatted salt string
+     * @return the formatted salt to use with Java Scrypt
+     */
+    public String translateSalt(String mcryptSalt) {
+        if (StringUtils.isEmpty(mcryptSalt)) {
+            throw new IllegalArgumentException("Cannot translate empty salt");
+        }
+
+        // Format should be $n$r$p$saltHex
+        Matcher matcher = MCRYPT_SALT_FORMAT.matcher(mcryptSalt);
+        if (!matcher.matches()) {
+            throw new IllegalArgumentException("Salt is not valid mcrypt format of $n$r$p$saltHex");
+        }
+
+        String[] components = mcryptSalt.split("\\$");
+        try {
+            return Scrypt.formatSalt(Hex.decodeHex(components[4].toCharArray()), Integer.valueOf(components[1]), Integer.valueOf(components[2]), Integer.valueOf(components[3]));
+        } catch (DecoderException e) {
+            final String msg = "Mcrypt salt was not properly hex-encoded";
+            logger.warn(msg);
+            throw new IllegalArgumentException(msg);
+        }
+    }
+
+    @Override
+    public byte[] generateSalt() {
+        byte[] salt = new byte[Scrypt.getDefaultSaltLength()];
+        new SecureRandom().nextBytes(salt);
+        return Scrypt.formatSalt(salt, n, r, p).getBytes(StandardCharsets.UTF_8);
+    }
+
+    @Override
+    public int getDefaultSaltLength() {
+        return Scrypt.getDefaultSaltLength();
+    }
+
+    protected int getN() {
+        return n;
+    }
+
+    protected int getR() {
+        return r;
+    }
+
+    protected int getP() {
+        return p;
+    }
+}