You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "exceptionfactory (via GitHub)" <gi...@apache.org> on 2023/05/15 14:45:04 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7238: NIFI-11531: refactored tests in nifi-security-utils

exceptionfactory commented on code in PR #7238:
URL: https://github.com/apache/nifi/pull/7238#discussion_r1193937717


##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/AESKeyedCipherProviderTest.java:
##########
@@ -14,311 +14,298 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.security.util.crypto
+package org.apache.nifi.security.util.crypto;
 
-import org.apache.commons.codec.binary.Hex
-import org.apache.nifi.security.util.EncryptionMethod
-import org.bouncycastle.jce.provider.BouncyCastleProvider
-import org.junit.jupiter.api.BeforeAll
-import org.junit.jupiter.api.Test
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
-import javax.crypto.Cipher
-import javax.crypto.SecretKey
-import javax.crypto.spec.SecretKeySpec
-import java.security.SecureRandom
-import java.security.Security
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
 
-import static org.junit.jupiter.api.Assertions.assertEquals
-import static org.junit.jupiter.api.Assertions.assertFalse
-import static org.junit.jupiter.api.Assertions.assertThrows
-import static org.junit.jupiter.api.Assertions.assertTrue
-import static org.junit.jupiter.api.Assumptions.assumeTrue
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
-class AESKeyedCipherProviderGroovyTest {
-    private static final Logger logger = LoggerFactory.getLogger(AESKeyedCipherProviderGroovyTest.class)
+public class AESKeyedCipherProviderTest {
+    private static final String KEY_HEX = "0123456789ABCDEFFEDCBA9876543210";
 
-    private static final String KEY_HEX = "0123456789ABCDEFFEDCBA9876543210"
+    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess";
 
-    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess"
+    private static final List<EncryptionMethod> keyedEncryptionMethods = Arrays.stream(EncryptionMethod.values())
+            .filter(EncryptionMethod::isKeyedCipher)
+            .collect(Collectors.toList());
 
-    private static final List<EncryptionMethod> keyedEncryptionMethods = EncryptionMethod.values().findAll { it.keyedCipher }
+    private static SecretKey key;
 
-    private static final SecretKey key = new SecretKeySpec(Hex.decodeHex(KEY_HEX as char[]), "AES")
 
     @BeforeAll
     static void setUpOnce() throws Exception {
-        Security.addProvider(new BouncyCastleProvider())
+        Security.addProvider(new BouncyCastleProvider());
 
-        logger.metaClass.methodMissing = { String name, args ->
-            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        try {
+            key = new SecretKeySpec(Hex.decodeHex(KEY_HEX.toCharArray()), "AES");
+        } catch (final DecoderException e) {
+            throw new RuntimeException(e);
         }
     }
 
     private static boolean isUnlimitedStrengthCryptoAvailable() {
-        Cipher.getMaxAllowedKeyLength("AES") > 128
+        try {
+            return Cipher.getMaxAllowedKeyLength("AES") > 128;
+        } catch (final NoSuchAlgorithmException e) {
+            throw new RuntimeException(e);
+        }
     }

Review Comment:
   This check is no longer needed, so the method can be removed and anything that depends on it can be adjusted to always test unlimited strength values.



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/AESKeyedCipherProviderTest.java:
##########
@@ -14,311 +14,298 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.security.util.crypto
+package org.apache.nifi.security.util.crypto;
 
-import org.apache.commons.codec.binary.Hex
-import org.apache.nifi.security.util.EncryptionMethod
-import org.bouncycastle.jce.provider.BouncyCastleProvider
-import org.junit.jupiter.api.BeforeAll
-import org.junit.jupiter.api.Test
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
-import javax.crypto.Cipher
-import javax.crypto.SecretKey
-import javax.crypto.spec.SecretKeySpec
-import java.security.SecureRandom
-import java.security.Security
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
 
-import static org.junit.jupiter.api.Assertions.assertEquals
-import static org.junit.jupiter.api.Assertions.assertFalse
-import static org.junit.jupiter.api.Assertions.assertThrows
-import static org.junit.jupiter.api.Assertions.assertTrue
-import static org.junit.jupiter.api.Assumptions.assumeTrue
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
-class AESKeyedCipherProviderGroovyTest {
-    private static final Logger logger = LoggerFactory.getLogger(AESKeyedCipherProviderGroovyTest.class)
+public class AESKeyedCipherProviderTest {
+    private static final String KEY_HEX = "0123456789ABCDEFFEDCBA9876543210";
 
-    private static final String KEY_HEX = "0123456789ABCDEFFEDCBA9876543210"
+    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess";
 
-    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess"
+    private static final List<EncryptionMethod> keyedEncryptionMethods = Arrays.stream(EncryptionMethod.values())
+            .filter(EncryptionMethod::isKeyedCipher)
+            .collect(Collectors.toList());
 
-    private static final List<EncryptionMethod> keyedEncryptionMethods = EncryptionMethod.values().findAll { it.keyedCipher }
+    private static SecretKey key;
 
-    private static final SecretKey key = new SecretKeySpec(Hex.decodeHex(KEY_HEX as char[]), "AES")
 
     @BeforeAll
     static void setUpOnce() throws Exception {
-        Security.addProvider(new BouncyCastleProvider())
+        Security.addProvider(new BouncyCastleProvider());
 
-        logger.metaClass.methodMissing = { String name, args ->
-            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        try {
+            key = new SecretKeySpec(Hex.decodeHex(KEY_HEX.toCharArray()), "AES");
+        } catch (final DecoderException e) {
+            throw new RuntimeException(e);
         }
     }
 
     private static boolean isUnlimitedStrengthCryptoAvailable() {
-        Cipher.getMaxAllowedKeyLength("AES") > 128
+        try {
+            return Cipher.getMaxAllowedKeyLength("AES") > 128;
+        } catch (final NoSuchAlgorithmException e) {
+            throw new RuntimeException(e);
+        }
     }
 
     @Test
     void testGetCipherShouldBeInternallyConsistent() throws Exception {
         // Arrange
-        KeyedCipherProvider cipherProvider = new AESKeyedCipherProvider()
+        KeyedCipherProvider cipherProvider = new AESKeyedCipherProvider();
 
         // Act
         for (EncryptionMethod em : keyedEncryptionMethods) {
-            logger.info("Using algorithm: ${em.getAlgorithm()}")
-
             // Initialize a cipher for encryption
-            Cipher cipher = cipherProvider.getCipher(em, key, true)
-            byte[] iv = cipher.getIV()
-            logger.info("IV: ${Hex.encodeHexString(iv)}")
+            Cipher cipher = cipherProvider.getCipher(em, key, true);
+            byte[] iv = cipher.getIV();
 
-            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"))
-            logger.info("Cipher text: ${Hex.encodeHexString(cipherBytes)} ${cipherBytes.length}")
+            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"));
 
-            cipher = cipherProvider.getCipher(em, key, iv, false)
-            byte[] recoveredBytes = cipher.doFinal(cipherBytes)
-            String recovered = new String(recoveredBytes, "UTF-8")
-            logger.info("Recovered: ${recovered}")
+            cipher = cipherProvider.getCipher(em, key, iv, false);
+            byte[] recoveredBytes = cipher.doFinal(cipherBytes);
+            String recovered = new String(recoveredBytes, "UTF-8");
 
             // Assert
-            assertEquals(PLAINTEXT, recovered)
+            assertEquals(PLAINTEXT, recovered);
         }
     }
 
     @Test
     void testGetCipherWithExternalIVShouldBeInternallyConsistent() throws Exception {
         // Arrange
-        KeyedCipherProvider cipherProvider = new AESKeyedCipherProvider()
+        KeyedCipherProvider cipherProvider = new AESKeyedCipherProvider();
 
         // Act
-        keyedEncryptionMethods.each { EncryptionMethod em ->
-            logger.info("Using algorithm: ${em.getAlgorithm()}")
-            byte[] iv = cipherProvider.generateIV()
-            logger.info("IV: ${Hex.encodeHexString(iv)}")
+        for (final EncryptionMethod em : keyedEncryptionMethods) {
+            byte[] iv = cipherProvider.generateIV();
 
             // Initialize a cipher for encryption
-            Cipher cipher = cipherProvider.getCipher(em, key, iv, true)
+            Cipher cipher = cipherProvider.getCipher(em, key, iv, true);
 
-            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"))
-            logger.info("Cipher text: ${Hex.encodeHexString(cipherBytes)} ${cipherBytes.length}")
+            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"));
 
-            cipher = cipherProvider.getCipher(em, key, iv, false)
-            byte[] recoveredBytes = cipher.doFinal(cipherBytes)
-            String recovered = new String(recoveredBytes, "UTF-8")
-            logger.info("Recovered: ${recovered}")
+            cipher = cipherProvider.getCipher(em, key, iv, false);
+            byte[] recoveredBytes = cipher.doFinal(cipherBytes);
+            String recovered = new String(recoveredBytes, "UTF-8");
 
             // Assert
-            assertEquals(PLAINTEXT, recovered)
+            assertEquals(PLAINTEXT, recovered);
         }
     }
 
     @Test
     void testGetCipherWithUnlimitedStrengthShouldBeInternallyConsistent() throws Exception {
         // Arrange
-        assumeTrue(isUnlimitedStrengthCryptoAvailable(), "Test is being skipped due to this JVM lacking JCE Unlimited Strength Jurisdiction Policy file.")
+        assumeTrue(isUnlimitedStrengthCryptoAvailable(), "Test is being skipped due to this JVM lacking JCE Unlimited Strength Jurisdiction Policy file.");

Review Comment:
   As mentioned above, this check can be removed.



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/BcryptSecureHasherTest.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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 at.favre.lib.crypto.bcrypt.Radix64Encoder;
+import org.bouncycastle.util.encoders.Hex;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.EnabledIfSystemProperty;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class BcryptSecureHasherTest {
+
+    @Test
+    void testShouldBeDeterministicWithStaticSalt() {
+        // Arrange
+        int cost = 4;
+
+        int testIterations = 10;
+        byte[] inputBytes = "This is a sensitive value".getBytes();
+
+        final String EXPECTED_HASH_HEX = "24326124303424526b6a4559512f526245447959554b6553304471622e596b4c5331655a2e6c61586550484c69464d783937564c566d47354250454f";
+
+        BcryptSecureHasher bcryptSH = new BcryptSecureHasher(cost);
+
+        final List<String> results = new ArrayList<>();
+
+        // Act
+        for (int i = 0; i < testIterations; i++) {
+            byte[] hash = bcryptSH.hashRaw(inputBytes);
+            String hashHex = new String(Hex.encode(hash));
+            results.add(hashHex);
+        }
+
+        // Assert
+        results.forEach(result -> assertEquals(EXPECTED_HASH_HEX, result));
+    }
+
+    @Test
+    void testShouldBeDifferentWithRandomSalt() {
+        // Arrange
+        int cost = 4;
+        int saltLength = 16;
+
+        int testIterations = 10;
+        byte[] inputBytes = "This is a sensitive value".getBytes();
+
+        final String EXPECTED_HASH_HEX = "24326124303424546d6c47615342546447463061574d6755324673642e38675a347a6149356d6b4d50594c542e344e68337962455a4678384b676a75";
+
+        BcryptSecureHasher bcryptSH = new BcryptSecureHasher(cost, saltLength);
+
+        final List<String> results = new ArrayList<>();
+
+        // Act
+        for (int i = 0; i < testIterations; i++) {
+            byte[] hash = bcryptSH.hashRaw(inputBytes);
+            String hashHex = new String(Hex.encode(hash));
+            results.add(hashHex);
+        }
+
+        // Assert
+        assertEquals(results.size(), results.stream().distinct().collect(Collectors.toList()).size());
+        results.forEach(result -> assertNotEquals(EXPECTED_HASH_HEX, result));
+    }
+
+    @Test
+    void testShouldHandleArbitrarySalt() {
+        // Arrange
+        int cost = 4;
+
+        final String input = "This is a sensitive value";
+        byte[] inputBytes = input.getBytes();
+
+        final String EXPECTED_HASH_HEX = "24326124303424526b6a4559512f526245447959554b6553304471622e596b4c5331655a2e6c61586550484c69464d783937564c566d47354250454f";
+        final String EXPECTED_HASH_BASE64 = "JDJhJDA0JFJrakVZUS9SYkVEeVlVS2VTMERxYi5Za0xTMWVaLmxhWGVQSExpRk14OTdWTFZtRzVCUEVP";
+        final byte[] EXPECTED_HASH_BYTES = Hex.decode(EXPECTED_HASH_HEX);
+
+        // Static salt instance
+        BcryptSecureHasher staticSaltHasher = new BcryptSecureHasher(cost);
+        BcryptSecureHasher arbitrarySaltHasher = new BcryptSecureHasher(cost, 16);
+
+        final byte[] STATIC_SALT = "NiFi Static Salt".getBytes(StandardCharsets.UTF_8);
+        final String DIFFERENT_STATIC_SALT = "Diff Static Salt";
+
+        // Act
+        byte[] staticSaltHash = staticSaltHasher.hashRaw(inputBytes);
+        byte[] arbitrarySaltHash = arbitrarySaltHasher.hashRaw(inputBytes, STATIC_SALT);
+        byte[] differentArbitrarySaltHash = arbitrarySaltHasher.hashRaw(inputBytes, DIFFERENT_STATIC_SALT.getBytes(StandardCharsets.UTF_8));
+        byte[] differentSaltHash = arbitrarySaltHasher.hashRaw(inputBytes);
+
+        String staticSaltHashHex = staticSaltHasher.hashHex(input);
+        String arbitrarySaltHashHex = arbitrarySaltHasher.hashHex(input, new String(STATIC_SALT, StandardCharsets.UTF_8));
+        String differentArbitrarySaltHashHex = arbitrarySaltHasher.hashHex(input, DIFFERENT_STATIC_SALT);
+        String differentSaltHashHex = arbitrarySaltHasher.hashHex(input);
+
+        String staticSaltHashBase64 = staticSaltHasher.hashBase64(input);
+        String arbitrarySaltHashBase64 = arbitrarySaltHasher.hashBase64(input, new String(STATIC_SALT, StandardCharsets.UTF_8));
+        String differentArbitrarySaltHashBase64 = arbitrarySaltHasher.hashBase64(input, DIFFERENT_STATIC_SALT);
+        String differentSaltHashBase64 = arbitrarySaltHasher.hashBase64(input);
+
+        // Assert
+        assertArrayEquals(EXPECTED_HASH_BYTES, staticSaltHash);
+        assertArrayEquals(EXPECTED_HASH_BYTES, arbitrarySaltHash);
+        assertFalse(Arrays.equals(EXPECTED_HASH_BYTES, differentArbitrarySaltHash));
+        assertFalse(Arrays.equals(EXPECTED_HASH_BYTES, differentSaltHash));
+
+        assertEquals(EXPECTED_HASH_HEX, staticSaltHashHex);
+        assertEquals(EXPECTED_HASH_HEX, arbitrarySaltHashHex);
+        assertNotEquals(EXPECTED_HASH_HEX, differentArbitrarySaltHashHex);
+        assertNotEquals(EXPECTED_HASH_HEX, differentSaltHashHex);
+
+        assertEquals(EXPECTED_HASH_BASE64, staticSaltHashBase64);
+        assertEquals(EXPECTED_HASH_BASE64, arbitrarySaltHashBase64);
+        assertNotEquals(EXPECTED_HASH_BASE64, differentArbitrarySaltHashBase64);
+        assertNotEquals(EXPECTED_HASH_BASE64, differentSaltHashBase64);
+    }
+
+    @Test
+    void testShouldValidateArbitrarySalt() {
+        // Arrange
+        int cost = 4;
+
+        final String input = "This is a sensitive value";
+        byte[] inputBytes = input.getBytes();
+
+        // Static salt instance
+        BcryptSecureHasher secureHasher = new BcryptSecureHasher(cost, 16);
+        final byte[] STATIC_SALT = "bad_sal".getBytes();
+
+        assertThrows(IllegalArgumentException.class, () -> new BcryptSecureHasher(cost, 7));
+
+        assertThrows(RuntimeException.class, () -> secureHasher.hashRaw(inputBytes, STATIC_SALT));
+        assertThrows(RuntimeException.class, () -> secureHasher.hashHex(input, new String(STATIC_SALT, StandardCharsets.UTF_8)));
+        assertThrows(RuntimeException.class, () -> secureHasher.hashBase64(input, new String(STATIC_SALT, StandardCharsets.UTF_8)));
+    }
+
+    @Test
+    void testShouldFormatHex() {
+        // Arrange
+        String input = "This is a sensitive value";
+
+        final String EXPECTED_HASH_HEX = "24326124313224526b6a4559512f526245447959554b6553304471622e5852696135344d4e356c5a44515243575874516c4c696d476669635a776871";
+
+        BcryptSecureHasher bcryptSH = new BcryptSecureHasher();
+
+        // Act
+        String hashHex = bcryptSH.hashHex(input);
+
+        // Assert
+        assertEquals(EXPECTED_HASH_HEX, hashHex);
+    }
+
+    @Test
+    void testShouldFormatBase64() {
+        // Arrange
+        String input = "This is a sensitive value";
+
+        final String EXPECTED_HASH_BASE64 = "JDJhJDEyJFJrakVZUS9SYkVEeVlVS2VTMERxYi5YUmlhNTRNTjVsWkRRUkNXWHRRbExpbUdmaWNad2hx";
+
+        BcryptSecureHasher bcryptSH = new BcryptSecureHasher();
+
+        // Act
+        String hashB64 = bcryptSH.hashBase64(input);
+
+        // Assert
+        assertEquals(EXPECTED_HASH_BASE64, hashB64);
+    }
+
+    @Test
+    void testShouldHandleNullInput() {
+        // Arrange
+        List<String> inputs = Arrays.asList(null, "");
+
+        final String EXPECTED_HASH_HEX = "";
+        final String EXPECTED_HASH_BASE64 = "";
+
+        BcryptSecureHasher bcryptSH = new BcryptSecureHasher();
+
+        final List<String> hexResults = new ArrayList<>();
+        final List<String> b64Results = new ArrayList<>();
+
+        // Act
+        for (final String input : inputs) {
+            String hashHex = bcryptSH.hashHex(input);
+            hexResults.add(hashHex);
+
+            String hashB64 = bcryptSH.hashBase64(input);
+            b64Results.add(hashB64);
+        }
+
+        // Assert
+        hexResults.forEach(result -> assertEquals(EXPECTED_HASH_HEX, result));
+        b64Results.forEach(result -> assertEquals(EXPECTED_HASH_BASE64, result));
+    }
+
+    /**
+     * This test can have the minimum time threshold updated to determine if the performance
+     * is still sufficient compared to the existing threat model.
+     */
+    @EnabledIfSystemProperty(named = "nifi.test.performance", matches = "true")
+    @Test
+    void testDefaultCostParamsShouldBeSufficient() {
+        // Arrange
+        int testIterations = 100;
+        byte[] inputBytes = "This is a sensitive value".getBytes();
+
+        BcryptSecureHasher bcryptSH = new BcryptSecureHasher();
+
+        final List<String> results = new ArrayList<>();
+        final List<Long> resultDurations = new ArrayList<>();
+
+        // Act
+        for (int i = 0; i < testIterations; i++) {
+            long startNanos = System.nanoTime();
+            byte[] hash = bcryptSH.hashRaw(inputBytes);
+            long endNanos = System.nanoTime();
+            long durationNanos = endNanos - startNanos;
+
+            String hashHex = new String(Hex.encode(hash));
+
+            results.add(hashHex);
+            resultDurations.add(durationNanos);
+        }
+
+        // Assert
+        final long MIN_DURATION_NANOS = 75_000_000; // 75 ms
+        assertTrue(Collections.min(resultDurations) > MIN_DURATION_NANOS);
+        assertTrue(resultDurations.stream().mapToLong(Long::longValue).sum() / testIterations > MIN_DURATION_NANOS);
+    }

Review Comment:
   As some algorithms are no longer used for framework encryption operations, this test method can be removed.



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/BcryptCipherProviderTest.java:
##########
@@ -0,0 +1,486 @@
+/*
+ * 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 at.favre.lib.crypto.bcrypt.BCrypt;
+import at.favre.lib.crypto.bcrypt.Radix64Encoder;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.Security;
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class BcryptCipherProviderTest {
+    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess";
+
+    private static List<EncryptionMethod> strongKDFEncryptionMethods;
+
+    private static final int DEFAULT_KEY_LENGTH = 128;
+    public static final String MICROBENCHMARK = "microbenchmark";
+    private static List<Integer> AES_KEY_LENGTHS;
+
+    @BeforeAll
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider());

Review Comment:
   ```suggestion
   ```



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/CipherUtilityTest.java:
##########
@@ -0,0 +1,322 @@
+/*
+ * 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 org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.security.util.KeyDerivationFunction;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class CipherUtilityTest {
+    private static final Pattern KEY_LENGTH_PATTERN = Pattern.compile("([\\d]+)BIT");
+    // TripleDES must precede DES for automatic grouping precedence
+    private static final List<String> CIPHERS = Arrays.asList("AES", "TRIPLEDES", "DES", "RC2", "RC4", "RC5", "TWOFISH");
+    private static final List<String> SYMMETRIC_ALGORITHMS = Arrays.stream(EncryptionMethod.values())
+            .map(it -> it.getAlgorithm())
+            .filter(algorithm -> algorithm.startsWith("PBE") || algorithm.startsWith("AES"))
+            .collect(Collectors.toList());
+    private static final Map<String, List<String>> ALGORITHMS_MAPPED_BY_CIPHER = SYMMETRIC_ALGORITHMS
+            .stream()
+            .collect(Collectors.groupingBy(algorithm -> CIPHERS.stream().filter(cipher -> algorithm.contains(cipher)).findFirst().get()));
+
+    // Manually mapped as of 03/21/21 1.13.0
+    private static final Map<Integer, List<String>> ALGORITHMS_MAPPED_BY_KEY_LENGTH = new HashMap<>();
+    static {
+        ALGORITHMS_MAPPED_BY_KEY_LENGTH.put(40, Arrays.asList("PBEWITHSHAAND40BITRC2-CBC",
+                "PBEWITHSHAAND40BITRC4"));
+        ALGORITHMS_MAPPED_BY_KEY_LENGTH.put(64, Arrays.asList("PBEWITHMD5ANDDES",
+                "PBEWITHSHA1ANDDES"));
+        ALGORITHMS_MAPPED_BY_KEY_LENGTH.put(112, Arrays.asList("PBEWITHSHAAND2-KEYTRIPLEDES-CBC",
+                "PBEWITHSHAAND3-KEYTRIPLEDES-CBC"));
+        ALGORITHMS_MAPPED_BY_KEY_LENGTH.put(128, Arrays.asList("PBEWITHMD5AND128BITAES-CBC-OPENSSL",
+                "PBEWITHMD5ANDRC2",
+                "PBEWITHSHA1ANDRC2",
+                "PBEWITHSHA256AND128BITAES-CBC-BC",
+                "PBEWITHSHAAND128BITAES-CBC-BC",
+                "PBEWITHSHAAND128BITRC2-CBC",
+                "PBEWITHSHAAND128BITRC4",
+                "PBEWITHSHAANDTWOFISH-CBC",
+                "AES/CBC/NoPadding",
+                "AES/CBC/PKCS7Padding",
+                "AES/CTR/NoPadding",
+                "AES/GCM/NoPadding"));
+        ALGORITHMS_MAPPED_BY_KEY_LENGTH.put(192, Arrays.asList("PBEWITHMD5AND192BITAES-CBC-OPENSSL",
+                "PBEWITHSHA256AND192BITAES-CBC-BC",
+                "PBEWITHSHAAND192BITAES-CBC-BC",
+                "AES/CBC/NoPadding",
+                "AES/CBC/PKCS7Padding",
+                "AES/CTR/NoPadding",
+                "AES/GCM/NoPadding"));
+        ALGORITHMS_MAPPED_BY_KEY_LENGTH.put(256, Arrays.asList("PBEWITHMD5AND256BITAES-CBC-OPENSSL",
+                "PBEWITHSHA256AND256BITAES-CBC-BC",
+                "PBEWITHSHAAND256BITAES-CBC-BC",
+                "AES/CBC/NoPadding",
+                "AES/CBC/PKCS7Padding",
+                "AES/CTR/NoPadding",
+                "AES/GCM/NoPadding"));
+    }
+
+    @BeforeAll
+    static void setUpOnce() {
+        Security.addProvider(new BouncyCastleProvider());

Review Comment:
   In this case, the security provider registration is necessary, so this line can be retained, while others should be removed.



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/AESKeyedCipherProviderTest.java:
##########
@@ -14,311 +14,298 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.security.util.crypto
+package org.apache.nifi.security.util.crypto;
 
-import org.apache.commons.codec.binary.Hex
-import org.apache.nifi.security.util.EncryptionMethod
-import org.bouncycastle.jce.provider.BouncyCastleProvider
-import org.junit.jupiter.api.BeforeAll
-import org.junit.jupiter.api.Test
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
-import javax.crypto.Cipher
-import javax.crypto.SecretKey
-import javax.crypto.spec.SecretKeySpec
-import java.security.SecureRandom
-import java.security.Security
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
 
-import static org.junit.jupiter.api.Assertions.assertEquals
-import static org.junit.jupiter.api.Assertions.assertFalse
-import static org.junit.jupiter.api.Assertions.assertThrows
-import static org.junit.jupiter.api.Assertions.assertTrue
-import static org.junit.jupiter.api.Assumptions.assumeTrue
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
-class AESKeyedCipherProviderGroovyTest {
-    private static final Logger logger = LoggerFactory.getLogger(AESKeyedCipherProviderGroovyTest.class)
+public class AESKeyedCipherProviderTest {
+    private static final String KEY_HEX = "0123456789ABCDEFFEDCBA9876543210";
 
-    private static final String KEY_HEX = "0123456789ABCDEFFEDCBA9876543210"
+    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess";
 
-    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess"
+    private static final List<EncryptionMethod> keyedEncryptionMethods = Arrays.stream(EncryptionMethod.values())
+            .filter(EncryptionMethod::isKeyedCipher)
+            .collect(Collectors.toList());
 
-    private static final List<EncryptionMethod> keyedEncryptionMethods = EncryptionMethod.values().findAll { it.keyedCipher }
+    private static SecretKey key;
 
-    private static final SecretKey key = new SecretKeySpec(Hex.decodeHex(KEY_HEX as char[]), "AES")
 
     @BeforeAll
     static void setUpOnce() throws Exception {
-        Security.addProvider(new BouncyCastleProvider())
+        Security.addProvider(new BouncyCastleProvider());

Review Comment:
   It should be possible to remove this BouncyCastleProvider registration, can you evaluate removing this?



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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 org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.util.encoders.Hex;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.EnabledIfSystemProperty;
+
+import java.nio.charset.StandardCharsets;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class Argon2SecureHasherTest {
+    @BeforeAll
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider());
+    }

Review Comment:
   ```suggestion
   ```



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/Argon2CipherProviderTest.java:
##########
@@ -0,0 +1,418 @@
+/*
+ * 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 org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class Argon2CipherProviderTest {
+    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess";
+
+    private static List<EncryptionMethod> strongKDFEncryptionMethods;
+
+    private static final int DEFAULT_KEY_LENGTH = 128;
+    private final String SALT_HEX = "0123456789ABCDEFFEDCBA9876543210";
+    private static List<Integer> VALID_KEY_LENGTHS;
+    private RandomIVPBECipherProvider cipherProvider;
+    private final List<Integer> FULL_SALT_LENGTH_RANGE = Arrays.asList(49, 50, 51, 52, 53);
+
+    @BeforeAll
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider());

Review Comment:
   As mentioned elsewhere, it should be possible to remove this registration.



##########
nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/Argon2CipherProviderTest.java:
##########
@@ -0,0 +1,418 @@
+/*
+ * 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 org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class Argon2CipherProviderTest {
+    private static final String PLAINTEXT = "ExactBlockSizeRequiredForProcess";
+
+    private static List<EncryptionMethod> strongKDFEncryptionMethods;
+
+    private static final int DEFAULT_KEY_LENGTH = 128;
+    private final String SALT_HEX = "0123456789ABCDEFFEDCBA9876543210";
+    private static List<Integer> VALID_KEY_LENGTHS;
+    private RandomIVPBECipherProvider cipherProvider;
+    private final List<Integer> FULL_SALT_LENGTH_RANGE = Arrays.asList(49, 50, 51, 52, 53);
+
+    @BeforeAll
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider());
+
+        strongKDFEncryptionMethods = Arrays.stream(EncryptionMethod.values())
+                .filter(EncryptionMethod::isCompatibleWithStrongKDFs)
+                .collect(Collectors.toList());
+
+        VALID_KEY_LENGTHS = Arrays.asList(128, 192, 256);
+    }
+
+    @BeforeEach
+    void setUp() {
+        // Very fast parameters to test for correctness rather than production values
+        cipherProvider = new Argon2CipherProvider(1024, 1, 3);
+    }
+
+    @Test
+    void testGetCipherShouldBeInternallyConsistent() throws Exception {
+        // Arrange
+        final String PASSWORD = "shortPassword";
+        final byte[] SALT = cipherProvider.generateSalt();
+
+        // Act
+        for (EncryptionMethod em : strongKDFEncryptionMethods) {
+            // Initialize a cipher for encryption
+            Cipher cipher = cipherProvider.getCipher(em, PASSWORD, SALT, DEFAULT_KEY_LENGTH, true);
+            byte[] iv = cipher.getIV();
+
+            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"));
+
+            cipher = cipherProvider.getCipher(em, PASSWORD, SALT, iv, DEFAULT_KEY_LENGTH, false);
+            byte[] recoveredBytes = cipher.doFinal(cipherBytes);
+            String recovered = new String(recoveredBytes, "UTF-8");
+
+            // Assert
+            assertEquals(PLAINTEXT, recovered);
+        }
+    }
+
+    @Test
+    void testArgon2ShouldSupportExternalCompatibility() throws Exception {
+        // Arrange
+
+        // Default values are hashLength = 32, memory = 1024, parallelism = 1, iterations = 3, but the provided salt will contain the parameters used
+        cipherProvider = new Argon2CipherProvider();
+
+        final String PLAINTEXT = "This is a plaintext message.";
+        final String PASSWORD = "thisIsABadPassword";
+        final int hashLength = 256;
+
+        // These values can be generated by running `$ ./openssl_argon2.rb` in the terminal
+        final byte[] SALT = Hex.decodeHex("68d29a1d8021f45954333767358a2492".toCharArray());
+        final byte[] IV = Hex.decodeHex("808590f35f9fba14dbda9c2bb2b76a79".toCharArray());
+
+        final String CIPHER_TEXT = "d672412857916880c79d573aa4f9d4971b85f07438d6f62f38a0e31314caa2e5";
+        byte[] cipherBytes = Hex.decodeHex(CIPHER_TEXT.toCharArray());
+        EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
+
+        // Sanity check
+        String rubyKeyHex = "8caf581795886d38f0c605e3d674f4961c658ee3625a8e8868be36c902d234ef";
+        Cipher rubyCipher = Cipher.getInstance(encryptionMethod.getAlgorithm(), "BC");
+        SecretKeySpec rubyKey = new SecretKeySpec(Hex.decodeHex(rubyKeyHex.toCharArray()), "AES");
+        IvParameterSpec ivSpec = new IvParameterSpec(IV);
+        rubyCipher.init(Cipher.ENCRYPT_MODE, rubyKey, ivSpec);
+        byte[] rubyCipherBytes = rubyCipher.doFinal(PLAINTEXT.getBytes());
+        rubyCipher.init(Cipher.DECRYPT_MODE, rubyKey, ivSpec);
+        assertArrayEquals(PLAINTEXT.getBytes(), rubyCipher.doFinal(rubyCipherBytes));
+        assertArrayEquals(PLAINTEXT.getBytes(), rubyCipher.doFinal(cipherBytes));
+
+        // $argon2id$v=19$m=memory,t=iterations,p=parallelism$saltB64$hashB64
+        final String FULL_HASH = "$argon2id$v=19$m=256,t=3,p=1$aNKaHYAh9FlUMzdnNYokkg$jK9YF5WIbTjwxgXj1nT0lhxljuNiWo6IaL42yQLSNO8";
+
+        final String FULL_SALT = FULL_HASH.substring(0, FULL_HASH.lastIndexOf("$"));
+
+        final String[] hashComponents = FULL_HASH.split("\\$");
+        final String saltB64 = hashComponents[4];
+        byte[] salt = Base64.decodeBase64(saltB64);
+        assertArrayEquals(SALT, salt);
+
+        // Act
+        Cipher cipher = cipherProvider.getCipher(encryptionMethod, PASSWORD, FULL_SALT.getBytes(), IV, hashLength, false);
+        byte[] recoveredBytes = cipher.doFinal(cipherBytes);
+        String recovered = new String(recoveredBytes, "UTF-8");
+
+        // Assert
+        assertEquals(PLAINTEXT, recovered);
+    }
+
+    @Test
+    void testGetCipherShouldRejectInvalidIV() throws Exception {
+        // Arrange
+        final String PASSWORD = "shortPassword";
+        final byte[] SALT = Hex.decodeHex(SALT_HEX.toCharArray());
+        final int MAX_LENGTH = 15;
+        final List<byte[]> INVALID_IVS = new ArrayList<>();
+        for (int length = 0; length <= MAX_LENGTH; length++) {
+            INVALID_IVS.add(new byte[length]);
+        }
+
+        EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
+
+        // Act
+        for (final byte[] badIV: INVALID_IVS) {
+            // Encrypt should print a warning about the bad IV but overwrite it
+            Cipher cipher = cipherProvider.getCipher(encryptionMethod, PASSWORD, SALT, badIV, DEFAULT_KEY_LENGTH, true);
+
+            // Decrypt should fail
+            IllegalArgumentException iae = assertThrows(IllegalArgumentException.class,
+                    () -> cipherProvider.getCipher(encryptionMethod, PASSWORD, SALT, badIV, DEFAULT_KEY_LENGTH, false));
+
+            // Assert
+            assertTrue(iae.getMessage().contains("Cannot decrypt without a valid IV"));
+        }
+    }
+
+    @Test
+    void testGetCipherWithExternalIVShouldBeInternallyConsistent() throws Exception {
+        // Arrange
+        final String PASSWORD = "shortPassword";
+        final byte[] SALT = cipherProvider.generateSalt();
+        final byte[] IV = Hex.decodeHex("01".repeat(16).toCharArray());
+
+        // Act
+        for (EncryptionMethod em : strongKDFEncryptionMethods) {
+            // Initialize a cipher for encryption
+            Cipher cipher = cipherProvider.getCipher(em, PASSWORD, SALT, IV, DEFAULT_KEY_LENGTH, true);
+
+            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"));
+
+            cipher = cipherProvider.getCipher(em, PASSWORD, SALT, IV, DEFAULT_KEY_LENGTH, false);
+            byte[] recoveredBytes = cipher.doFinal(cipherBytes);
+            String recovered = new String(recoveredBytes, "UTF-8");
+
+            // Assert
+            assertEquals(PLAINTEXT, recovered);
+        }
+    }
+
+    @Test
+    void testGetCipherWithUnlimitedStrengthShouldBeInternallyConsistent() throws Exception {
+        // Arrange
+        final String PASSWORD = "shortPassword";
+        final byte[] SALT = cipherProvider.generateSalt();
+
+        final int LONG_KEY_LENGTH = 256;
+
+        // Act
+        for (EncryptionMethod em : strongKDFEncryptionMethods) {
+            // Initialize a cipher for encryption
+            Cipher cipher = cipherProvider.getCipher(em, PASSWORD, SALT, LONG_KEY_LENGTH, true);
+            byte[] iv = cipher.getIV();
+
+            byte[] cipherBytes = cipher.doFinal(PLAINTEXT.getBytes("UTF-8"));
+
+            cipher = cipherProvider.getCipher(em, PASSWORD, SALT, iv, LONG_KEY_LENGTH, false);
+            byte[] recoveredBytes = cipher.doFinal(cipherBytes);
+            String recovered = new String(recoveredBytes, "UTF-8");
+
+            // Assert
+            assertEquals(PLAINTEXT, recovered);
+        }
+    }
+
+    @Test
+    void testGetCipherShouldNotAcceptInvalidSalts() throws Exception {
+        // Arrange
+        final String PASSWORD = "thisIsABadPassword";
+
+        final List<String> INVALID_SALTS = Arrays.asList("argon2", "$3a$11$", "x", "$2a$10$");
+        final String LENGTH_MESSAGE = "The raw salt must be greater than or equal to 8 bytes";
+
+        EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
+
+        // Act
+        for (final String salt : INVALID_SALTS) {
+            IllegalArgumentException iae = assertThrows(IllegalArgumentException.class,
+                    () -> cipherProvider.getCipher(encryptionMethod, PASSWORD, salt.getBytes(), DEFAULT_KEY_LENGTH, true));
+
+            // Assert
+            assertTrue(iae.getMessage().contains(LENGTH_MESSAGE));
+        }
+    }
+
+    @Test
+    void testGetCipherShouldHandleUnformattedSalts() throws Exception {
+        // Arrange
+        final String PASSWORD = "thisIsABadPassword";

Review Comment:
   It would be helpful to refactor this to a shared static value instead of repeating it in all methods.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org