You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2018/08/22 17:56:46 UTC

nifi git commit: NIFI-5540 Added unit test to demonstrate missing default sensitive properties key in flow election encryptor creation.

Repository: nifi
Updated Branches:
  refs/heads/master 5b57935ba -> 744b15b4a


NIFI-5540 Added unit test to demonstrate missing default sensitive properties key in flow election encryptor creation.

NIFI-5540 Added failing unit test and ignored regression test to demonstrate missing default sensitive properties key in flow election encryptor creation.

NIFI-5540 Added equality logic to StringEncryptor and utility equality methods to CryptoUtils.

NIFI-5540 Added default sensitive properties key population logic and log warning to StringEncryptor.

NIFI-5540 Cleaned up formatting.

NIFI-5540 Cleaned up boolean logic.

NIFI-5540 Added Javadoc to StringEncryptor.

NIFI-5540 Added unit test for StringEncryptor#equals().

NIFI-5540 Added performance benchmarking unit tests for constantTimeEquals methods for String, byte[], and char[].

NIFI-5540 Fixed checkstyle issue.

NIFI-5540 Fixed unit tests for default key population.

This closes #2959.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/master
Commit: 744b15b4a7a7533ef81fc2333df0cd212c3779eb
Parents: 5b57935
Author: Andy LoPresto <al...@apache.org>
Authored: Mon Aug 20 15:24:39 2018 -0700
Committer: Bryan Bende <bb...@apache.org>
Committed: Wed Aug 22 13:56:16 2018 -0400

----------------------------------------------------------------------
 .../apache/nifi/security/kms/CryptoUtils.java   |  65 ++++++
 .../nifi/security/kms/CryptoUtilsTest.groovy    | 197 +++++++++++++++++++
 .../PopularVoteFlowElectionFactoryBean.java     |   2 +-
 ...opularVoteFlowElectionFactoryBeanTest.groovy | 113 +++++++++++
 .../apache/nifi/encrypt/StringEncryptor.java    | 133 ++++++++++++-
 .../nifi/encrypt/StringEncryptorTest.groovy     | 134 ++++++++++++-
 6 files changed, 637 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/744b15b4/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
index 44b9787..657aaed 100644
--- 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
@@ -20,7 +20,12 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.security.KeyManagementException;
+import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
 import java.util.Base64;
@@ -285,4 +290,64 @@ public class CryptoUtils {
 
         return encryptedRepo && keyProviderConfigured;
     }
+
+    /**
+     * Returns true if the two parameters are equal. This method is null-safe and evaluates the
+     * equality in constant-time rather than "short-circuiting" on the first inequality. This
+     * prevents timing attacks (side channel attacks) when comparing passwords or hash values.
+     *
+     * @param a a String to compare
+     * @param b a String to compare
+     * @return true if the values are equal
+     */
+    public static boolean constantTimeEquals(String a, String b) {
+        if (a == null) {
+            return b == null;
+        } else {
+            // This returns true IFF b != null and the byte[] are equal; if b == null, a is not, and they are not equal
+            return b != null && constantTimeEquals(a.getBytes(StandardCharsets.UTF_8), b.getBytes(StandardCharsets.UTF_8));
+        }
+    }
+
+    /**
+     * Returns true if the two parameters are equal. This method is null-safe and evaluates the
+     * equality in constant-time rather than "short-circuiting" on the first inequality. This
+     * prevents timing attacks (side channel attacks) when comparing passwords or hash values.
+     * Does not convert the character arrays to {@code String}s when converting to {@code byte[]}
+     * to avoid putting sensitive data in the String pool.
+     *
+     * @param a a char[] to compare
+     * @param b a char[] to compare
+     * @return true if the values are equal
+     */
+    public static boolean constantTimeEquals(char[] a, char[] b) {
+        return constantTimeEquals(convertCharsToBytes(a), convertCharsToBytes(b));
+    }
+
+
+    /**
+     * Returns true if the two parameters are equal. This method is null-safe and evaluates the
+     * equality in constant-time rather than "short-circuiting" on the first inequality. This
+     * prevents timing attacks (side channel attacks) when comparing passwords or hash values.
+     *
+     * @param a a byte[] to compare
+     * @param b a byte[] to compare
+     * @return true if the values are equal
+     */
+    public static boolean constantTimeEquals(byte[] a, byte[] b) {
+        return MessageDigest.isEqual(a, b);
+    }
+
+    /**
+     * Returns a {@code byte[]} containing the value of the provided {@code char[]} without using {@code new String(chars).getBytes()} which would put sensitive data (the password) in the String pool.
+     *
+     * @param chars the characters to convert
+     * @return the byte[]
+     */
+    private static byte[] convertCharsToBytes(char[] chars) {
+        CharBuffer charBuffer = CharBuffer.wrap(chars);
+        ByteBuffer byteBuffer = Charset.forName("UTF-8").encode(charBuffer);
+        return Arrays.copyOfRange(byteBuffer.array(),
+                byteBuffer.position(), byteBuffer.limit());
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/744b15b4/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
index 954b7ca..b8d7b5b 100644
--- 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
@@ -488,6 +488,203 @@ class CryptoUtilsTest {
         assert emptyMsg.getMessage() == "The key provider file is not present and readable"
     }
 
+    @Test
+    void testShouldEvaluateConstantTimeEqualsForStrings() {
+        // Arrange
+        String plaintext = "This is a short string."
+        String firstCharOff = "this is a short string."
+        String lastCharOff = "This is a short string,"
+
+        final int ITERATIONS = 10_000
+        final int WARM_UP_ITERATIONS = 1_000 * ITERATIONS
+
+        def scenarios = ["identical": plaintext, "first off": firstCharOff, "last off": lastCharOff]
+        def results = [:]
+        def timings = [:]
+
+        boolean isEqual = true
+        long nanos = 0
+        long scNanos = 0
+
+        // Prepare the JVM
+        (WARM_UP_ITERATIONS).times { int i ->
+            def scIterationNanos = time("warm up sc") {
+                assert plaintext == plaintext
+            }
+            scNanos += scIterationNanos
+            def iterationNanos = time("warm up") {
+                assert CryptoUtils.constantTimeEquals(plaintext, plaintext)
+            }
+            nanos += iterationNanos
+        }
+        logger.info("${"warm up sc".padLeft(10)}: ${nanos} ns (avg: ${nanos / (WARM_UP_ITERATIONS)} ns)")
+        logger.info("${"warm up".padLeft(10)}: ${scNanos} ns (avg: ${scNanos / (WARM_UP_ITERATIONS)} ns)")
+
+        // Act
+        scenarios.each { String scenario, String value ->
+            isEqual = true
+            scNanos = 0
+            nanos = 0
+            ITERATIONS.times { int i ->
+                def scIterationNanos = time(scenario + " sc") {
+                    (plaintext == value)
+                }
+                scNanos += scIterationNanos
+                def iterationNanos = time(scenario) {
+                    isEqual = CryptoUtils.constantTimeEquals(plaintext, value)
+                }
+                nanos += iterationNanos
+            }
+            def scenarioWidth = 16
+            logger.info("${(scenario + " sc").padLeft(scenarioWidth)}: ${scNanos} ns (avg: ${scNanos / ITERATIONS} ns)")
+            logger.info("${scenario.padLeft(scenarioWidth)}: ${nanos} ns (avg: ${nanos / ITERATIONS} ns)")
+            results[scenario] = isEqual
+            timings[scenario] = nanos
+        }
+
+        // Assert
+        assert results["identical"]
+        assert !results["first off"]
+        assert !results["last off"]
+
+        // TODO: Assert timings are within std dev?
+    }
+
+    @Test
+    void testShouldEvaluateConstantTimeEqualsForBytes() {
+        // Arrange
+        String plaintext = "This is a short string."
+        String firstCharOff = "this is a short string."
+        String lastCharOff = "This is a short string,"
+
+        final int ITERATIONS = 10_000
+        final int WARM_UP_ITERATIONS = 1_000 * ITERATIONS
+
+        def scenarios = ["identical": plaintext, "first off": firstCharOff, "last off": lastCharOff]
+        def results = [:]
+        def timings = [:]
+
+        boolean isEqual = true
+        long nanos = 0
+        long scNanos = 0
+
+        // Prepare the JVM
+        byte[] plaintextBytes = plaintext.getBytes("UTF-8")
+        (WARM_UP_ITERATIONS).times { int i ->
+            def scIterationNanos = time("warm up sc") {
+                assert plaintext == plaintext
+            }
+            scNanos += scIterationNanos
+            def iterationNanos = time("warm up") {
+                assert CryptoUtils.constantTimeEquals(plaintextBytes, plaintextBytes)
+            }
+            nanos += iterationNanos
+        }
+        logger.info("${"warm up sc".padLeft(10)}: ${nanos} ns (avg: ${nanos / (WARM_UP_ITERATIONS)} ns)")
+        logger.info("${"warm up".padLeft(10)}: ${scNanos} ns (avg: ${scNanos / (WARM_UP_ITERATIONS)} ns)")
+
+        // Act
+        scenarios.each { String scenario, String value ->
+            isEqual = true
+            scNanos = 0
+            nanos = 0
+            byte[] valueBytes = value.getBytes("UTF-8")
+            ITERATIONS.times { int i ->
+                def scIterationNanos = time(scenario + " sc") {
+                    (plaintextBytes == valueBytes)
+                }
+                scNanos += scIterationNanos
+                def iterationNanos = time(scenario) {
+                    isEqual = CryptoUtils.constantTimeEquals(plaintextBytes, valueBytes)
+                }
+                nanos += iterationNanos
+            }
+            def scenarioWidth = 16
+            logger.info("${(scenario + " sc").padLeft(scenarioWidth)}: ${scNanos} ns (avg: ${scNanos / ITERATIONS} ns)")
+            logger.info("${scenario.padLeft(scenarioWidth)}: ${nanos} ns (avg: ${nanos / ITERATIONS} ns)")
+            results[scenario] = isEqual
+            timings[scenario] = nanos
+        }
+
+        // Assert
+        assert results["identical"]
+        assert !results["first off"]
+        assert !results["last off"]
+
+        // TODO: Assert timings are within std dev?
+    }
+
+    @Test
+    void testShouldEvaluateConstantTimeEqualsForChars() {
+        // Arrange
+        String plaintext = "This is a short string."
+        String firstCharOff = "this is a short string."
+        String lastCharOff = "This is a short string,"
+
+        final int ITERATIONS = 10_000
+        final int WARM_UP_ITERATIONS = 1_000 * ITERATIONS
+
+        def scenarios = ["identical": plaintext, "first off": firstCharOff, "last off": lastCharOff]
+        def results = [:]
+        def timings = [:]
+
+        boolean isEqual = true
+        long nanos = 0
+        long scNanos = 0
+
+        // Prepare the JVM
+        def plaintextChars = plaintext.chars
+        (WARM_UP_ITERATIONS).times { int i ->
+            def scIterationNanos = time("warm up sc") {
+                assert plaintext == plaintext
+            }
+            scNanos += scIterationNanos
+            def iterationNanos = time("warm up") {
+                assert CryptoUtils.constantTimeEquals(plaintextChars, plaintextChars)
+            }
+            nanos += iterationNanos
+        }
+        logger.info("${"warm up sc".padLeft(10)}: ${nanos} ns (avg: ${nanos / (WARM_UP_ITERATIONS)} ns)")
+        logger.info("${"warm up".padLeft(10)}: ${scNanos} ns (avg: ${scNanos / (WARM_UP_ITERATIONS)} ns)")
+
+        // Act
+        scenarios.each { String scenario, String value ->
+            isEqual = true
+            scNanos = 0
+            nanos = 0
+            def valueChars = value.chars
+            ITERATIONS.times { int i ->
+                def scIterationNanos = time(scenario + " sc") {
+                    (plaintextChars == valueChars)
+                }
+                scNanos += scIterationNanos
+                def iterationNanos = time(scenario) {
+                    isEqual = CryptoUtils.constantTimeEquals(plaintextChars, valueChars)
+                }
+                nanos += iterationNanos
+            }
+            def scenarioWidth = 16
+            logger.info("${(scenario + " sc").padLeft(scenarioWidth)}: ${scNanos} ns (avg: ${scNanos / ITERATIONS} ns)")
+            logger.info("${scenario.padLeft(scenarioWidth)}: ${nanos} ns (avg: ${nanos / ITERATIONS} ns)")
+            results[scenario] = isEqual
+            timings[scenario] = nanos
+        }
+
+        // Assert
+        assert results["identical"]
+        assert !results["first off"]
+        assert !results["last off"]
+
+        // TODO: Assert timings are within std dev?
+    }
+
+    private static long time(String name = "closure", Closure closure) {
+        long start = System.nanoTime()
+        closure.run()
+        long end = System.nanoTime()
+        end - start
+    }
+
     private static String generateEncryptedKey(SecretKey masterKey) {
         byte[] ivBytes = new byte[16]
         byte[] keyBytes = new byte[isUnlimitedStrengthCryptoAvailable() ? 32 : 16]

http://git-wip-us.apache.org/repos/asf/nifi/blob/744b15b4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
index 641d767..4ea4225 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
@@ -31,7 +31,7 @@ public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVo
     private NiFiProperties properties;
 
     @Override
-    public PopularVoteFlowElection getObject() throws Exception {
+    public PopularVoteFlowElection getObject() {
         final String maxWaitTime = properties.getFlowElectionMaxWaitTime();
         long maxWaitMillis;
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/744b15b4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
new file mode 100644
index 0000000..9fc6372
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
@@ -0,0 +1,113 @@
+/*
+ * 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.cluster.coordination.flow
+
+import org.apache.nifi.encrypt.StringEncryptor
+import org.apache.nifi.security.util.EncryptionMethod
+import org.apache.nifi.util.NiFiProperties
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+@RunWith(JUnit4.class)
+class PopularVoteFlowElectionFactoryBeanTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(PopularVoteFlowElectionFactoryBeanTest.class)
+    private final String DEFAULT_SENSITIVE_PROPS_KEY = "nififtw!"
+
+    @BeforeClass
+    static void setUpOnce() {
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() {
+        super.setUp()
+
+    }
+
+    @After
+    void tearDown() {
+
+    }
+
+    NiFiProperties mockProperties(Map<String, String> defaults = [:]) {
+        def mockProps = NiFiProperties.createBasicNiFiProperties(null, [
+                (NiFiProperties.SENSITIVE_PROPS_ALGORITHM):EncryptionMethod.MD5_256AES.algorithm,
+                (NiFiProperties.SENSITIVE_PROPS_PROVIDER):EncryptionMethod.MD5_256AES.provider,
+        ] + defaults)
+
+        mockProps
+    }
+
+    @Test
+    void testGetObjectShouldPopulateDefaultSensitivePropsKeyIfEmpty() {
+        // Arrange
+        PopularVoteFlowElectionFactoryBean electionFactoryBean = new PopularVoteFlowElectionFactoryBean()
+        electionFactoryBean.properties = mockProperties()
+
+        final StringEncryptor DEFAULT_ENCRYPTOR = new StringEncryptor(EncryptionMethod.MD5_256AES.algorithm, EncryptionMethod.MD5_256AES.provider, DEFAULT_SENSITIVE_PROPS_KEY)
+        final String EXPECTED_PLAINTEXT = "my.test.value"
+        final String EXPECTED_CIPHERTEXT = DEFAULT_ENCRYPTOR.encrypt(EXPECTED_PLAINTEXT)
+        logger.info("Expected ciphertext: ${EXPECTED_CIPHERTEXT}")
+
+        // Act
+        PopularVoteFlowElection election = electionFactoryBean.object
+        logger.info("Got object: ${election}")
+
+        // Assert
+
+        // Violates LoD but need to evaluate nested encryptor can decrypt
+        def encryptor = election.fingerprintFactory.encryptor
+        String decrypted = encryptor.decrypt(EXPECTED_CIPHERTEXT)
+        logger.info("Decrypted plain text: ${decrypted}")
+        assert decrypted == EXPECTED_PLAINTEXT
+    }
+
+    @Test
+    void testGetObjectShouldPopulateSensitivePropsKeyIfPresent() {
+        // Arrange
+        final String REVERSE_KEY = DEFAULT_SENSITIVE_PROPS_KEY.reverse()
+
+        PopularVoteFlowElectionFactoryBean electionFactoryBean = new PopularVoteFlowElectionFactoryBean()
+        electionFactoryBean.properties = mockProperties([(NiFiProperties.SENSITIVE_PROPS_KEY): REVERSE_KEY])
+
+        final StringEncryptor REVERSE_ENCRYPTOR = new StringEncryptor(EncryptionMethod.MD5_256AES.algorithm, EncryptionMethod.MD5_256AES.provider, REVERSE_KEY)
+        final String EXPECTED_PLAINTEXT = "my.test.value"
+        final String EXPECTED_CIPHERTEXT = REVERSE_ENCRYPTOR.encrypt(EXPECTED_PLAINTEXT)
+        logger.info("Expected ciphertext: ${EXPECTED_CIPHERTEXT}")
+
+        // Act
+        PopularVoteFlowElection election = electionFactoryBean.object
+        logger.info("Got object: ${election}")
+
+        // Assert
+
+        // Violates LoD but need to evaluate nested encryptor can decrypt
+        def encryptor = election.fingerprintFactory.encryptor
+        String decrypted = encryptor.decrypt(EXPECTED_CIPHERTEXT)
+        logger.info("Decrypted plain text: ${decrypted}")
+        assert decrypted == EXPECTED_PLAINTEXT
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/744b15b4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
index e8d7509..15750f5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
@@ -23,6 +23,7 @@ import java.security.Security;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Objects;
 import javax.crypto.Cipher;
 import javax.crypto.spec.PBEKeySpec;
 import javax.crypto.spec.SecretKeySpec;
@@ -185,13 +186,18 @@ public class StringEncryptor {
 
         final String sensitivePropAlgorithmVal = niFiProperties.getProperty(NF_SENSITIVE_PROPS_ALGORITHM);
         final String sensitivePropProviderVal = niFiProperties.getProperty(NF_SENSITIVE_PROPS_PROVIDER);
-        final String sensitivePropValueNifiPropVar = niFiProperties.getProperty(NF_SENSITIVE_PROPS_KEY, DEFAULT_SENSITIVE_PROPS_KEY);
+        String sensitivePropValueNifiPropVar = niFiProperties.getProperty(NF_SENSITIVE_PROPS_KEY);
+        // TODO: This method should be removed in 2.0.0 and replaced globally with the String, String, String method
+        if (StringUtils.isBlank(sensitivePropValueNifiPropVar)) {
+            printBlankKeyWarning();
+            sensitivePropValueNifiPropVar = DEFAULT_SENSITIVE_PROPS_KEY;
+        }
 
         return createEncryptor(sensitivePropAlgorithmVal, sensitivePropProviderVal, sensitivePropValueNifiPropVar);
     }
 
     /**
-     * Creates an instance of the NiFi sensitive property encryptor.
+     * Creates an instance of the NiFi sensitive property encryptor. If the password is blank, the default will be used and an error will be printed to the log.
      *
      * @param algorithm the encryption (and key derivation) algorithm ({@link EncryptionMethod#algorithm})
      * @param provider  the JCA Security provider ({@link EncryptionMethod#provider})
@@ -207,13 +213,31 @@ public class StringEncryptor {
             throw new EncryptionException(NF_SENSITIVE_PROPS_PROVIDER + " must be set");
         }
 
+        // Can't throw an exception because users who have not populated a key expect fallback to default.
+        // TODO: This should be removed in 2.0.0 and replaced with strict enforcement of a explicit unique key
         if (StringUtils.isBlank(password)) {
-            throw new EncryptionException(NF_SENSITIVE_PROPS_KEY + " must be set");
+            printBlankKeyWarning();
+            password = DEFAULT_SENSITIVE_PROPS_KEY;
         }
 
         return new StringEncryptor(algorithm, provider, password);
     }
 
+    private static void printBlankKeyWarning() {
+        logger.error(StringUtils.repeat("*", 80));
+        logger.error(centerString("A blank sensitive properties key was provided"));
+        logger.error(centerString("Specify a unique key in nifi.properties"));
+        logger.error(centerString("for nifi.sensitive.props.key"));
+        logger.error(centerString(""));
+        logger.error(centerString("The Encrypt Config Tool in NiFi Toolkit can be used to"));
+        logger.error(centerString("migrate the flow to the new key"));
+        logger.error(StringUtils.repeat("*", 80));
+    }
+
+    private static String centerString(String msg) {
+        return "*" + StringUtils.center(msg, 78, " ") + "*";
+    }
+
     protected void initialize() {
         if (isInitialized()) {
             logger.debug("Attempted to initialize an already-initialized StringEncryptor");
@@ -249,7 +273,7 @@ public class StringEncryptor {
 
     private boolean passwordIsValid(PBEKeySpec password) {
         try {
-            return password.getPassword() != null;
+            return password.getPassword().length > 0;
         } catch (IllegalStateException | NullPointerException e) {
             return false;
         }
@@ -382,7 +406,7 @@ public class StringEncryptor {
         }
     }
 
-    private byte[] decryptPBE(byte[] cipherBytes) throws DecoderException {
+    private byte[] decryptPBE(byte[] cipherBytes) {
         PBECipherProvider pbecp = (PBECipherProvider) cipherProvider;
         final EncryptionMethod encryptionMethod = EncryptionMethod.forAlgorithm(algorithm);
 
@@ -452,4 +476,103 @@ public class StringEncryptor {
     protected static boolean providerIsValid(String provider) {
         return SUPPORTED_PROVIDERS.contains(provider);
     }
+
+    /**
+     * Returns {@code true} if the two {@code StringEncryptor} objects are logically equivalent.
+     * This requires the same {@code algorithm}, {@code provider}, {@code encoding}, and
+     * {@code key}/{@code password}.
+     * <p>
+     * A {@code ciphertext} generated by one object can be decrypted by a separate object if they are equal as determined by this method.
+     *
+     * @param o the other StringEncryptor
+     * @return true if these instances are equal
+     */
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        StringEncryptor that = (StringEncryptor) o;
+        return Objects.equals(algorithm, that.algorithm)
+                && Objects.equals(provider, that.provider)
+                && Objects.equals(encoding, that.encoding)
+                && secretsAreEqual(that.password, that.key);
+    }
+
+    /**
+     * Returns true if the provided {@code password} and {@code key} match those contained in this {@code StringEncryptor}. This method does <strong>not</strong> compare {@code password == key}.
+     * <p>
+     * Internally, uses {@link #isPBEKeySpecEqual(PBEKeySpec, PBEKeySpec)} and {@link SecretKeySpec#equals(Object)}.
+     *
+     * @param otherPassword the password {@link PBEKeySpec}
+     * @param otherKey      the key {@link SecretKeySpec}
+     * @return true if the passwords match and the keys match
+     */
+    private boolean secretsAreEqual(PBEKeySpec otherPassword, SecretKeySpec otherKey) {
+        // SecretKeySpec implements null-safe equals(), but PBEKeySpec does not
+        return isPBEKeySpecEqual(this.password, otherPassword) && Objects.equals(this.key, otherKey);
+    }
+
+    /**
+     * Returns true if the two {@link PBEKeySpec} objects are logically equivalent (same params and password).
+     *
+     * @param a a PBEKeySpec to compare
+     * @param b a PBEKeySpec to compare
+     * @return true if they can be used for encryption interchangeably
+     */
+    private static boolean isPBEKeySpecEqual(PBEKeySpec a, PBEKeySpec b) {
+        if (a != null) {
+            if (b == null) {
+                return false;
+            } else {
+                // Compare all the accessors that will not throw exceptions
+                boolean nonNullsEqual = a.getIterationCount() == b.getIterationCount()
+                        && a.getKeyLength() == b.getKeyLength()
+                        && Arrays.equals(a.getSalt(), b.getSalt());
+
+                // Compare the passwords using constant-time equality while catching exceptions
+                boolean passwordsEqual;
+                try {
+                    passwordsEqual = CryptoUtils.constantTimeEquals(a.getPassword(), b.getPassword());
+                } catch (IllegalStateException e) {
+                    logger.warn("Encountered an error trying to compare password equality (one or more passwords have been cleared)");
+                    // Assume any key spec with password cleared is unusable; return false
+                    return false;
+                }
+
+                // Logging for debug assistance
+                if (logger.isDebugEnabled()) {
+                    logger.debug("The PBEKeySpec objects have equal non-null elements ({}) and equal passwords ({})", new Object[]{String.valueOf(nonNullsEqual), String.valueOf(passwordsEqual)});
+                }
+                return nonNullsEqual && passwordsEqual;
+            }
+        } else {
+            // If here, a == null
+            return b == null;
+        }
+    }
+
+
+    /**
+     * Returns the hashcode of this object. Does not include {@code cipherProvider} in hashcode calculations.
+     *
+     * @return the hashcode
+     */
+    @Override
+    public int hashCode() {
+        return Objects.hash(algorithm, provider, encoding, password, key);
+    }
+
+    /**
+     * Returns a String containing the {@code algorithm}, {@code provider}, {@code encoding}, and {@code cipherProvider} class name.
+     *
+     * @return a String representation of the object state
+     */
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder("StringEncryptor using ").append(algorithm)
+                .append(" from ").append(provider)
+                .append(" with ").append(encoding).append(" encoding and cipher provider ")
+                .append(cipherProvider.getClass().getName());
+        return sb.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/744b15b4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/encrypt/StringEncryptorTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/encrypt/StringEncryptorTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/encrypt/StringEncryptorTest.groovy
index 37e0302..c07e0ec 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/encrypt/StringEncryptorTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/encrypt/StringEncryptorTest.groovy
@@ -32,6 +32,7 @@ import org.junit.After
 import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
+import org.junit.Ignore
 import org.junit.Test
 import org.junit.runner.RunWith
 import org.junit.runners.JUnit4
@@ -88,7 +89,7 @@ class StringEncryptorTest {
     static void setUpOnce() throws Exception {
         Security.addProvider(new BouncyCastleProvider())
 
-        limitedPbeEncryptionMethods.removeAll { it.algorithm =~ "SHA.*(CBC)?"}
+        limitedPbeEncryptionMethods.removeAll { it.algorithm =~ "SHA.*(CBC)?" }
 
         logger.metaClass.methodMissing = { String name, args ->
             logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
@@ -479,4 +480,135 @@ class StringEncryptorTest {
         assert !uninitializedIsInitialized
         assert initializedIsInitialized
     }
+
+    /**
+     * Checks the {@link StringEncryptor#createEncryptor(NiFiProperties)} method which injects a default {@code nifi.sensitive.props.key} if one is not provided.
+     *
+     * @throws Exception
+     */
+    @Test
+    void testNiFiPropertiesCreateEncryptorShouldPopulateDefaultKeyIfMissing() throws Exception {
+        // Arrange
+        NiFiProperties propertiesWithoutKey = new StandardNiFiProperties([(ALGORITHM): DEFAULT_ALGORITHM, (PROVIDER): DEFAULT_PROVIDER] as Properties)
+
+        final StringEncryptor DEFAULT_ENCRYPTOR = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        logger.info("Created encryptor from constructor using default values: ${DEFAULT_ENCRYPTOR}")
+
+        // Act
+        StringEncryptor propertiesEncryptor = StringEncryptor.createEncryptor(propertiesWithoutKey)
+        logger.info("Created encryptor from NiFiProperties: ${propertiesEncryptor}")
+
+        // Assert
+        assert propertiesEncryptor == DEFAULT_ENCRYPTOR
+    }
+
+    /**
+     * Checks the {@link StringEncryptor#createEncryptor(String, String, String)} method which throws an exception if {@code nifi.sensitive.props.key} is not provided.
+     *
+     * @throws Exception
+     */
+    @Ignore("Regression test for old behavior")
+    @Test
+    void testStringCreateEncryptorShouldRequireKey() throws Exception {
+        // Arrange
+        final StringEncryptor DEFAULT_ENCRYPTOR = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        logger.info("Created encryptor from constructor using default values: ${DEFAULT_ENCRYPTOR}")
+
+        // Act
+        def constructMsg = shouldFail(EncryptionException) {
+            StringEncryptor stringEncryptor = StringEncryptor.createEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, "")
+        }
+        logger.expected(constructMsg)
+
+        // Assert
+        assert constructMsg =~ "key must be set"
+    }
+
+    /**
+     * Checks the {@link StringEncryptor#createEncryptor(String, String, String)} method which injects a default {@code nifi.sensitive.props.key} if one is not provided.
+     *
+     * @throws Exception
+     */
+    @Test
+    void testStringCreateEncryptorShouldPopulateDefaultKeyIfMissing() throws Exception {
+        // Arrange
+        final StringEncryptor DEFAULT_ENCRYPTOR = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        logger.info("Created encryptor from constructor using default values: ${DEFAULT_ENCRYPTOR}")
+
+        // Act
+        StringEncryptor propertiesEncryptor = StringEncryptor.createEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, "")
+        logger.info("Created encryptor from NiFiProperties: ${propertiesEncryptor}")
+
+        // Assert
+        assert propertiesEncryptor == DEFAULT_ENCRYPTOR
+    }
+
+    @Test
+    void testEquals() throws Exception {
+        // Arrange
+        final StringEncryptor DEFAULT_ENCRYPTOR = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        logger.info("Created encryptor from constructor using default values: ${DEFAULT_ENCRYPTOR}")
+
+        StringEncryptor identityEncryptor = DEFAULT_ENCRYPTOR
+        logger.info("Created encryptor by assigning identity: ${identityEncryptor}")
+
+        StringEncryptor sameValueEncryptor = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        logger.info("Created encryptor from constructor using same values: ${DEFAULT_ENCRYPTOR}")
+
+        // Clone not supported
+//        StringEncryptor cloneEncryptor = DEFAULT_ENCRYPTOR.clone()
+//        logger.info("Created encryptor from cloning default: ${cloneEncryptor}")
+
+        StringEncryptor base64Encryptor = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        base64Encryptor.setEncoding("BASE64")
+        logger.info("Created encryptor with Base64 encoding: ${base64Encryptor}")
+
+        StringEncryptor algorithmEncryptor = new StringEncryptor("PBEWITHSHAAND128BITAES-CBC-BC", DEFAULT_PROVIDER, DEFAULT_PASSWORD)
+        logger.info("Created encryptor with ${algorithmEncryptor.algorithm} algorithm: ${algorithmEncryptor}")
+
+        StringEncryptor providerEncryptor = new StringEncryptor(DEFAULT_ALGORITHM, "SunJCE", DEFAULT_PASSWORD)
+        logger.info("Created encryptor with ${providerEncryptor.provider} provider: ${providerEncryptor}")
+
+        StringEncryptor passwordEncryptor = new StringEncryptor(DEFAULT_ALGORITHM, DEFAULT_PROVIDER, DEFAULT_PASSWORD.reverse())
+        logger.info("Created encryptor with ${DEFAULT_PASSWORD.reverse()} password: ${passwordEncryptor}")
+        
+        // Act
+        boolean defaultIsEqual = DEFAULT_ENCRYPTOR.equals(DEFAULT_ENCRYPTOR)
+        logger.info("[${defaultIsEqual.toString().padLeft(5)}]: default == default")
+
+        boolean identityIsEqual = DEFAULT_ENCRYPTOR.equals(identityEncryptor)
+        logger.info("[${identityIsEqual.toString().padLeft(5)}]: default == identity")
+
+        boolean sameValueIsEqual = DEFAULT_ENCRYPTOR.equals(sameValueEncryptor)
+        logger.info("[${sameValueIsEqual.toString().padLeft(5)}]: default == same value")
+        
+//        boolean cloneIsEqual = DEFAULT_ENCRYPTOR.equals(cloneEncryptor)
+//        logger.info("[${cloneIsEqual.toString().padLeft(5)}]: ${DEFAULT_ENCRYPTOR} | ${cloneEncryptor}")
+
+        // Should be unequal
+
+        boolean base64IsEqual = DEFAULT_ENCRYPTOR.equals(base64Encryptor)
+        logger.info("[${base64IsEqual.toString().padLeft(5)}]: default == base64")
+       
+        boolean algorithmIsEqual = DEFAULT_ENCRYPTOR.equals(algorithmEncryptor)
+        logger.info("[${algorithmIsEqual.toString().padLeft(5)}]: default == algorithm")
+       
+        boolean providerIsEqual = DEFAULT_ENCRYPTOR.equals(providerEncryptor)
+        logger.info("[${providerIsEqual.toString().padLeft(5)}]: default == provider")
+       
+        boolean passwordIsEqual = DEFAULT_ENCRYPTOR.equals(passwordEncryptor)
+        logger.info("[${passwordIsEqual.toString().padLeft(5)}]: default == password")
+       
+        
+        // Assert
+        assert defaultIsEqual
+        assert identityIsEqual
+        assert sameValueIsEqual
+//        assert cloneIsEqual
+
+        assert !base64IsEqual
+        assert !algorithmIsEqual
+        assert !providerIsEqual
+        assert !passwordIsEqual
+    }
 }