You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2017/08/07 19:23:32 UTC

[1/2] nifi git commit: NIFI-4139 - Moved key provider interface and implementations from nifi-data-provenance-utils module to nifi-security-utils module. - Refactored duplicate byte[] concatenation methods from utility classes and removed deprecation war

Repository: nifi
Updated Branches:
  refs/heads/master 8b54c2604 -> 675d98900


http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/CryptoUtilsTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/CryptoUtilsTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/CryptoUtilsTest.groovy
new file mode 100644
index 0000000..954b7ca
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/CryptoUtilsTest.groovy
@@ -0,0 +1,506 @@
+/*
+ * 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.kms
+
+import org.apache.commons.lang3.SystemUtils
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.bouncycastle.util.encoders.Hex
+import org.junit.After
+import org.junit.AfterClass
+import org.junit.Assume
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.ClassRule
+import org.junit.Test
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.crypto.Cipher
+import javax.crypto.SecretKey
+import javax.crypto.spec.IvParameterSpec
+import javax.crypto.spec.SecretKeySpec
+import java.nio.charset.StandardCharsets
+import java.nio.file.Files
+import java.nio.file.attribute.PosixFilePermission
+import java.security.KeyManagementException
+import java.security.SecureRandom
+import java.security.Security
+
+import static groovy.test.GroovyAssert.shouldFail
+
+@RunWith(JUnit4.class)
+class CryptoUtilsTest {
+    private static final Logger logger = LoggerFactory.getLogger(CryptoUtilsTest.class)
+
+    private static final String KEY_ID = "K1"
+    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
+    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
+    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
+
+    private static
+    final Set<PosixFilePermission> ALL_POSIX_ATTRS = PosixFilePermission.values() as Set<PosixFilePermission>
+
+    @ClassRule
+    public static TemporaryFolder tempFolder = new TemporaryFolder()
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() throws Exception {
+        tempFolder.create()
+    }
+
+    @After
+    void tearDown() throws Exception {
+        tempFolder?.delete()
+    }
+
+    @AfterClass
+    static void tearDownOnce() throws Exception {
+
+    }
+
+    private static boolean isUnlimitedStrengthCryptoAvailable() {
+        Cipher.getMaxAllowedKeyLength("AES") > 128
+    }
+
+    private static boolean isRootUser() {
+        ProcessBuilder pb = new ProcessBuilder(["id", "-u"])
+        Process process = pb.start()
+        InputStream responseStream = process.getInputStream()
+        BufferedReader responseReader = new BufferedReader(new InputStreamReader(responseStream))
+        responseReader.text.trim() == "0"
+    }
+
+    @Test
+    void testShouldConcatenateByteArrays() {
+        // Arrange
+        byte[] bytes1 = "These are some bytes".getBytes(StandardCharsets.UTF_8)
+        byte[] bytes2 = "These are some other bytes".getBytes(StandardCharsets.UTF_8)
+        final byte[] EXPECTED_CONCATENATED_BYTES = ((bytes1 as List) << (bytes2 as List)).flatten() as byte[]
+        logger.info("Expected concatenated bytes: ${Hex.toHexString(EXPECTED_CONCATENATED_BYTES)}")
+
+        // Act
+        byte[] concat = CryptoUtils.concatByteArrays(bytes1, bytes2)
+        logger.info("  Actual concatenated bytes: ${Hex.toHexString(concat)}")
+
+        // Assert
+        assert concat == EXPECTED_CONCATENATED_BYTES
+    }
+
+    @Test
+    void testShouldValidateStaticKeyProvider() {
+        // Arrange
+        String staticProvider = StaticKeyProvider.class.name
+        String providerLocation = null
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX])
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert keyProviderIsValid
+    }
+
+    @Test
+    void testShouldValidateLegacyStaticKeyProvider() {
+        // Arrange
+        String staticProvider = StaticKeyProvider.class.name.replaceFirst("security.kms", "provenance")
+        String providerLocation = null
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX])
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert keyProviderIsValid
+    }
+
+    @Test
+    void testShouldNotValidateStaticKeyProviderMissingKeyId() {
+        // Arrange
+        String staticProvider = StaticKeyProvider.class.name
+        String providerLocation = null
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, null, [(KEY_ID): KEY_HEX])
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${null} / ${KEY_HEX} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !keyProviderIsValid
+    }
+
+    @Test
+    void testShouldNotValidateStaticKeyProviderMissingKey() {
+        // Arrange
+        String staticProvider = StaticKeyProvider.class.name
+        String providerLocation = null
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, null)
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !keyProviderIsValid
+    }
+
+    @Test
+    void testShouldNotValidateStaticKeyProviderWithInvalidKey() {
+        // Arrange
+        String staticProvider = StaticKeyProvider.class.name
+        String providerLocation = null
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX[0..<-2]])
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX[0..<-2]} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !keyProviderIsValid
+    }
+
+    @Test
+    void testShouldValidateFileBasedKeyProvider() {
+        // Arrange
+        String fileBasedProvider = FileBasedKeyProvider.class.name
+        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
+        String providerLocation = fileBasedProviderFile.path
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, KEY_ID, null)
+        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert keyProviderIsValid
+    }
+
+    @Test
+    void testShouldValidateLegacyFileBasedKeyProvider() {
+        // Arrange
+        String fileBasedProvider = FileBasedKeyProvider.class.name.replaceFirst("security.kms", "provenance")
+        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
+        String providerLocation = fileBasedProviderFile.path
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, KEY_ID, null)
+        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert keyProviderIsValid
+    }
+
+    @Test
+    void testShouldNotValidateMissingFileBasedKeyProvider() {
+        // Arrange
+        String fileBasedProvider = FileBasedKeyProvider.class.name
+        File fileBasedProviderFile = new File(tempFolder.root, "filebased_missing.kp")
+        String providerLocation = fileBasedProviderFile.path
+        logger.info("Created (no actual file) temporary file based key provider: ${providerLocation}")
+
+        // Act
+        String missingLocation = providerLocation
+        boolean missingKeyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, missingLocation, KEY_ID, null)
+        logger.info("Key Provider ${fileBasedProvider} with location ${missingLocation} and keyId ${KEY_ID} / ${null} is ${missingKeyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !missingKeyProviderIsValid
+    }
+
+    @Test
+    void testShouldNotValidateUnreadableFileBasedKeyProvider() {
+        // Arrange
+        Assume.assumeFalse("This test does not run on Windows", SystemUtils.IS_OS_WINDOWS)
+        Assume.assumeFalse("This test does not run for root users", isRootUser())
+
+        String fileBasedProvider = FileBasedKeyProvider.class.name
+        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
+        String providerLocation = fileBasedProviderFile.path
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Make it unreadable
+        markFileUnreadable(fileBasedProviderFile)
+
+        // Act
+        boolean unreadableKeyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, KEY_ID, null)
+        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${unreadableKeyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !unreadableKeyProviderIsValid
+
+        // Make the file deletable so cleanup can occur
+        markFileReadable(fileBasedProviderFile)
+    }
+
+    private static void markFileReadable(File fileBasedProviderFile) {
+        if (SystemUtils.IS_OS_WINDOWS) {
+            fileBasedProviderFile.setReadable(true, false)
+        } else {
+            Files.setPosixFilePermissions(fileBasedProviderFile.toPath(), ALL_POSIX_ATTRS)
+        }
+    }
+
+    private static void markFileUnreadable(File fileBasedProviderFile) {
+        if (SystemUtils.IS_OS_WINDOWS) {
+            fileBasedProviderFile.setReadable(false, false)
+        } else {
+            Files.setPosixFilePermissions(fileBasedProviderFile.toPath(), [] as Set<PosixFilePermission>)
+        }
+    }
+
+    @Test
+    void testShouldNotValidateFileBasedKeyProviderMissingKeyId() {
+        // Arrange
+        String fileBasedProvider = FileBasedKeyProvider.class.name
+        File fileBasedProviderFile = tempFolder.newFile("missing_key_id.kp")
+        String providerLocation = fileBasedProviderFile.path
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, null, null)
+        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${null} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !keyProviderIsValid
+    }
+
+    @Test
+    void testShouldNotValidateUnknownKeyProvider() {
+        // Arrange
+        String providerImplementation = "org.apache.nifi.provenance.ImaginaryKeyProvider"
+        String providerLocation = null
+
+        // Act
+        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(providerImplementation, providerLocation, KEY_ID, null)
+        logger.info("Key Provider ${providerImplementation} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
+
+        // Assert
+        assert !keyProviderIsValid
+    }
+
+    @Test
+    void testShouldValidateKey() {
+        // Arrange
+        String validKey = KEY_HEX
+        String validLowercaseKey = KEY_HEX.toLowerCase()
+
+        String tooShortKey = KEY_HEX[0..<-2]
+        String tooLongKey = KEY_HEX + KEY_HEX // Guaranteed to be 2x the max valid key length
+        String nonHexKey = KEY_HEX.replaceFirst(/A/, "X")
+
+        def validKeys = [validKey, validLowercaseKey]
+        def invalidKeys = [tooShortKey, tooLongKey, nonHexKey]
+
+        // If unlimited strength is available, also validate 128 and 196 bit keys
+        if (isUnlimitedStrengthCryptoAvailable()) {
+            validKeys << KEY_HEX_128
+            validKeys << KEY_HEX_256[0..<48]
+        } else {
+            invalidKeys << KEY_HEX_256[0..<48]
+            invalidKeys << KEY_HEX_256
+        }
+
+        // Act
+        def validResults = validKeys.collect { String key ->
+            logger.info("Validating ${key}")
+            CryptoUtils.keyIsValid(key)
+        }
+
+        def invalidResults = invalidKeys.collect { String key ->
+            logger.info("Validating ${key}")
+            CryptoUtils.keyIsValid(key)
+        }
+
+        // Assert
+        assert validResults.every()
+        assert invalidResults.every { !it }
+    }
+
+    @Test
+    void testShouldReadKeys() {
+        // Arrange
+        String masterKeyHex = KEY_HEX
+        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
+
+        // Generate the file
+        String keyFileName = "keys.nkp"
+        File keyFile = tempFolder.newFile(keyFileName)
+        final int KEY_COUNT = 5
+        List<String> lines = []
+        KEY_COUNT.times { int i ->
+            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
+        }
+
+        keyFile.text = lines.join("\n")
+
+        logger.info("File contents: \n${keyFile.text}")
+
+        // Act
+        def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
+        logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
+
+        // Assert
+        assert readKeys.size() == KEY_COUNT
+    }
+
+    @Test
+    void testShouldReadKeysWithDuplicates() {
+        // Arrange
+        String masterKeyHex = KEY_HEX
+        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
+
+        // Generate the file
+        String keyFileName = "keys.nkp"
+        File keyFile = tempFolder.newFile(keyFileName)
+        final int KEY_COUNT = 3
+        List<String> lines = []
+        KEY_COUNT.times { int i ->
+            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
+        }
+
+        lines.add("key3=${generateEncryptedKey(masterKey)}")
+
+        keyFile.text = lines.join("\n")
+
+        logger.info("File contents: \n${keyFile.text}")
+
+        // Act
+        def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
+        logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
+
+        // Assert
+        assert readKeys.size() == KEY_COUNT
+    }
+
+    @Test
+    void testShouldReadKeysWithSomeMalformed() {
+        // Arrange
+        String masterKeyHex = KEY_HEX
+        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
+
+        // Generate the file
+        String keyFileName = "keys.nkp"
+        File keyFile = tempFolder.newFile(keyFileName)
+        final int KEY_COUNT = 5
+        List<String> lines = []
+        KEY_COUNT.times { int i ->
+            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
+        }
+
+        // Insert the malformed keys in the middle
+        lines.add(2, "keyX1==${generateEncryptedKey(masterKey)}")
+        lines.add(4, "=${generateEncryptedKey(masterKey)}")
+        lines.add(6, "keyX3=non Base64-encoded data")
+
+        keyFile.text = lines.join("\n")
+
+        logger.info("File contents: \n${keyFile.text}")
+
+        // Act
+        def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
+        logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
+
+        // Assert
+        assert readKeys.size() == KEY_COUNT
+    }
+
+    @Test
+    void testShouldNotReadKeysIfAllMalformed() {
+        // Arrange
+        String masterKeyHex = KEY_HEX
+        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
+
+        // Generate the file
+        String keyFileName = "keys.nkp"
+        File keyFile = tempFolder.newFile(keyFileName)
+        final int KEY_COUNT = 5
+        List<String> lines = []
+
+        // All of these keys are malformed
+        KEY_COUNT.times { int i ->
+            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)[0..<-4]}")
+        }
+
+        keyFile.text = lines.join("\n")
+
+        logger.info("File contents: \n${keyFile.text}")
+
+        // Act
+        def msg = shouldFail(KeyManagementException) {
+            def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
+            logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
+        }
+
+        // Assert
+        assert msg.getMessage() == "The provided file contained no valid keys"
+    }
+
+    @Test
+    void testShouldNotReadKeysIfEmptyOrMissing() {
+        // Arrange
+        String masterKeyHex = KEY_HEX
+        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
+
+        // Generate the file
+        String keyFileName = "empty.nkp"
+        File keyFile = tempFolder.newFile(keyFileName)
+        logger.info("File contents: \n${keyFile.text}")
+
+        // Act
+        def missingMsg = shouldFail(KeyManagementException) {
+            def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
+            logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
+        }
+        logger.expected("Missing file: ${missingMsg}")
+
+        def emptyMsg = shouldFail(KeyManagementException) {
+            def readKeys = CryptoUtils.readKeys(null, masterKey)
+            logger.info("Read ${readKeys.size()} keys from ${null}")
+        }
+        logger.expected("Empty file: ${emptyMsg}")
+
+        // Assert
+        assert missingMsg.getMessage() == "The provided file contained no valid keys"
+        assert emptyMsg.getMessage() == "The key provider file is not present and readable"
+    }
+
+    private static String generateEncryptedKey(SecretKey masterKey) {
+        byte[] ivBytes = new byte[16]
+        byte[] keyBytes = new byte[isUnlimitedStrengthCryptoAvailable() ? 32 : 16]
+
+        SecureRandom sr = new SecureRandom()
+        sr.nextBytes(ivBytes)
+        sr.nextBytes(keyBytes)
+
+        Cipher masterCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
+        masterCipher.init(Cipher.ENCRYPT_MODE, masterKey, new IvParameterSpec(ivBytes))
+        byte[] cipherBytes = masterCipher.doFinal(keyBytes)
+
+        Base64.encoder.encodeToString(CryptoUtils.concatByteArrays(ivBytes, cipherBytes))
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/KeyProviderFactoryTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/KeyProviderFactoryTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/KeyProviderFactoryTest.groovy
new file mode 100644
index 0000000..4abc390
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/kms/KeyProviderFactoryTest.groovy
@@ -0,0 +1,229 @@
+/*
+ * 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.kms
+
+import org.apache.nifi.util.NiFiProperties
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.bouncycastle.util.encoders.Hex
+import org.junit.After
+import org.junit.AfterClass
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.ClassRule
+import org.junit.Test
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.crypto.Cipher
+import javax.crypto.SecretKey
+import javax.crypto.spec.IvParameterSpec
+import javax.crypto.spec.SecretKeySpec
+import java.security.KeyManagementException
+import java.security.SecureRandom
+import java.security.Security
+
+import static groovy.test.GroovyAssert.shouldFail
+
+@RunWith(JUnit4.class)
+class KeyProviderFactoryTest {
+    private static final Logger logger = LoggerFactory.getLogger(KeyProviderFactoryTest.class)
+
+    private static final String KEY_ID = "K1"
+    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
+    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
+    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
+
+    private static final String LEGACY_SKP_FQCN = "org.apache.nifi.provenance.StaticKeyProvider"
+    private static final String LEGACY_FBKP_FQCN = "org.apache.nifi.provenance.FileBasedKeyProvider"
+
+    private static final String ORIGINAL_PROPERTIES_PATH = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)
+
+    private static final SecretKey MASTER_KEY = new SecretKeySpec(Hex.decode(KEY_HEX), "AES")
+
+    @ClassRule
+    public static TemporaryFolder tempFolder = new TemporaryFolder()
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+
+        logger.info("Original \$PROPERTIES_FILE_PATH is ${ORIGINAL_PROPERTIES_PATH}")
+        String testPath = new File("src/test/resources/${isUnlimitedStrengthCryptoAvailable() ? "256" : "128"}/conf/.").getAbsolutePath()
+        logger.info("Temporarily setting to ${testPath}")
+        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, testPath)
+    }
+
+    @Before
+    void setUp() throws Exception {
+        tempFolder.create()
+    }
+
+    @After
+    void tearDown() throws Exception {
+        tempFolder?.delete()
+    }
+
+    @AfterClass
+    static void tearDownOnce() throws Exception {
+        if (ORIGINAL_PROPERTIES_PATH) {
+            logger.info("Restored \$PROPERTIES_FILE_PATH to ${ORIGINAL_PROPERTIES_PATH}")
+            System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, ORIGINAL_PROPERTIES_PATH)
+        }
+    }
+
+    private static boolean isUnlimitedStrengthCryptoAvailable() {
+        Cipher.getMaxAllowedKeyLength("AES") > 128
+    }
+
+    private static void populateKeyDefinitionsFile(String path = "src/test/resources/conf/filebased.kp") {
+        String masterKeyHex = KEY_HEX
+        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
+
+        // Generate the file
+        File keyFile = new File(path)
+        final int KEY_COUNT = 1
+        List<String> lines = []
+        KEY_COUNT.times { int i ->
+            lines.add("K${i + 1}=${generateEncryptedKey(masterKey)}")
+        }
+
+        keyFile.text = lines.join("\n")
+    }
+
+    private static String generateEncryptedKey(SecretKey masterKey) {
+        byte[] ivBytes = new byte[16]
+        byte[] keyBytes = new byte[isUnlimitedStrengthCryptoAvailable() ? 32 : 16]
+
+        SecureRandom sr = new SecureRandom()
+        sr.nextBytes(ivBytes)
+        sr.nextBytes(keyBytes)
+
+        Cipher masterCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
+        masterCipher.init(Cipher.ENCRYPT_MODE, masterKey, new IvParameterSpec(ivBytes))
+        byte[] cipherBytes = masterCipher.doFinal(keyBytes)
+
+        Base64.encoder.encodeToString(CryptoUtils.concatByteArrays(ivBytes, cipherBytes))
+    }
+
+    @Test
+    void testShouldBuildStaticKeyProvider() {
+        // Arrange
+        String staticProvider = StaticKeyProvider.class.name
+        String providerLocation = null
+
+        // Act
+        KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], null)
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
+
+        // Assert
+        assert keyProvider instanceof StaticKeyProvider
+        assert keyProvider.getAvailableKeyIds() == [KEY_ID]
+    }
+
+    @Test
+    void testShouldBuildStaticKeyProviderWithLegacyPackage() {
+        // Arrange
+        String staticProvider = LEGACY_SKP_FQCN
+        String providerLocation = null
+
+        // Act
+        KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], null)
+        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
+
+        // Assert
+        assert keyProvider instanceof StaticKeyProvider
+        assert keyProvider.getAvailableKeyIds() == [KEY_ID]
+    }
+
+    @Test
+    void testShouldBuildFileBasedKeyProvider() {
+        // Arrange
+        String fileBasedProvider = FileBasedKeyProvider.class.name
+        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
+        String providerLocation = fileBasedProviderFile.path
+        populateKeyDefinitionsFile(providerLocation)
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Act
+        KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], MASTER_KEY)
+        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
+
+        // Assert
+        assert keyProvider instanceof FileBasedKeyProvider
+        assert keyProvider.getAvailableKeyIds() == [KEY_ID]
+    }
+
+    @Test
+    void testShouldBuildFileBasedKeyProviderWithLegacyPackage() {
+        // Arrange
+        String fileBasedProvider = LEGACY_FBKP_FQCN
+        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
+        String providerLocation = fileBasedProviderFile.path
+        populateKeyDefinitionsFile(providerLocation)
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Act
+        KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], MASTER_KEY)
+        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
+
+        // Assert
+        assert keyProvider instanceof FileBasedKeyProvider
+        assert keyProvider.getAvailableKeyIds() == [KEY_ID]
+    }
+
+    @Test
+    void testShouldNotBuildFileBasedKeyProviderWithoutMasterKey() {
+        // Arrange
+        String fileBasedProvider = FileBasedKeyProvider.class.name
+        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
+        String providerLocation = fileBasedProviderFile.path
+        populateKeyDefinitionsFile(providerLocation)
+        logger.info("Created temporary file based key provider: ${providerLocation}")
+
+        // Act
+        def msg = shouldFail(KeyManagementException) {
+            KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], null)
+            logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
+        }
+
+        // Assert
+        assert msg =~ "The master key must be provided to decrypt the individual keys"
+    }
+
+    @Test
+    void testShouldNotBuildUnknownKeyProvider() {
+        // Arrange
+        String providerImplementation = "org.apache.nifi.provenance.ImaginaryKeyProvider"
+        String providerLocation = null
+
+        // Act
+        def msg = shouldFail(KeyManagementException) {
+            KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(providerImplementation, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], null)
+            logger.info("Key Provider ${providerImplementation} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
+        }
+
+        // Assert
+        assert msg =~ "Invalid key provider implementation provided"
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/test/resources/128/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/test/resources/128/conf/bootstrap.conf b/nifi-commons/nifi-security-utils/src/test/resources/128/conf/bootstrap.conf
new file mode 100644
index 0000000..4be6e3e
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/resources/128/conf/bootstrap.conf
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+# Master key in hexadecimal format for encrypted sensitive configuration values
+nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA9876543210
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/test/resources/256/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/test/resources/256/conf/bootstrap.conf b/nifi-commons/nifi-security-utils/src/test/resources/256/conf/bootstrap.conf
new file mode 100644
index 0000000..e7e5914
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/resources/256/conf/bootstrap.conf
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+# Master key in hexadecimal format for encrypted sensitive configuration values
+nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index c2e9f8a..81350a6 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -2819,7 +2819,7 @@ The simplest configuration is below:
 ....
 nifi.provenance.repository.implementation=org.apache.nifi.provenance.EncryptedWriteAheadProvenanceRepository
 nifi.provenance.repository.debug.frequency=100
-nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.provenance.StaticKeyProvider
+nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.security.kms.StaticKeyProvider
 nifi.provenance.repository.encryption.key.provider.location=
 nifi.provenance.repository.encryption.key.id=Key1
 nifi.provenance.repository.encryption.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-docs/src/main/asciidoc/user-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc
index 1adbf3f..0cf8bb0 100644
--- a/nifi-docs/src/main/asciidoc/user-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/user-guide.adoc
@@ -1920,7 +1920,7 @@ The `StaticKeyProvider` implementation defines keys directly in `nifi.properties
 
 The following configuration section would result in a key provider with two available keys, "Key1" (active) and "AnotherKey".
 ....
-nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.provenance.StaticKeyProvider
+nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.security.kms.StaticKeyProvider
 nifi.provenance.repository.encryption.key.id=Key1
 nifi.provenance.repository.encryption.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
 nifi.provenance.repository.encryption.key.id.AnotherKey=0101010101010101010101010101010101010101010101010101010101010101

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepository.java
index a2d455b..00cc599 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepository.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepository.java
@@ -18,11 +18,13 @@ package org.apache.nifi.provenance;
 
 import java.io.IOException;
 import java.security.KeyManagementException;
-import java.util.Map;
-import java.util.stream.Collectors;
 import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
 import org.apache.nifi.provenance.serialization.RecordReaders;
 import org.apache.nifi.provenance.store.EventFileManager;
 import org.apache.nifi.provenance.store.RecordReaderFactory;
@@ -30,6 +32,8 @@ import org.apache.nifi.provenance.store.RecordWriterFactory;
 import org.apache.nifi.provenance.toc.StandardTocWriter;
 import org.apache.nifi.provenance.toc.TocUtil;
 import org.apache.nifi.provenance.toc.TocWriter;
+import org.apache.nifi.security.kms.KeyProvider;
+import org.apache.nifi.security.kms.KeyProviderFactory;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,7 +74,13 @@ public class EncryptedWriteAheadProvenanceRepository extends WriteAheadProvenanc
         ProvenanceEventEncryptor provenanceEventEncryptor;
         if (getConfig().supportsEncryption()) {
             try {
-                KeyProvider keyProvider = buildKeyProvider();
+                KeyProvider keyProvider;
+                if (KeyProviderFactory.requiresMasterKey(getConfig().getKeyProviderImplementation())) {
+                    SecretKey masterKey = getMasterKey();
+                    keyProvider = buildKeyProvider(masterKey);
+                } else {
+                    keyProvider = buildKeyProvider();
+                }
                 provenanceEventEncryptor = new AESProvenanceEventEncryptor();
                 provenanceEventEncryptor.initialize(keyProvider);
             } catch (KeyManagementException e) {
@@ -111,6 +121,10 @@ public class EncryptedWriteAheadProvenanceRepository extends WriteAheadProvenanc
     }
 
     private KeyProvider buildKeyProvider() throws KeyManagementException {
+        return buildKeyProvider(null);
+    }
+
+    private KeyProvider buildKeyProvider(SecretKey masterKey) throws KeyManagementException {
         RepositoryConfiguration config = super.getConfig();
         if (config == null) {
             throw new KeyManagementException("The repository configuration is missing");
@@ -122,38 +136,17 @@ public class EncryptedWriteAheadProvenanceRepository extends WriteAheadProvenanc
                     + NiFiProperties.PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_IMPLEMENTATION_CLASS);
         }
 
-        // TODO: Extract to factory
-        KeyProvider keyProvider;
-        if (StaticKeyProvider.class.getName().equals(implementationClassName)) {
-            // Get all the keys (map) from config
-            if (CryptoUtils.isValidKeyProvider(implementationClassName, config.getKeyProviderLocation(), config.getKeyId(), config.getEncryptionKeys())) {
-                Map<String, SecretKey> formedKeys = config.getEncryptionKeys().entrySet().stream()
-                        .collect(Collectors.toMap(
-                                Map.Entry::getKey,
-                                e -> {
-                                    try {
-                                        return CryptoUtils.formKeyFromHex(e.getValue());
-                                    } catch (KeyManagementException e1) {
-                                        // This should never happen because the hex has already been validated
-                                        logger.error("Encountered an error: ", e1);
-                                        return null;
-                                    }
-                                }));
-                keyProvider = new StaticKeyProvider(formedKeys);
-            } else {
-                final String msg = "The StaticKeyProvider definition is not valid";
-                logger.error(msg);
-                throw new KeyManagementException(msg);
-            }
-        } else if (FileBasedKeyProvider.class.getName().equals(implementationClassName)) {
-            keyProvider = new FileBasedKeyProvider(config.getKeyProviderLocation());
-            if (!keyProvider.keyExists(config.getKeyId())) {
-                throw new KeyManagementException("The specified key ID " + config.getKeyId() + " is not in the key definition file");
-            }
-        } else {
-            throw new KeyManagementException("Invalid key provider implementation provided: " + implementationClassName);
-        }
+        return KeyProviderFactory.buildKeyProvider(implementationClassName, config.getKeyProviderLocation(), config.getKeyId(), config.getEncryptionKeys(), masterKey);
+    }
 
-        return keyProvider;
+    private static SecretKey getMasterKey() throws KeyManagementException {
+        try {
+            // Get the master encryption key from bootstrap.conf
+            String masterKeyHex = NiFiPropertiesLoader.extractKeyFromBootstrapFile();
+            return new SecretKeySpec(Hex.decodeHex(masterKeyHex.toCharArray()), "AES");
+        } catch (IOException | DecoderException e) {
+            logger.error("Encountered an error: ", e);
+            throw new KeyManagementException(e);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java
index 5a75172..1c8e33b 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java
@@ -28,6 +28,7 @@ import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.provenance.search.SearchableField;
+import org.apache.nifi.security.kms.CryptoUtils;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -371,6 +372,8 @@ public class RepositoryConfiguration {
         return keyProviderIsConfigured;
     }
 
+    // TODO: Add verbose error output for encryption support failure if requested
+
     public Map<String, String> getEncryptionKeys() {
         return encryptionKeys;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java
index 7ae4adc..8355426 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java
@@ -30,7 +30,6 @@ import java.util.zip.GZIPInputStream;
 import org.apache.nifi.properties.NiFiPropertiesLoader;
 import org.apache.nifi.provenance.ByteArraySchemaRecordReader;
 import org.apache.nifi.provenance.ByteArraySchemaRecordWriter;
-import org.apache.nifi.provenance.CryptoUtils;
 import org.apache.nifi.provenance.EncryptedSchemaRecordReader;
 import org.apache.nifi.provenance.EventIdFirstSchemaRecordReader;
 import org.apache.nifi.provenance.EventIdFirstSchemaRecordWriter;
@@ -39,6 +38,7 @@ import org.apache.nifi.provenance.lucene.LuceneUtil;
 import org.apache.nifi.provenance.toc.StandardTocReader;
 import org.apache.nifi.provenance.toc.TocReader;
 import org.apache.nifi.provenance.toc.TocUtil;
+import org.apache.nifi.security.kms.CryptoUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy
index ec8c225..c146982 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy
@@ -24,6 +24,7 @@ import org.apache.nifi.provenance.toc.StandardTocWriter
 import org.apache.nifi.provenance.toc.TocReader
 import org.apache.nifi.provenance.toc.TocUtil
 import org.apache.nifi.provenance.toc.TocWriter
+import org.apache.nifi.security.kms.KeyProvider
 import org.apache.nifi.util.file.FileUtils
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.bouncycastle.util.encoders.Hex

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy
index 01c4982..582a805 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy
@@ -20,6 +20,7 @@ import org.apache.nifi.events.EventReporter
 import org.apache.nifi.flowfile.FlowFile
 import org.apache.nifi.provenance.serialization.RecordReaders
 import org.apache.nifi.reporting.Severity
+import org.apache.nifi.security.kms.StaticKeyProvider
 import org.apache.nifi.util.file.FileUtils
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/security/util/crypto/PasswordBasedEncryptorGroovyTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/security/util/crypto/PasswordBasedEncryptorGroovyTest.groovy b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/security/util/crypto/PasswordBasedEncryptorGroovyTest.groovy
index 7008381..f140d02 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/security/util/crypto/PasswordBasedEncryptorGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/security/util/crypto/PasswordBasedEncryptorGroovyTest.groovy
@@ -18,6 +18,7 @@ package org.apache.nifi.security.util.crypto
 
 import org.apache.commons.codec.binary.Hex
 import org.apache.nifi.processor.io.StreamCallback
+import org.apache.nifi.security.kms.CryptoUtils
 import org.apache.nifi.security.util.EncryptionMethod
 import org.apache.nifi.security.util.KeyDerivationFunction
 import org.apache.nifi.stream.io.ByteArrayOutputStream
@@ -33,7 +34,7 @@ import org.slf4j.LoggerFactory
 import javax.crypto.Cipher
 import java.security.Security
 
-public class PasswordBasedEncryptorGroovyTest {
+class PasswordBasedEncryptorGroovyTest {
     private static final Logger logger = LoggerFactory.getLogger(PasswordBasedEncryptorGroovyTest.class)
 
     private static final String TEST_RESOURCES_PREFIX = "src/test/resources/TestEncryptContent/"
@@ -44,7 +45,7 @@ public class PasswordBasedEncryptorGroovyTest {
     private static final String LEGACY_PASSWORD = "Hello, World!"
 
     @BeforeClass
-    public static void setUpOnce() throws Exception {
+    static void setUpOnce() throws Exception {
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
@@ -53,15 +54,15 @@ public class PasswordBasedEncryptorGroovyTest {
     }
 
     @Before
-    public void setUp() throws Exception {
+    void setUp() throws Exception {
     }
 
     @After
-    public void tearDown() throws Exception {
+    void tearDown() throws Exception {
     }
 
     @Test
-    public void testShouldEncryptAndDecrypt() throws Exception {
+    void testShouldEncryptAndDecrypt() throws Exception {
         // Arrange
         final String PLAINTEXT = "This is a plaintext message."
         logger.info("Plaintext: {}", PLAINTEXT)
@@ -107,7 +108,7 @@ public class PasswordBasedEncryptorGroovyTest {
     }
 
     @Test
-    public void testShouldDecryptLegacyOpenSSLSaltedCipherText() throws Exception {
+    void testShouldDecryptLegacyOpenSSLSaltedCipherText() throws Exception {
         // Arrange
         Assume.assumeTrue("Skipping test because unlimited strength crypto policy not installed", PasswordBasedEncryptor.supportsUnlimitedStrength())
 
@@ -136,7 +137,7 @@ public class PasswordBasedEncryptorGroovyTest {
     }
 
     @Test
-    public void testShouldDecryptLegacyOpenSSLUnsaltedCipherText() throws Exception {
+    void testShouldDecryptLegacyOpenSSLUnsaltedCipherText() throws Exception {
         // Arrange
         Assume.assumeTrue("Skipping test because unlimited strength crypto policy not installed", PasswordBasedEncryptor.supportsUnlimitedStrength())
 
@@ -165,7 +166,7 @@ public class PasswordBasedEncryptorGroovyTest {
     }
 
     @Test
-    public void testShouldDecryptNiFiLegacySaltedCipherTextWithVariableSaltLength() throws Exception {
+    void testShouldDecryptNiFiLegacySaltedCipherTextWithVariableSaltLength() throws Exception {
         // Arrange
         final String PLAINTEXT = new File("${TEST_RESOURCES_PREFIX}/plain.txt").text
         logger.info("Plaintext: {}", PLAINTEXT)
@@ -201,7 +202,7 @@ public class PasswordBasedEncryptorGroovyTest {
                 byte[] cipherBytes = legacyCipher.doFinal(PLAINTEXT.bytes)
                 logger.info("Cipher bytes: ${Hex.encodeHexString(cipherBytes)}")
 
-                byte[] completeCipherStreamBytes = CipherUtility.concatBytes(legacySalt, cipherBytes)
+                byte[] completeCipherStreamBytes = CryptoUtils.concatByteArrays(legacySalt, cipherBytes)
                 logger.info("Complete cipher stream: ${Hex.encodeHexString(completeCipherStreamBytes)}")
 
                 InputStream cipherStream = new ByteArrayInputStream(completeCipherStreamBytes)


[2/2] nifi git commit: NIFI-4139 - Moved key provider interface and implementations from nifi-data-provenance-utils module to nifi-security-utils module. - Refactored duplicate byte[] concatenation methods from utility classes and removed deprecation war

Posted by mc...@apache.org.
NIFI-4139
- Moved key provider interface and implementations from nifi-data-provenance-utils module to nifi-security-utils module.
- Refactored duplicate byte[] concatenation methods from utility classes and removed deprecation warnings from CipherUtility.
- Created KeyProviderFactory to encapsulate key provider instantiation logic.
- Added logic to handle legacy package configuration values for key providers.
- Added unit tests.
- Added resource files for un/limited strength cryptography scenarios.
- Added ASL to test resources.
- Moved legacy FQCN handling logic to CryptUtils.
- Added unit tests to ensure application startup logic handles legacy FQCNs.
- Moved master key extraction/provision out of FBKP.
- Removed nifi-security-utils dependency on nifi-properties-loader module.
- Added unit tests.


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

Branch: refs/heads/master
Commit: 675d9890031ebc79a21e136f82046c90dc8ddd89
Parents: 8b54c26
Author: Andy LoPresto <al...@apache.org>
Authored: Thu Jul 27 17:11:10 2017 -0700
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Aug 7 15:21:58 2017 -0400

----------------------------------------------------------------------
 .../provenance/AESProvenanceEventEncryptor.java |   2 +
 .../org/apache/nifi/provenance/CryptoUtils.java | 248 ---------
 .../nifi/provenance/FileBasedKeyProvider.java   |  67 ---
 .../org/apache/nifi/provenance/KeyProvider.java |  60 ---
 .../provenance/ProvenanceEventEncryptor.java    |   1 +
 .../nifi/provenance/StaticKeyProvider.java      |  96 ----
 .../AESProvenanceEventEncryptorTest.groovy      |   2 +
 .../nifi/provenance/CryptoUtilsTest.groovy      | 476 -----------------
 .../apache/nifi/security/kms/CryptoUtils.java   | 288 +++++++++++
 .../nifi/security/kms/FileBasedKeyProvider.java |  48 ++
 .../apache/nifi/security/kms/KeyProvider.java   |  60 +++
 .../nifi/security/kms/KeyProviderFactory.java   |  72 +++
 .../nifi/security/kms/StaticKeyProvider.java    |  96 ++++
 .../security/util/crypto/CipherUtility.java     |  11 +-
 .../nifi/security/kms/CryptoUtilsTest.groovy    | 506 +++++++++++++++++++
 .../security/kms/KeyProviderFactoryTest.groovy  | 229 +++++++++
 .../src/test/resources/128/conf/bootstrap.conf  |  19 +
 .../src/test/resources/256/conf/bootstrap.conf  |  19 +
 .../src/main/asciidoc/administration-guide.adoc |   2 +-
 nifi-docs/src/main/asciidoc/user-guide.adoc     |   2 +-
 ...EncryptedWriteAheadProvenanceRepository.java |  63 +--
 .../provenance/RepositoryConfiguration.java     |   3 +
 .../provenance/serialization/RecordReaders.java |   2 +-
 ...EncryptedSchemaRecordReaderWriterTest.groovy |   1 +
 ...tedWriteAheadProvenanceRepositoryTest.groovy |   1 +
 .../PasswordBasedEncryptorGroovyTest.groovy     |  19 +-
 26 files changed, 1389 insertions(+), 1004 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java
index d2cc9ca..27dc2fa 100644
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java
+++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java
@@ -31,6 +31,8 @@ import javax.crypto.Cipher;
 import javax.crypto.IllegalBlockSizeException;
 import javax.crypto.SecretKey;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.security.kms.CryptoUtils;
+import org.apache.nifi.security.kms.KeyProvider;
 import org.apache.nifi.security.util.EncryptionMethod;
 import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java
deleted file mode 100644
index 1b8f11a..0000000
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * 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.provenance;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-import javax.crypto.BadPaddingException;
-import javax.crypto.Cipher;
-import javax.crypto.IllegalBlockSizeException;
-import javax.crypto.SecretKey;
-import javax.crypto.spec.SecretKeySpec;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.security.util.EncryptionMethod;
-import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
-import org.apache.nifi.util.NiFiProperties;
-import org.bouncycastle.util.encoders.Hex;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class CryptoUtils {
-    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
-    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
-    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
-    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
-
-    private static final List<Integer> UNLIMITED_KEY_LENGTHS = Arrays.asList(32, 48, 64);
-
-    public static final int IV_LENGTH = 16;
-
-    public static boolean isUnlimitedStrengthCryptoAvailable() {
-        try {
-            return Cipher.getMaxAllowedKeyLength("AES") > 128;
-        } catch (NoSuchAlgorithmException e) {
-            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
-            return false;
-        }
-    }
-
-    /**
-     * Utility method which returns true if the string is null, empty, or entirely whitespace.
-     *
-     * @param src the string to evaluate
-     * @return true if empty
-     */
-    public static boolean isEmpty(String src) {
-        return src == null || src.trim().isEmpty();
-    }
-
-    /**
-     * Concatenates multiple byte[] into a single byte[].
-     *
-     * @param arrays the component byte[] in order
-     * @return a concatenated byte[]
-     * @throws IOException this should never be thrown
-     */
-    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
-        int totalByteLength = 0;
-        for (byte[] bytes : arrays) {
-            totalByteLength += bytes.length;
-        }
-        byte[] totalBytes = new byte[totalByteLength];
-        int currentLength = 0;
-        for (byte[] bytes : arrays) {
-            System.arraycopy(bytes, 0, totalBytes, currentLength, bytes.length);
-            currentLength += bytes.length;
-        }
-        return totalBytes;
-    }
-
-    /**
-     * Returns true if the provided configuration values successfully define the specified {@link KeyProvider}.
-     *
-     * @param keyProviderImplementation the FQ class name of the {@link KeyProvider} implementation
-     * @param keyProviderLocation       the location of the definition (for {@link FileBasedKeyProvider}, etc.)
-     * @param keyId                     the active key ID
-     * @param encryptionKeys            a map of key IDs to key material in hex format
-     * @return true if the provided configuration is valid
-     */
-    public static boolean isValidKeyProvider(String keyProviderImplementation, String keyProviderLocation, String keyId, Map<String, String> encryptionKeys) {
-        if (STATIC_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
-            // Ensure the keyId and key(s) are valid
-            if (encryptionKeys == null) {
-                return false;
-            } else {
-                boolean everyKeyValid = encryptionKeys.values().stream().allMatch(CryptoUtils::keyIsValid);
-                return everyKeyValid && StringUtils.isNotEmpty(keyId);
-            }
-        } else if (FILE_BASED_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
-            // Ensure the file can be read and the keyId is populated (does not read file to validate)
-            final File kpf = new File(keyProviderLocation);
-            return kpf.exists() && kpf.canRead() && StringUtils.isNotEmpty(keyId);
-        } else {
-            logger.error("The attempt to validate the key provider failed keyProviderImplementation = "
-                    + keyProviderImplementation + " , keyProviderLocation = "
-                    + keyProviderLocation + " , keyId = "
-                    + keyId + " , encryptionKeys = "
-                    + ((encryptionKeys == null) ? "0" : encryptionKeys.size()));
-
-            return false;
-        }
-    }
-
-    /**
-     * Returns true if the provided key is valid hex and is the correct length for the current system's JCE policies.
-     *
-     * @param encryptionKeyHex the key in hexadecimal
-     * @return true if this key is valid
-     */
-    public static boolean keyIsValid(String encryptionKeyHex) {
-        return isHexString(encryptionKeyHex)
-                && (isUnlimitedStrengthCryptoAvailable()
-                ? UNLIMITED_KEY_LENGTHS.contains(encryptionKeyHex.length())
-                : encryptionKeyHex.length() == 32);
-    }
-
-    /**
-     * Returns true if the input is valid hexadecimal (does not enforce length and is case-insensitive).
-     *
-     * @param hexString the string to evaluate
-     * @return true if the string is valid hex
-     */
-    public static boolean isHexString(String hexString) {
-        return StringUtils.isNotEmpty(hexString) && HEX_PATTERN.matcher(hexString).matches();
-    }
-
-    /**
-     * Returns a {@link SecretKey} formed from the hexadecimal key bytes (validity is checked).
-     *
-     * @param keyHex the key in hex form
-     * @return the SecretKey
-     */
-    public static SecretKey formKeyFromHex(String keyHex) throws KeyManagementException {
-        if (keyIsValid(keyHex)) {
-            return new SecretKeySpec(Hex.decode(keyHex), "AES");
-        } else {
-            throw new KeyManagementException("The provided key material is not valid");
-        }
-    }
-
-    /**
-     * Returns a map containing the key IDs and the parsed key from a key provider definition file.
-     * The values in the file are decrypted using the master key provided. If the file is missing or empty,
-     * cannot be read, or if no valid keys are read, a {@link KeyManagementException} will be thrown.
-     *
-     * @param filepath  the key definition file path
-     * @param masterKey the master key used to decrypt each key definition
-     * @return a Map of key IDs to SecretKeys
-     * @throws KeyManagementException if the file is missing or invalid
-     */
-    public static Map<String, SecretKey> readKeys(String filepath, SecretKey masterKey) throws KeyManagementException {
-        Map<String, SecretKey> keys = new HashMap<>();
-
-        if (StringUtils.isBlank(filepath)) {
-            throw new KeyManagementException("The key provider file is not present and readable");
-        }
-        File file = new File(filepath);
-        if (!file.exists() || !file.canRead()) {
-            throw new KeyManagementException("The key provider file is not present and readable");
-        }
-
-        try (BufferedReader br = new BufferedReader(new FileReader(file))) {
-            AESKeyedCipherProvider masterCipherProvider = new AESKeyedCipherProvider();
-
-            String line;
-            int l = 1;
-            while ((line = br.readLine()) != null) {
-                String[] components = line.split("=", 2);
-                if (components.length != 2 || StringUtils.isAnyEmpty(components)) {
-                    logger.warn("Line " + l + " is not properly formatted -- keyId=Base64EncodedKey...");
-                }
-                String keyId = components[0];
-                if (StringUtils.isNotEmpty(keyId)) {
-                    try {
-                        byte[] base64Bytes = Base64.getDecoder().decode(components[1]);
-                        byte[] ivBytes = Arrays.copyOfRange(base64Bytes, 0, IV_LENGTH);
-
-                        Cipher masterCipher = null;
-                        try {
-                            masterCipher = masterCipherProvider.getCipher(EncryptionMethod.AES_GCM, masterKey, ivBytes, false);
-                        } catch (Exception e) {
-                            throw new KeyManagementException("Error building cipher to decrypt FileBaseKeyProvider definition at " + filepath, e);
-                        }
-                        byte[] individualKeyBytes = masterCipher.doFinal(Arrays.copyOfRange(base64Bytes, IV_LENGTH, base64Bytes.length));
-
-                        SecretKey key = new SecretKeySpec(individualKeyBytes, "AES");
-                        logger.debug("Read and decrypted key for " + keyId);
-                        if (keys.containsKey(keyId)) {
-                            logger.warn("Multiple key values defined for " + keyId + " -- using most recent value");
-                        }
-                        keys.put(keyId, key);
-                    } catch (IllegalArgumentException e) {
-                        logger.error("Encountered an error decoding Base64 for " + keyId + ": " + e.getLocalizedMessage());
-                    } catch (BadPaddingException | IllegalBlockSizeException e) {
-                        logger.error("Encountered an error decrypting key for " + keyId + ": " + e.getLocalizedMessage());
-                    }
-                }
-                l++;
-            }
-
-            if (keys.isEmpty()) {
-                throw new KeyManagementException("The provided file contained no valid keys");
-            }
-
-            logger.info("Read " + keys.size() + " keys from FileBasedKeyProvider " + filepath);
-            return keys;
-        } catch (IOException e) {
-            throw new KeyManagementException("Error reading FileBasedKeyProvider definition at " + filepath, e);
-        }
-
-    }
-
-    public static boolean isProvenanceRepositoryEncryptionConfigured(NiFiProperties niFiProperties) {
-        final String implementationClassName = niFiProperties.getProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
-        // Referencing EWAPR.class.getName() would require a dependency on the module
-        boolean encryptedRepo = "org.apache.nifi.provenance.EncryptedWriteAheadProvenanceRepository".equals(implementationClassName);
-        boolean keyProviderConfigured = isValidKeyProvider(
-                niFiProperties.getProperty(NiFiProperties.PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_IMPLEMENTATION_CLASS),
-                niFiProperties.getProperty(NiFiProperties.PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_LOCATION),
-                niFiProperties.getProvenanceRepoEncryptionKeyId(),
-                niFiProperties.getProvenanceRepoEncryptionKeys());
-
-        return encryptedRepo && keyProviderConfigured;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/FileBasedKeyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/FileBasedKeyProvider.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/FileBasedKeyProvider.java
deleted file mode 100644
index b70b3e8..0000000
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/FileBasedKeyProvider.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.provenance;
-
-import java.io.IOException;
-import java.security.KeyManagementException;
-import javax.crypto.SecretKey;
-import javax.crypto.spec.SecretKeySpec;
-import javax.naming.OperationNotSupportedException;
-import org.apache.nifi.properties.NiFiPropertiesLoader;
-import org.bouncycastle.util.encoders.Hex;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FileBasedKeyProvider extends StaticKeyProvider {
-    private static final Logger logger = LoggerFactory.getLogger(FileBasedKeyProvider.class);
-
-    private String filepath;
-
-    FileBasedKeyProvider(String location) throws KeyManagementException {
-        this(location, getMasterKey());
-    }
-
-    FileBasedKeyProvider(String location, SecretKey masterKey) throws KeyManagementException {
-        super(CryptoUtils.readKeys(location, masterKey));
-        this.filepath = location;
-    }
-
-    private static SecretKey getMasterKey() throws KeyManagementException {
-        try {
-            // Get the master encryption key from bootstrap.conf
-            String masterKeyHex = NiFiPropertiesLoader.extractKeyFromBootstrapFile();
-            return new SecretKeySpec(Hex.decode(masterKeyHex), "AES");
-        } catch (IOException e) {
-            logger.error("Encountered an error: ", e);
-            throw new KeyManagementException(e);
-        }
-    }
-
-    /**
-     * Adds the key to the provider and associates it with the given ID. Some implementations may not allow this operation.
-     *
-     * @param keyId the key identifier
-     * @param key   the key
-     * @return true if the key was successfully added
-     * @throws OperationNotSupportedException if this implementation doesn't support adding keys
-     * @throws KeyManagementException         if the key is invalid, the ID conflicts, etc.
-     */
-    @Override
-    public boolean addKey(String keyId, SecretKey key) throws OperationNotSupportedException, KeyManagementException {
-        throw new OperationNotSupportedException("This implementation does not allow adding keys. Modify the file backing this provider at " + filepath);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/KeyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/KeyProvider.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/KeyProvider.java
deleted file mode 100644
index 39f6384..0000000
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/KeyProvider.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.provenance;
-
-import java.security.KeyManagementException;
-import java.util.List;
-import javax.crypto.SecretKey;
-import javax.naming.OperationNotSupportedException;
-
-public interface KeyProvider {
-
-    /**
-     * Returns the key identified by this ID or throws an exception if one is not available.
-     *
-     * @param keyId the key identifier
-     * @return the key
-     * @throws KeyManagementException if the key cannot be retrieved
-     */
-     SecretKey getKey(String keyId) throws KeyManagementException;
-
-    /**
-     * Returns true if the key exists and is available. Null or empty IDs will return false.
-     *
-     * @param keyId the key identifier
-     * @return true if the key can be used
-     */
-     boolean keyExists(String keyId);
-
-    /**
-     * Returns a list of available key identifiers (useful for encryption, as retired keys may not be listed here even if they are available for decryption for legacy/BC reasons).
-     *
-     * @return a List of keyIds (empty list if none are available)
-     */
-     List<String> getAvailableKeyIds();
-
-    /**
-     * Adds the key to the provider and associates it with the given ID. Some implementations may not allow this operation.
-     *
-     * @param keyId the key identifier
-     * @param key the key
-     * @return true if the key was successfully added
-     * @throws OperationNotSupportedException if this implementation doesn't support adding keys
-     * @throws KeyManagementException if the key is invalid, the ID conflicts, etc.
-     */
-     boolean addKey(String keyId, SecretKey key) throws OperationNotSupportedException, KeyManagementException;
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProvenanceEventEncryptor.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProvenanceEventEncryptor.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProvenanceEventEncryptor.java
index c7690e1..26df307 100644
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProvenanceEventEncryptor.java
+++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProvenanceEventEncryptor.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.provenance;
 
 import java.security.KeyManagementException;
+import org.apache.nifi.security.kms.KeyProvider;
 
 public interface ProvenanceEventEncryptor {
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StaticKeyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StaticKeyProvider.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StaticKeyProvider.java
deleted file mode 100644
index e0981dc..0000000
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StaticKeyProvider.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.provenance;
-
-import java.security.KeyManagementException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.crypto.SecretKey;
-import javax.naming.OperationNotSupportedException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Reference implementation for static key provider (used during tests).
- */
-public class StaticKeyProvider implements KeyProvider {
-    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
-
-    private Map<String, SecretKey> keys = new HashMap<>();
-
-    StaticKeyProvider(String keyId, String keyHex) throws KeyManagementException {
-        this.keys.put(keyId, CryptoUtils.formKeyFromHex(keyHex));
-    }
-
-    StaticKeyProvider(Map<String, SecretKey> keys) throws KeyManagementException {
-        this.keys.putAll(keys);
-    }
-
-    /**
-     * Returns the key identified by this ID or throws an exception if one is not available.
-     *
-     * @param keyId the key identifier
-     * @return the key
-     * @throws KeyManagementException if the key cannot be retrieved
-     */
-    @Override
-    public SecretKey getKey(String keyId) throws KeyManagementException {
-        logger.debug("Attempting to get key: " + keyId);
-        if (keyExists(keyId)) {
-            return keys.get(keyId);
-        } else {
-            throw new KeyManagementException("No key available for " + keyId);
-        }
-    }
-
-    /**
-     * Returns true if the key exists and is available. Null or empty IDs will return false.
-     *
-     * @param keyId the key identifier
-     * @return true if the key can be used
-     */
-    @Override
-    public boolean keyExists(String keyId) {
-        return keys.containsKey(keyId);
-    }
-
-    /**
-     * Returns a singleton list of the available key identifier.
-     *
-     * @return a List containing the {@code KEY_ID}
-     */
-    @Override
-    public List<String> getAvailableKeyIds() {
-        return new ArrayList<>(keys.keySet());
-    }
-
-    /**
-     * Adds the key to the provider and associates it with the given ID. Some implementations may not allow this operation.
-     *
-     * @param keyId the key identifier
-     * @param key   the key
-     * @return true if the key was successfully added
-     * @throws OperationNotSupportedException if this implementation doesn't support adding keys
-     * @throws KeyManagementException         if the key is invalid, the ID conflicts, etc.
-     */
-    @Override
-    public boolean addKey(String keyId, SecretKey key) throws OperationNotSupportedException, KeyManagementException {
-        throw new OperationNotSupportedException("This implementation does not allow adding keys");
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/AESProvenanceEventEncryptorTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/AESProvenanceEventEncryptorTest.groovy b/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/AESProvenanceEventEncryptorTest.groovy
index 61b35d0..2916de1 100644
--- a/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/AESProvenanceEventEncryptorTest.groovy
+++ b/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/AESProvenanceEventEncryptorTest.groovy
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.provenance
 
+import org.apache.nifi.security.kms.CryptoUtils
+import org.apache.nifi.security.kms.KeyProvider
 import org.apache.nifi.security.util.EncryptionMethod
 import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider
 import org.bouncycastle.jce.provider.BouncyCastleProvider

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/CryptoUtilsTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/CryptoUtilsTest.groovy b/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/CryptoUtilsTest.groovy
deleted file mode 100644
index 9ba8e97..0000000
--- a/nifi-commons/nifi-data-provenance-utils/src/test/groovy/org/apache/nifi/provenance/CryptoUtilsTest.groovy
+++ /dev/null
@@ -1,476 +0,0 @@
-/*
- * 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.provenance
-
-import org.apache.commons.lang3.SystemUtils
-import org.bouncycastle.jce.provider.BouncyCastleProvider
-import org.bouncycastle.util.encoders.Hex
-import org.junit.After
-import org.junit.AfterClass
-import org.junit.Assume
-import org.junit.Before
-import org.junit.BeforeClass
-import org.junit.ClassRule
-import org.junit.Test
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.JUnit4
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
-
-import javax.crypto.Cipher
-import javax.crypto.SecretKey
-import javax.crypto.spec.IvParameterSpec
-import javax.crypto.spec.SecretKeySpec
-import java.nio.charset.StandardCharsets
-import java.nio.file.Files
-import java.nio.file.attribute.PosixFilePermission
-import java.security.KeyManagementException
-import java.security.SecureRandom
-import java.security.Security
-
-import static groovy.test.GroovyAssert.shouldFail
-
-@RunWith(JUnit4.class)
-class CryptoUtilsTest {
-    private static final Logger logger = LoggerFactory.getLogger(CryptoUtilsTest.class)
-
-    private static final String KEY_ID = "K1"
-    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
-    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
-    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
-
-    private static
-    final Set<PosixFilePermission> ALL_POSIX_ATTRS = PosixFilePermission.values() as Set<PosixFilePermission>
-
-    @ClassRule
-    public static TemporaryFolder tempFolder = new TemporaryFolder()
-
-    @BeforeClass
-    static void setUpOnce() throws Exception {
-        Security.addProvider(new BouncyCastleProvider())
-
-        logger.metaClass.methodMissing = { String name, args ->
-            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
-        }
-    }
-
-    @Before
-    void setUp() throws Exception {
-        tempFolder.create()
-    }
-
-    @After
-    void tearDown() throws Exception {
-        tempFolder?.delete()
-    }
-
-    @AfterClass
-    static void tearDownOnce() throws Exception {
-
-    }
-
-    private static boolean isUnlimitedStrengthCryptoAvailable() {
-        Cipher.getMaxAllowedKeyLength("AES") > 128
-    }
-
-    private static boolean isRootUser() {
-        ProcessBuilder pb = new ProcessBuilder(["id", "-u"])
-        Process process = pb.start()
-        InputStream responseStream = process.getInputStream()
-        BufferedReader responseReader = new BufferedReader(new InputStreamReader(responseStream))
-        responseReader.text.trim() == "0"
-    }
-
-    @Test
-    void testShouldConcatenateByteArrays() {
-        // Arrange
-        byte[] bytes1 = "These are some bytes".getBytes(StandardCharsets.UTF_8)
-        byte[] bytes2 = "These are some other bytes".getBytes(StandardCharsets.UTF_8)
-        final byte[] EXPECTED_CONCATENATED_BYTES = ((bytes1 as List) << (bytes2 as List)).flatten() as byte[]
-        logger.info("Expected concatenated bytes: ${Hex.toHexString(EXPECTED_CONCATENATED_BYTES)}")
-
-        // Act
-        byte[] concat = CryptoUtils.concatByteArrays(bytes1, bytes2)
-        logger.info("  Actual concatenated bytes: ${Hex.toHexString(concat)}")
-
-        // Assert
-        assert concat == EXPECTED_CONCATENATED_BYTES
-    }
-
-    @Test
-    void testShouldValidateStaticKeyProvider() {
-        // Arrange
-        String staticProvider = StaticKeyProvider.class.name
-        String providerLocation = null
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX])
-        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert keyProviderIsValid
-    }
-
-    @Test
-    void testShouldNotValidateStaticKeyProviderMissingKeyId() {
-        // Arrange
-        String staticProvider = StaticKeyProvider.class.name
-        String providerLocation = null
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, null, [(KEY_ID): KEY_HEX])
-        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${null} / ${KEY_HEX} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !keyProviderIsValid
-    }
-
-    @Test
-    void testShouldNotValidateStaticKeyProviderMissingKey() {
-        // Arrange
-        String staticProvider = StaticKeyProvider.class.name
-        String providerLocation = null
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, null)
-        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !keyProviderIsValid
-    }
-
-    @Test
-    void testShouldNotValidateStaticKeyProviderWithInvalidKey() {
-        // Arrange
-        String staticProvider = StaticKeyProvider.class.name
-        String providerLocation = null
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(staticProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX[0..<-2]])
-        logger.info("Key Provider ${staticProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX[0..<-2]} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !keyProviderIsValid
-    }
-
-    @Test
-    void testShouldValidateFileBasedKeyProvider() {
-        // Arrange
-        String fileBasedProvider = FileBasedKeyProvider.class.name
-        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
-        String providerLocation = fileBasedProviderFile.path
-        logger.info("Created temporary file based key provider: ${providerLocation}")
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, KEY_ID, null)
-        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert keyProviderIsValid
-    }
-
-    @Test
-    void testShouldNotValidateMissingFileBasedKeyProvider() {
-        // Arrange
-        String fileBasedProvider = FileBasedKeyProvider.class.name
-        File fileBasedProviderFile = new File(tempFolder.root, "filebased_missing.kp")
-        String providerLocation = fileBasedProviderFile.path
-        logger.info("Created (no actual file) temporary file based key provider: ${providerLocation}")
-
-        // Act
-        String missingLocation = providerLocation
-        boolean missingKeyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, missingLocation, KEY_ID, null)
-        logger.info("Key Provider ${fileBasedProvider} with location ${missingLocation} and keyId ${KEY_ID} / ${null} is ${missingKeyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !missingKeyProviderIsValid
-    }
-
-    @Test
-    void testShouldNotValidateUnreadableFileBasedKeyProvider() {
-        // Arrange
-        Assume.assumeFalse("This test does not run on Windows", SystemUtils.IS_OS_WINDOWS)
-        Assume.assumeFalse("This test does not run for root users", isRootUser())
-
-        String fileBasedProvider = FileBasedKeyProvider.class.name
-        File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
-        String providerLocation = fileBasedProviderFile.path
-        logger.info("Created temporary file based key provider: ${providerLocation}")
-
-        // Make it unreadable
-        markFileUnreadable(fileBasedProviderFile)
-
-        // Act
-        boolean unreadableKeyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, KEY_ID, null)
-        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${unreadableKeyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !unreadableKeyProviderIsValid
-
-        // Make the file deletable so cleanup can occur
-        markFileReadable(fileBasedProviderFile)
-    }
-
-    private static void markFileReadable(File fileBasedProviderFile) {
-        if (SystemUtils.IS_OS_WINDOWS) {
-            fileBasedProviderFile.setReadable(true, false)
-        } else {
-            Files.setPosixFilePermissions(fileBasedProviderFile.toPath(), ALL_POSIX_ATTRS)
-        }
-    }
-
-    private static void markFileUnreadable(File fileBasedProviderFile) {
-        if (SystemUtils.IS_OS_WINDOWS) {
-            fileBasedProviderFile.setReadable(false, false)
-        } else {
-            Files.setPosixFilePermissions(fileBasedProviderFile.toPath(), [] as Set<PosixFilePermission>)
-        }
-    }
-
-    @Test
-    void testShouldNotValidateFileBasedKeyProviderMissingKeyId() {
-        // Arrange
-        String fileBasedProvider = FileBasedKeyProvider.class.name
-        File fileBasedProviderFile = tempFolder.newFile("missing_key_id.kp")
-        String providerLocation = fileBasedProviderFile.path
-        logger.info("Created temporary file based key provider: ${providerLocation}")
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(fileBasedProvider, providerLocation, null, null)
-        logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${null} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !keyProviderIsValid
-    }
-
-    @Test
-    void testShouldNotValidateUnknownKeyProvider() {
-        // Arrange
-        String providerImplementation = "org.apache.nifi.provenance.ImaginaryKeyProvider"
-        String providerLocation = null
-
-        // Act
-        boolean keyProviderIsValid = CryptoUtils.isValidKeyProvider(providerImplementation, providerLocation, KEY_ID, null)
-        logger.info("Key Provider ${providerImplementation} with location ${providerLocation} and keyId ${KEY_ID} / ${null} is ${keyProviderIsValid ? "valid" : "invalid"}")
-
-        // Assert
-        assert !keyProviderIsValid
-    }
-
-    @Test
-    void testShouldValidateKey() {
-        // Arrange
-        String validKey = KEY_HEX
-        String validLowercaseKey = KEY_HEX.toLowerCase()
-
-        String tooShortKey = KEY_HEX[0..<-2]
-        String tooLongKey = KEY_HEX + KEY_HEX // Guaranteed to be 2x the max valid key length
-        String nonHexKey = KEY_HEX.replaceFirst(/A/, "X")
-
-        def validKeys = [validKey, validLowercaseKey]
-        def invalidKeys = [tooShortKey, tooLongKey, nonHexKey]
-
-        // If unlimited strength is available, also validate 128 and 196 bit keys
-        if (isUnlimitedStrengthCryptoAvailable()) {
-            validKeys << KEY_HEX_128
-            validKeys << KEY_HEX_256[0..<48]
-        } else {
-            invalidKeys << KEY_HEX_256[0..<48]
-            invalidKeys << KEY_HEX_256
-        }
-
-        // Act
-        def validResults = validKeys.collect { String key ->
-            logger.info("Validating ${key}")
-            CryptoUtils.keyIsValid(key)
-        }
-
-        def invalidResults = invalidKeys.collect { String key ->
-            logger.info("Validating ${key}")
-            CryptoUtils.keyIsValid(key)
-        }
-
-        // Assert
-        assert validResults.every()
-        assert invalidResults.every { !it }
-    }
-
-    @Test
-    void testShouldReadKeys() {
-        // Arrange
-        String masterKeyHex = KEY_HEX
-        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
-
-        // Generate the file
-        String keyFileName = "keys.nkp"
-        File keyFile = tempFolder.newFile(keyFileName)
-        final int KEY_COUNT = 5
-        List<String> lines = []
-        KEY_COUNT.times { int i ->
-            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
-        }
-
-        keyFile.text = lines.join("\n")
-
-        logger.info("File contents: \n${keyFile.text}")
-
-        // Act
-        def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
-        logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
-
-        // Assert
-        assert readKeys.size() == KEY_COUNT
-    }
-
-    @Test
-    void testShouldReadKeysWithDuplicates() {
-        // Arrange
-        String masterKeyHex = KEY_HEX
-        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
-
-        // Generate the file
-        String keyFileName = "keys.nkp"
-        File keyFile = tempFolder.newFile(keyFileName)
-        final int KEY_COUNT = 3
-        List<String> lines = []
-        KEY_COUNT.times { int i ->
-            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
-        }
-
-        lines.add("key3=${generateEncryptedKey(masterKey)}")
-
-        keyFile.text = lines.join("\n")
-
-        logger.info("File contents: \n${keyFile.text}")
-
-        // Act
-        def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
-        logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
-
-        // Assert
-        assert readKeys.size() == KEY_COUNT
-    }
-
-    @Test
-    void testShouldReadKeysWithSomeMalformed() {
-        // Arrange
-        String masterKeyHex = KEY_HEX
-        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
-
-        // Generate the file
-        String keyFileName = "keys.nkp"
-        File keyFile = tempFolder.newFile(keyFileName)
-        final int KEY_COUNT = 5
-        List<String> lines = []
-        KEY_COUNT.times { int i ->
-            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
-        }
-
-        // Insert the malformed keys in the middle
-        lines.add(2, "keyX1==${generateEncryptedKey(masterKey)}")
-        lines.add(4, "=${generateEncryptedKey(masterKey)}")
-        lines.add(6, "keyX3=non Base64-encoded data")
-
-        keyFile.text = lines.join("\n")
-
-        logger.info("File contents: \n${keyFile.text}")
-
-        // Act
-        def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
-        logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
-
-        // Assert
-        assert readKeys.size() == KEY_COUNT
-    }
-
-    @Test
-    void testShouldNotReadKeysIfAllMalformed() {
-        // Arrange
-        String masterKeyHex = KEY_HEX
-        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
-
-        // Generate the file
-        String keyFileName = "keys.nkp"
-        File keyFile = tempFolder.newFile(keyFileName)
-        final int KEY_COUNT = 5
-        List<String> lines = []
-
-        // All of these keys are malformed
-        KEY_COUNT.times { int i ->
-            lines.add("key${i + 1}=${generateEncryptedKey(masterKey)[0..<-4]}")
-        }
-
-        keyFile.text = lines.join("\n")
-
-        logger.info("File contents: \n${keyFile.text}")
-
-        // Act
-        def msg = shouldFail(KeyManagementException) {
-            def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
-            logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
-        }
-
-        // Assert
-        assert msg.getMessage() == "The provided file contained no valid keys"
-    }
-
-    @Test
-    void testShouldNotReadKeysIfEmptyOrMissing() {
-        // Arrange
-        String masterKeyHex = KEY_HEX
-        SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
-
-        // Generate the file
-        String keyFileName = "empty.nkp"
-        File keyFile = tempFolder.newFile(keyFileName)
-        logger.info("File contents: \n${keyFile.text}")
-
-        // Act
-        def missingMsg = shouldFail(KeyManagementException) {
-            def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
-            logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
-        }
-        logger.expected("Missing file: ${missingMsg}")
-
-        def emptyMsg = shouldFail(KeyManagementException) {
-            def readKeys = CryptoUtils.readKeys(null, masterKey)
-            logger.info("Read ${readKeys.size()} keys from ${null}")
-        }
-        logger.expected("Empty file: ${emptyMsg}")
-
-        // Assert
-        assert missingMsg.getMessage() == "The provided file contained no valid keys"
-        assert emptyMsg.getMessage() == "The key provider file is not present and readable"
-    }
-
-    private static String generateEncryptedKey(SecretKey masterKey) {
-        byte[] ivBytes = new byte[16]
-        byte[] keyBytes = new byte[isUnlimitedStrengthCryptoAvailable() ? 32 : 16]
-
-        SecureRandom sr = new SecureRandom()
-        sr.nextBytes(ivBytes)
-        sr.nextBytes(keyBytes)
-
-        Cipher masterCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
-        masterCipher.init(Cipher.ENCRYPT_MODE, masterKey, new IvParameterSpec(ivBytes))
-        byte[] cipherBytes = masterCipher.doFinal(keyBytes)
-
-        Base64.encoder.encodeToString(CryptoUtils.concatByteArrays(ivBytes, cipherBytes))
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/CryptoUtils.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/CryptoUtils.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/CryptoUtils.java
new file mode 100644
index 0000000..44b9787
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/CryptoUtils.java
@@ -0,0 +1,288 @@
+/*
+ * 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.kms;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import javax.crypto.BadPaddingException;
+import javax.crypto.Cipher;
+import javax.crypto.IllegalBlockSizeException;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
+import org.apache.nifi.util.NiFiProperties;
+import org.bouncycastle.util.encoders.Hex;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CryptoUtils {
+    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
+    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.security.kms.StaticKeyProvider";
+    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.security.kms.FileBasedKeyProvider";
+
+    private static final String LEGACY_SKP_FQCN = "org.apache.nifi.provenance.StaticKeyProvider";
+    private static final String LEGACY_FBKP_FQCN = "org.apache.nifi.provenance.FileBasedKeyProvider";
+
+
+    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
+
+    private static final List<Integer> UNLIMITED_KEY_LENGTHS = Arrays.asList(32, 48, 64);
+
+    public static final int IV_LENGTH = 16;
+
+    public static boolean isUnlimitedStrengthCryptoAvailable() {
+        try {
+            return Cipher.getMaxAllowedKeyLength("AES") > 128;
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
+            return false;
+        }
+    }
+
+    /**
+     * Utility method which returns true if the string is null, empty, or entirely whitespace.
+     *
+     * @param src the string to evaluate
+     * @return true if empty
+     */
+    public static boolean isEmpty(String src) {
+        return src == null || src.trim().isEmpty();
+    }
+
+    /**
+     * Concatenates multiple byte[] into a single byte[].
+     *
+     * @param arrays the component byte[] in order
+     * @return a concatenated byte[]
+     * @throws IOException this should never be thrown
+     */
+    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
+        int totalByteLength = 0;
+        for (byte[] bytes : arrays) {
+            totalByteLength += bytes.length;
+        }
+        byte[] totalBytes = new byte[totalByteLength];
+        int currentLength = 0;
+        for (byte[] bytes : arrays) {
+            System.arraycopy(bytes, 0, totalBytes, currentLength, bytes.length);
+            currentLength += bytes.length;
+        }
+        return totalBytes;
+    }
+
+    /**
+     * Returns true if the provided configuration values successfully define the specified {@link KeyProvider}.
+     *
+     * @param keyProviderImplementation the FQ class name of the {@link KeyProvider} implementation
+     * @param keyProviderLocation       the location of the definition (for {@link FileBasedKeyProvider}, etc.)
+     * @param keyId                     the active key ID
+     * @param encryptionKeys            a map of key IDs to key material in hex format
+     * @return true if the provided configuration is valid
+     */
+    public static boolean isValidKeyProvider(String keyProviderImplementation, String keyProviderLocation, String keyId, Map<String, String> encryptionKeys) {
+        logger.debug("Attempting to validate the key provider: keyProviderImplementation = "
+                + keyProviderImplementation + " , keyProviderLocation = "
+                + keyProviderLocation + " , keyId = "
+                + keyId + " , encryptionKeys = "
+                + ((encryptionKeys == null) ? "0" : encryptionKeys.size()));
+
+        try {
+            keyProviderImplementation = handleLegacyPackages(keyProviderImplementation);
+        } catch (KeyManagementException e) {
+            logger.error("The attempt to validate the key provider failed keyProviderImplementation = "
+                    + keyProviderImplementation + " , keyProviderLocation = "
+                    + keyProviderLocation + " , keyId = "
+                    + keyId + " , encryptionKeys = "
+                    + ((encryptionKeys == null) ? "0" : encryptionKeys.size()));
+
+            return false;
+        }
+
+        if (STATIC_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
+            // Ensure the keyId and key(s) are valid
+            if (encryptionKeys == null) {
+                return false;
+            } else {
+                boolean everyKeyValid = encryptionKeys.values().stream().allMatch(CryptoUtils::keyIsValid);
+                return everyKeyValid && StringUtils.isNotEmpty(keyId);
+            }
+        } else if (FILE_BASED_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
+            // Ensure the file can be read and the keyId is populated (does not read file to validate)
+            final File kpf = new File(keyProviderLocation);
+            return kpf.exists() && kpf.canRead() && StringUtils.isNotEmpty(keyId);
+        } else {
+            logger.error("The attempt to validate the key provider failed keyProviderImplementation = "
+                    + keyProviderImplementation + " , keyProviderLocation = "
+                    + keyProviderLocation + " , keyId = "
+                    + keyId + " , encryptionKeys = "
+                    + ((encryptionKeys == null) ? "0" : encryptionKeys.size()));
+
+            return false;
+        }
+    }
+
+    static String handleLegacyPackages(String implementationClassName) throws KeyManagementException {
+        if (org.apache.nifi.util.StringUtils.isBlank(implementationClassName)) {
+            throw new KeyManagementException("Invalid key provider implementation provided: " + implementationClassName);
+        }
+        if (implementationClassName.equalsIgnoreCase(LEGACY_SKP_FQCN)) {
+            return StaticKeyProvider.class.getName();
+        } else if (implementationClassName.equalsIgnoreCase(LEGACY_FBKP_FQCN)) {
+            return FileBasedKeyProvider.class.getName();
+        } else {
+            return implementationClassName;
+        }
+    }
+
+    /**
+     * Returns true if the provided key is valid hex and is the correct length for the current system's JCE policies.
+     *
+     * @param encryptionKeyHex the key in hexadecimal
+     * @return true if this key is valid
+     */
+    public static boolean keyIsValid(String encryptionKeyHex) {
+        return isHexString(encryptionKeyHex)
+                && (isUnlimitedStrengthCryptoAvailable()
+                ? UNLIMITED_KEY_LENGTHS.contains(encryptionKeyHex.length())
+                : encryptionKeyHex.length() == 32);
+    }
+
+    /**
+     * Returns true if the input is valid hexadecimal (does not enforce length and is case-insensitive).
+     *
+     * @param hexString the string to evaluate
+     * @return true if the string is valid hex
+     */
+    public static boolean isHexString(String hexString) {
+        return StringUtils.isNotEmpty(hexString) && HEX_PATTERN.matcher(hexString).matches();
+    }
+
+    /**
+     * Returns a {@link SecretKey} formed from the hexadecimal key bytes (validity is checked).
+     *
+     * @param keyHex the key in hex form
+     * @return the SecretKey
+     */
+    public static SecretKey formKeyFromHex(String keyHex) throws KeyManagementException {
+        if (keyIsValid(keyHex)) {
+            return new SecretKeySpec(Hex.decode(keyHex), "AES");
+        } else {
+            throw new KeyManagementException("The provided key material is not valid");
+        }
+    }
+
+    /**
+     * Returns a map containing the key IDs and the parsed key from a key provider definition file.
+     * The values in the file are decrypted using the master key provided. If the file is missing or empty,
+     * cannot be read, or if no valid keys are read, a {@link KeyManagementException} will be thrown.
+     *
+     * @param filepath  the key definition file path
+     * @param masterKey the master key used to decrypt each key definition
+     * @return a Map of key IDs to SecretKeys
+     * @throws KeyManagementException if the file is missing or invalid
+     */
+    public static Map<String, SecretKey> readKeys(String filepath, SecretKey masterKey) throws KeyManagementException {
+        Map<String, SecretKey> keys = new HashMap<>();
+
+        if (StringUtils.isBlank(filepath)) {
+            throw new KeyManagementException("The key provider file is not present and readable");
+        }
+        if (masterKey == null) {
+            throw new KeyManagementException("The master key must be provided to decrypt the individual keys");
+        }
+
+        File file = new File(filepath);
+        if (!file.exists() || !file.canRead()) {
+            throw new KeyManagementException("The key provider file is not present and readable");
+        }
+
+        try (BufferedReader br = new BufferedReader(new FileReader(file))) {
+            AESKeyedCipherProvider masterCipherProvider = new AESKeyedCipherProvider();
+
+            String line;
+            int l = 1;
+            while ((line = br.readLine()) != null) {
+                String[] components = line.split("=", 2);
+                if (components.length != 2 || StringUtils.isAnyEmpty(components)) {
+                    logger.warn("Line " + l + " is not properly formatted -- keyId=Base64EncodedKey...");
+                }
+                String keyId = components[0];
+                if (StringUtils.isNotEmpty(keyId)) {
+                    try {
+                        byte[] base64Bytes = Base64.getDecoder().decode(components[1]);
+                        byte[] ivBytes = Arrays.copyOfRange(base64Bytes, 0, IV_LENGTH);
+
+                        Cipher masterCipher = null;
+                        try {
+                            masterCipher = masterCipherProvider.getCipher(EncryptionMethod.AES_GCM, masterKey, ivBytes, false);
+                        } catch (Exception e) {
+                            throw new KeyManagementException("Error building cipher to decrypt FileBaseKeyProvider definition at " + filepath, e);
+                        }
+                        byte[] individualKeyBytes = masterCipher.doFinal(Arrays.copyOfRange(base64Bytes, IV_LENGTH, base64Bytes.length));
+
+                        SecretKey key = new SecretKeySpec(individualKeyBytes, "AES");
+                        logger.debug("Read and decrypted key for " + keyId);
+                        if (keys.containsKey(keyId)) {
+                            logger.warn("Multiple key values defined for " + keyId + " -- using most recent value");
+                        }
+                        keys.put(keyId, key);
+                    } catch (IllegalArgumentException e) {
+                        logger.error("Encountered an error decoding Base64 for " + keyId + ": " + e.getLocalizedMessage());
+                    } catch (BadPaddingException | IllegalBlockSizeException e) {
+                        logger.error("Encountered an error decrypting key for " + keyId + ": " + e.getLocalizedMessage());
+                    }
+                }
+                l++;
+            }
+
+            if (keys.isEmpty()) {
+                throw new KeyManagementException("The provided file contained no valid keys");
+            }
+
+            logger.info("Read " + keys.size() + " keys from FileBasedKeyProvider " + filepath);
+            return keys;
+        } catch (IOException e) {
+            throw new KeyManagementException("Error reading FileBasedKeyProvider definition at " + filepath, e);
+        }
+
+    }
+
+    public static boolean isProvenanceRepositoryEncryptionConfigured(NiFiProperties niFiProperties) {
+        final String implementationClassName = niFiProperties.getProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
+        // Referencing EWAPR.class.getName() would require a dependency on the module
+        boolean encryptedRepo = "org.apache.nifi.provenance.EncryptedWriteAheadProvenanceRepository".equals(implementationClassName);
+        boolean keyProviderConfigured = isValidKeyProvider(
+                niFiProperties.getProperty(NiFiProperties.PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_IMPLEMENTATION_CLASS),
+                niFiProperties.getProperty(NiFiProperties.PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_LOCATION),
+                niFiProperties.getProvenanceRepoEncryptionKeyId(),
+                niFiProperties.getProvenanceRepoEncryptionKeys());
+
+        return encryptedRepo && keyProviderConfigured;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/FileBasedKeyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/FileBasedKeyProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/FileBasedKeyProvider.java
new file mode 100644
index 0000000..5c4691d
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/FileBasedKeyProvider.java
@@ -0,0 +1,48 @@
+/*
+ * 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.kms;
+
+import java.security.KeyManagementException;
+import javax.crypto.SecretKey;
+import javax.naming.OperationNotSupportedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileBasedKeyProvider extends StaticKeyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(FileBasedKeyProvider.class);
+
+    private String filepath;
+
+    public FileBasedKeyProvider(String location, SecretKey masterKey) throws KeyManagementException {
+        super(CryptoUtils.readKeys(location, masterKey));
+        this.filepath = location;
+    }
+
+    /**
+     * Adds the key to the provider and associates it with the given ID. Some implementations may not allow this operation.
+     *
+     * @param keyId the key identifier
+     * @param key   the key
+     * @return true if the key was successfully added
+     * @throws OperationNotSupportedException if this implementation doesn't support adding keys
+     * @throws KeyManagementException         if the key is invalid, the ID conflicts, etc.
+     */
+    @Override
+    public boolean addKey(String keyId, SecretKey key) throws OperationNotSupportedException, KeyManagementException {
+        throw new OperationNotSupportedException("This implementation does not allow adding keys. Modify the file backing this provider at " + filepath);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProvider.java
new file mode 100644
index 0000000..4cbafc2
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProvider.java
@@ -0,0 +1,60 @@
+/*
+ * 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.kms;
+
+import java.security.KeyManagementException;
+import java.util.List;
+import javax.crypto.SecretKey;
+import javax.naming.OperationNotSupportedException;
+
+public interface KeyProvider {
+
+    /**
+     * Returns the key identified by this ID or throws an exception if one is not available.
+     *
+     * @param keyId the key identifier
+     * @return the key
+     * @throws KeyManagementException if the key cannot be retrieved
+     */
+     SecretKey getKey(String keyId) throws KeyManagementException;
+
+    /**
+     * Returns true if the key exists and is available. Null or empty IDs will return false.
+     *
+     * @param keyId the key identifier
+     * @return true if the key can be used
+     */
+     boolean keyExists(String keyId);
+
+    /**
+     * Returns a list of available key identifiers (useful for encryption, as retired keys may not be listed here even if they are available for decryption for legacy/BC reasons).
+     *
+     * @return a List of keyIds (empty list if none are available)
+     */
+     List<String> getAvailableKeyIds();
+
+    /**
+     * Adds the key to the provider and associates it with the given ID. Some implementations may not allow this operation.
+     *
+     * @param keyId the key identifier
+     * @param key the key
+     * @return true if the key was successfully added
+     * @throws OperationNotSupportedException if this implementation doesn't support adding keys
+     * @throws KeyManagementException if the key is invalid, the ID conflicts, etc.
+     */
+     boolean addKey(String keyId, SecretKey key) throws OperationNotSupportedException, KeyManagementException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProviderFactory.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProviderFactory.java
new file mode 100644
index 0000000..054aa25
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/KeyProviderFactory.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.kms;
+
+import java.security.KeyManagementException;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.crypto.SecretKey;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KeyProviderFactory {
+    private static final Logger logger = LoggerFactory.getLogger(KeyProviderFactory.class);
+
+    public static KeyProvider buildKeyProvider(String implementationClassName, String keyProviderLocation, String keyId, Map<String, String> encryptionKeys,
+                                               SecretKey masterKey) throws KeyManagementException {
+        KeyProvider keyProvider;
+
+       implementationClassName = CryptoUtils.handleLegacyPackages(implementationClassName);
+
+        if (StaticKeyProvider.class.getName().equals(implementationClassName)) {
+            // Get all the keys (map) from config
+            if (CryptoUtils.isValidKeyProvider(implementationClassName, keyProviderLocation, keyId, encryptionKeys)) {
+                Map<String, SecretKey> formedKeys = encryptionKeys.entrySet().stream()
+                        .collect(Collectors.toMap(
+                                Map.Entry::getKey,
+                                e -> {
+                                    try {
+                                        return CryptoUtils.formKeyFromHex(e.getValue());
+                                    } catch (KeyManagementException e1) {
+                                        // This should never happen because the hex has already been validated
+                                        logger.error("Encountered an error: ", e1);
+                                        return null;
+                                    }
+                                }));
+                keyProvider = new StaticKeyProvider(formedKeys);
+            } else {
+                final String msg = "The StaticKeyProvider definition is not valid";
+                logger.error(msg);
+                throw new KeyManagementException(msg);
+            }
+        } else if (FileBasedKeyProvider.class.getName().equals(implementationClassName)) {
+            keyProvider = new FileBasedKeyProvider(keyProviderLocation, masterKey);
+            if (!keyProvider.keyExists(keyId)) {
+                throw new KeyManagementException("The specified key ID " + keyId + " is not in the key definition file");
+            }
+        } else {
+            throw new KeyManagementException("Invalid key provider implementation provided: " + implementationClassName);
+        }
+
+        return keyProvider;
+    }
+
+    public static boolean requiresMasterKey(String implementationClassName) throws KeyManagementException {
+        implementationClassName = CryptoUtils.handleLegacyPackages(implementationClassName);
+        return FileBasedKeyProvider.class.getName().equals(implementationClassName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/StaticKeyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/StaticKeyProvider.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/StaticKeyProvider.java
new file mode 100644
index 0000000..f14d124
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/kms/StaticKeyProvider.java
@@ -0,0 +1,96 @@
+/*
+ * 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.kms;
+
+import java.security.KeyManagementException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.crypto.SecretKey;
+import javax.naming.OperationNotSupportedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reference implementation for static key provider (used during tests).
+ */
+public class StaticKeyProvider implements KeyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
+
+    private Map<String, SecretKey> keys = new HashMap<>();
+
+    public StaticKeyProvider(String keyId, String keyHex) throws KeyManagementException {
+        this.keys.put(keyId, CryptoUtils.formKeyFromHex(keyHex));
+    }
+
+    public StaticKeyProvider(Map<String, SecretKey> keys) throws KeyManagementException {
+        this.keys.putAll(keys);
+    }
+
+    /**
+     * Returns the key identified by this ID or throws an exception if one is not available.
+     *
+     * @param keyId the key identifier
+     * @return the key
+     * @throws KeyManagementException if the key cannot be retrieved
+     */
+    @Override
+    public SecretKey getKey(String keyId) throws KeyManagementException {
+        logger.debug("Attempting to get key: " + keyId);
+        if (keyExists(keyId)) {
+            return keys.get(keyId);
+        } else {
+            throw new KeyManagementException("No key available for " + keyId);
+        }
+    }
+
+    /**
+     * Returns true if the key exists and is available. Null or empty IDs will return false.
+     *
+     * @param keyId the key identifier
+     * @return true if the key can be used
+     */
+    @Override
+    public boolean keyExists(String keyId) {
+        return keys.containsKey(keyId);
+    }
+
+    /**
+     * Returns a singleton list of the available key identifier.
+     *
+     * @return a List containing the {@code KEY_ID}
+     */
+    @Override
+    public List<String> getAvailableKeyIds() {
+        return new ArrayList<>(keys.keySet());
+    }
+
+    /**
+     * Adds the key to the provider and associates it with the given ID. Some implementations may not allow this operation.
+     *
+     * @param keyId the key identifier
+     * @param key   the key
+     * @return true if the key was successfully added
+     * @throws OperationNotSupportedException if this implementation doesn't support adding keys
+     * @throws KeyManagementException         if the key is invalid, the ID conflicts, etc.
+     */
+    @Override
+    public boolean addKey(String keyId, SecretKey key) throws OperationNotSupportedException, KeyManagementException {
+        throw new OperationNotSupportedException("This implementation does not allow adding keys");
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/675d9890/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 6ba8056..2bf952b 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.security.util.crypto;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -32,7 +33,6 @@ import org.apache.commons.codec.binary.Base64;
 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.ByteArrayOutputStream;
 import org.apache.nifi.stream.io.StreamUtils;
 
 public class CipherUtility {
@@ -316,13 +316,4 @@ public class CipherUtility {
             return -1;
         }
     }
-
-    public static byte[] concatBytes(byte[]... arrays) throws IOException {
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        for (byte[] bytes : arrays) {
-            outputStream.write(bytes);
-        }
-
-        return outputStream.toByteArray();
-    }
 }
\ No newline at end of file