You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2020/04/28 00:30:27 UTC

[nifi] branch master updated: NIFI-7377 Cleaned up nifi-stateless logs. Refactored masking logic to CipherUtility and indicated masking with label and Base64 output. Added JSON masking logic to nifi-stateless module. Added argument masking functionality to Program. Moved groovy unit tests to proper Maven directory structure. Modified plain argument output to use filtering/masking methods in provided utility. Refactored utility methods. Updated unit tests.

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

alopresto pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 148537d  NIFI-7377 Cleaned up nifi-stateless logs. Refactored masking logic to CipherUtility and indicated masking with label and Base64 output. Added JSON masking logic to nifi-stateless module. Added argument masking functionality to Program. Moved groovy unit tests to proper Maven directory structure. Modified plain argument output to use filtering/masking methods in provided utility. Refactored utility methods. Updated unit tests.
148537d is described below

commit 148537d64a017b73160b0d49943183c18f883ab0
Author: Andy LoPresto <al...@apache.org>
AuthorDate: Mon Apr 20 20:36:45 2020 +0200

    NIFI-7377 Cleaned up nifi-stateless logs.
    Refactored masking logic to CipherUtility and indicated masking with label and Base64 output.
    Added JSON masking logic to nifi-stateless module.
    Added argument masking functionality to Program.
    Moved groovy unit tests to proper Maven directory structure.
    Modified plain argument output to use filtering/masking methods in provided utility.
    Refactored utility methods.
    Updated unit tests.
    
    This closes #4222.
    
    Co-authored-by: Pierre Villard <pi...@gmail.com>
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 .../nifi/security/util/crypto/CipherUtility.java   |  36 ++++
 .../nifi/fingerprint/FingerprintFactory.java       |  41 ++--
 .../FingerprintFactoryGroovyTest.groovy            |   3 +
 .../nifi/fingerprint/FingerprintFactoryTest.java   |   2 +-
 .../nifi-framework/nifi-stateless/pom.xml          |  12 +-
 .../apache/nifi/stateless/core/StatelessFlow.java  |  65 +++---
 .../nifi/stateless/core/StatelessFlowFile.java     |  16 +-
 .../core/security/StatelessSecurityUtility.java    | 174 ++++++++++++++++
 .../apache/nifi/stateless/runtimes/Program.java    |  80 ++++++--
 .../openwhisk/StatelessNiFiOpenWhiskAction.java    |  17 +-
 .../stateless/runtimes/yarn/YARNServiceUtil.java   |  10 +-
 .../security/StatelessSecurityUtilityTest.groovy   | 222 +++++++++++++++++++++
 .../nifi/stateless/runtimes/ProgramTest.groovy     | 101 ++++++++++
 13 files changed, 679 insertions(+), 100 deletions(-)

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 369b015..5c1eb27 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
@@ -406,4 +406,40 @@ public class CipherUtility {
         }
         return saltLength;
     }
+
+    /**
+     * Returns a securely-derived, deterministic value from the provided plaintext property
+     * value. This is because sensitive values should not be disclosed through the
+     * logs. However, the equality or difference of the sensitive value can be important, so it cannot be ignored completely.
+     *
+     * The specific derivation process is unimportant as long as it is a salted,
+     * cryptographically-secure hash function with an iteration cost sufficient for password
+     * storage in other applications.
+     *
+     * @param sensitivePropertyValue the plaintext property value
+     * @return a deterministic string value which represents this input but is safe to print in a log
+     */
+    public static String getLoggableRepresentationOfSensitiveValue(String sensitivePropertyValue) {
+        // TODO: Use DI/IoC to inject this implementation in the constructor of the FingerprintFactory
+        // There is little initialization cost, so it doesn't make sense to cache this as a field
+        SecureHasher secureHasher = new Argon2SecureHasher();
+
+        // TODO: Extend {@link StringEncryptor} with secure hashing capability and inject?
+        return getLoggableRepresentationOfSensitiveValue(sensitivePropertyValue, secureHasher);
+    }
+
+    /**
+     * Returns a securely-derived, deterministic value from the provided plaintext property
+     * value. This is because sensitive values should not be disclosed through the
+     * logs. However, the equality or difference of the sensitive value can be important, so it cannot be ignored completely.
+     *
+     * The specific derivation process is determined by the provided {@link SecureHasher} implementation.
+     *
+     * @param sensitivePropertyValue the plaintext property value
+     * @param secureHasher an instance of {@link SecureHasher} which will be used to mask the value
+     * @return a deterministic string value which represents this input but is safe to print in a log
+     */
+    public static String getLoggableRepresentationOfSensitiveValue(String sensitivePropertyValue, SecureHasher secureHasher) {
+        return "[MASKED] (" + secureHasher.hashBase64(sensitivePropertyValue) + ")";
+    }
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index c8f9356..c3c95e6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -16,6 +16,21 @@
  */
 package org.apache.nifi.fingerprint;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.stream.Stream;
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
@@ -25,8 +40,7 @@ import org.apache.nifi.controller.serialization.FlowEncodingVersion;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.security.util.crypto.Argon2SecureHasher;
-import org.apache.nifi.security.util.crypto.SecureHasher;
+import org.apache.nifi.security.util.crypto.CipherUtility;
 import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.DomUtils;
 import org.apache.nifi.util.LoggingXmlParserErrorHandler;
@@ -41,22 +55,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.stream.Stream;
-
 /**
  * <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
  *
@@ -549,12 +547,7 @@ public class FingerprintFactory {
      * @return a deterministic string value which represents this input but is safe to print in a log
      */
     private String getLoggableRepresentationOfSensitiveValue(String encryptedPropertyValue) {
-        // TODO: Use DI/IoC to inject this implementation in the constructor of the FingerprintFactory
-        // There is little initialization cost, so it doesn't make sense to cache this as a field
-        SecureHasher secureHasher = new Argon2SecureHasher();
-
-        // TODO: Extend {@link StringEncryptor} with secure hashing capability and inject?
-        return secureHasher.hashHex(decrypt(encryptedPropertyValue));
+        return CipherUtility.getLoggableRepresentationOfSensitiveValue(decrypt(encryptedPropertyValue));
     }
 
     private StringBuilder addPortFingerprint(final StringBuilder builder, final Element portElem) throws FingerprintException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
index 969cad8..c16fe95 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
@@ -84,5 +84,8 @@ class FingerprintFactoryGroovyTest extends GroovyTestCase {
 
         // Assert the fingerprint does not contain the password
         assert !(fingerprint =~ "originalPlaintextPassword")
+        def maskedValue = (fingerprint =~ /\[MASKED\] \([\w\/\+=]+\)/)
+        assert maskedValue
+        logger.info("Masked value: ${maskedValue[0]}")
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
index 5baaf6d..b1317a3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
@@ -268,7 +268,7 @@ public class FingerprintFactoryTest {
         when(component.getVersionedComponentId()).thenReturn(Optional.empty());
 
         // Assert fingerprints with expected one.
-        final String hashedProxyPassword = new Argon2SecureHasher().hashHex(proxyPassword);
+        final String hashedProxyPassword = "[MASKED] (" + new Argon2SecureHasher().hashBase64(proxyPassword) + ")";
         final String expected = "id" +
                 "NO_VALUE" +
                 "http://node1:8080/nifi, http://node2:8080/nifi" +
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
index 2f5ca86..b331b39 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
@@ -52,6 +52,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.12.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
             <version>1.12.0-SNAPSHOT</version>
         </dependency>
@@ -68,7 +73,7 @@
         <dependency>
             <groupId>com.google.code.gson</groupId>
             <artifactId>gson</artifactId>
-            <version>2.7</version>
+            <version>2.8.2</version>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>
@@ -113,6 +118,11 @@
             <type>war</type>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.codehaus.groovy</groupId>
+            <artifactId>groovy-json</artifactId>
+            <version>${nifi.groovy.version}</version>
+        </dependency>
 
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java
index 8ff03a6..1329970 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java
@@ -19,6 +19,19 @@ package org.apache.nifi.stateless.core;
 import com.google.gson.JsonArray;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonObject;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javax.net.ssl.SSLContext;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.ControllerService;
@@ -45,20 +58,7 @@ import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.stateless.bootstrap.ExtensionDiscovery;
 import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
 import org.apache.nifi.stateless.bootstrap.RunnableFlow;
-
-import javax.net.ssl.SSLContext;
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.stream.Collectors;
+import org.apache.nifi.stateless.core.security.StatelessSecurityUtility;
 
 public class StatelessFlow implements RunnableFlow {
 
@@ -319,6 +319,7 @@ public class StatelessFlow implements RunnableFlow {
 
 
 
+    @Override
     public boolean run(final Queue<InMemoryFlowFile> output) {
         while (!this.stopRequested) {
             for (final StatelessComponent pw : roots) {
@@ -332,6 +333,7 @@ public class StatelessFlow implements RunnableFlow {
         return true;
     }
 
+    @Override
     public boolean runOnce(Queue<InMemoryFlowFile> output) {
         for (final StatelessComponent pw : roots) {
             final boolean successful = pw.runRecursive(output);
@@ -354,6 +356,7 @@ public class StatelessFlow implements RunnableFlow {
         }
 
         final JsonObject sslObject = config.get(SSL).getAsJsonObject();
+        // TODO: Only evaluates to true when all properties are present; some flows can have truststore properties and no keystore or vice-versa
         if (sslObject.has(KEYSTORE) && sslObject.has(KEYSTORE_PASS) && sslObject.has(KEYSTORE_TYPE)
                 && sslObject.has(TRUSTSTORE) && sslObject.has(TRUSTSTORE_PASS) && sslObject.has(TRUSTSTORE_TYPE)) {
 
@@ -377,38 +380,38 @@ public class StatelessFlow implements RunnableFlow {
         return null;
     }
 
-    public static StatelessFlow createAndEnqueueFromJSON(final JsonObject args, final ClassLoader systemClassLoader, final File narWorkingDir)
+    public static StatelessFlow createAndEnqueueFromJSON(final JsonObject jsonObject, final ClassLoader systemClassLoader, final File narWorkingDir)
             throws InitializationException, IOException, ProcessorInstantiationException, NiFiRegistryException {
-        if (args == null) {
+        if (jsonObject == null) {
             throw new IllegalArgumentException("Flow arguments can not be null");
         }
 
-        System.out.println("Running flow from json: " + args.toString());
+        System.out.println("Running flow from json: " + StatelessSecurityUtility.getLoggableRepresentationOfJsonObject(jsonObject));
 
-        if (!args.has(REGISTRY) || !args.has(BUCKETID) || !args.has(FLOWID)) {
+        if (!jsonObject.has(REGISTRY) || !jsonObject.has(BUCKETID) || !jsonObject.has(FLOWID)) {
             throw new IllegalArgumentException("The following parameters must be provided: " + REGISTRY + ", " + BUCKETID + ", " + FLOWID);
         }
 
-        final String registryurl = args.getAsJsonPrimitive(REGISTRY).getAsString();
-        final String bucketID = args.getAsJsonPrimitive(BUCKETID).getAsString();
-        final String flowID = args.getAsJsonPrimitive(FLOWID).getAsString();
+        final String registryurl = jsonObject.getAsJsonPrimitive(REGISTRY).getAsString();
+        final String bucketID = jsonObject.getAsJsonPrimitive(BUCKETID).getAsString();
+        final String flowID = jsonObject.getAsJsonPrimitive(FLOWID).getAsString();
 
         int flowVersion = -1;
-        if (args.has(FLOWVERSION)) {
-            flowVersion = args.getAsJsonPrimitive(FLOWVERSION).getAsInt();
+        if (jsonObject.has(FLOWVERSION)) {
+            flowVersion = jsonObject.getAsJsonPrimitive(FLOWVERSION).getAsInt();
         }
 
         boolean materializeContent = true;
-        if (args.has(MATERIALIZECONTENT)) {
-            materializeContent = args.getAsJsonPrimitive(MATERIALIZECONTENT).getAsBoolean();
+        if (jsonObject.has(MATERIALIZECONTENT)) {
+            materializeContent = jsonObject.getAsJsonPrimitive(MATERIALIZECONTENT).getAsBoolean();
         }
 
         final List<String> failurePorts = new ArrayList<>();
-        if (args.has(FAILUREPORTS)) {
-            args.getAsJsonArray(FAILUREPORTS).forEach(port ->failurePorts.add(port.getAsString()));
+        if (jsonObject.has(FAILUREPORTS)) {
+            jsonObject.getAsJsonArray(FAILUREPORTS).forEach(port ->failurePorts.add(port.getAsString()));
         }
 
-        final SSLContext sslContext = getSSLContext(args);
+        final SSLContext sslContext = getSSLContext(jsonObject);
         final VersionedFlowSnapshot snapshot = new RegistryUtil(registryurl, sslContext).getFlowByID(bucketID, flowID, flowVersion);
 
         final Map<VariableDescriptor, String> inputVariables = new HashMap<>();
@@ -423,8 +426,8 @@ public class StatelessFlow implements RunnableFlow {
 
         final Set<Parameter> parameters = new HashSet<>();
         final Set<String> parameterNames = new HashSet<>();
-        if (args.has(PARAMETERS)) {
-            final JsonElement parametersElement = args.get(PARAMETERS);
+        if (jsonObject.has(PARAMETERS)) {
+            final JsonElement parametersElement = jsonObject.get(PARAMETERS);
             final JsonObject parametersObject = parametersElement.getAsJsonObject();
 
             for (final Map.Entry<String, JsonElement> entry : parametersObject.entrySet()) {
@@ -467,7 +470,7 @@ public class StatelessFlow implements RunnableFlow {
         final ParameterContext parameterContext = new StatelessParameterContext(parameters);
         final ExtensionManager extensionManager = ExtensionDiscovery.discover(narWorkingDir, systemClassLoader);
         final StatelessFlow flow = new StatelessFlow(snapshot.getFlowContents(), extensionManager, () -> inputVariables, failurePorts, materializeContent, sslContext, parameterContext);
-        flow.enqueueFromJSON(args);
+        flow.enqueueFromJSON(jsonObject);
         return flow;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlowFile.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlowFile.java
index 7801572..1f8a5b8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlowFile.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlowFile.java
@@ -17,13 +17,6 @@
 package org.apache.nifi.stateless.core;
 
 import com.google.gson.JsonObject;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
-import org.apache.nifi.processor.exception.FlowFileAccessException;
-import org.apache.nifi.stream.io.StreamUtils;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -40,6 +33,12 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.exception.FlowFileAccessException;
+import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
+import org.apache.nifi.stream.io.StreamUtils;
 
 public class StatelessFlowFile implements InMemoryFlowFile {
 
@@ -104,7 +103,7 @@ public class StatelessFlowFile implements InMemoryFlowFile {
         this.id = nextID.getAndIncrement();
         this.entryDate = System.currentTimeMillis();
         this.lastEnqueuedDate = entryDate;
-        attributes.put(CoreAttributes.FILENAME.key(), String.valueOf(System.nanoTime()) + ".statelessFlowFile");
+        attributes.put(CoreAttributes.FILENAME.key(), System.nanoTime() + ".statelessFlowFile");
         attributes.put(CoreAttributes.PATH.key(), "target");
         attributes.put(CoreAttributes.UUID.key(), UUID.randomUUID().toString());
     }
@@ -241,6 +240,7 @@ public class StatelessFlowFile implements InMemoryFlowFile {
         try {
             result.addProperty("content", new String(this.getDataArray(), StandardCharsets.UTF_8));
         } catch (IOException e) {
+            // TODO: Hex encode binary content if it is not plaintext
             result.addProperty("content", "Exception getting content: " + e.getMessage());
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/security/StatelessSecurityUtility.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/security/StatelessSecurityUtility.java
new file mode 100644
index 0000000..0ff5742
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/security/StatelessSecurityUtility.java
@@ -0,0 +1,174 @@
+/*
+ * 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.stateless.core.security;
+
+import static org.apache.nifi.stateless.runtimes.Program.JSON_FLAG;
+import static org.apache.nifi.stateless.runtimes.Program.YARN_JSON_FLAG;
+
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.security.util.crypto.CipherUtility;
+import org.apache.nifi.stateless.runtimes.Program;
+
+public class StatelessSecurityUtility {
+
+    /**
+     * Returns a masked value of this input.
+     *
+     * @param sensitiveValue the provided input
+     * @return a securely-hashed, deterministic output value
+     */
+    public static String getLoggableRepresentationOfSensitiveValue(String sensitiveValue) {
+        return CipherUtility.getLoggableRepresentationOfSensitiveValue(sensitiveValue);
+    }
+
+    /**
+     * Returns a String representation of this JSON object with a masked value for any sensitive parameters.
+     *
+     * @param json the JSON object
+     * @return the string contents with sensitive parameters masked
+     */
+    public static String getLoggableRepresentationOfJsonObject(final JsonObject json) {
+        JsonObject localJson = null;
+        boolean maskedParams = false;
+
+        if (json.has("parameters")) {
+            JsonObject parameters = json.getAsJsonObject("parameters");
+            for (Map.Entry<String, JsonElement> e : parameters.entrySet()) {
+                if (e.getValue().isJsonObject()) {
+                    JsonObject paramDescriptorMap = (JsonObject) e.getValue();
+                    if (paramDescriptorMap.has("sensitive") && paramDescriptorMap.getAsJsonPrimitive("sensitive").getAsBoolean()) {
+                        maskedParams = true;
+                        if (localJson == null) {
+                            localJson = json.deepCopy();
+                        }
+                        // Point the PDM reference to the copied JSON so we don't modify the parameter internals
+                        paramDescriptorMap = localJson.getAsJsonObject("parameters").getAsJsonObject(e.getKey()).getAsJsonObject();
+                        // This parameter is sensitive; replace its "value" with the masked value
+                        String maskedValue = getLoggableRepresentationOfSensitiveValue(paramDescriptorMap.getAsJsonPrimitive("value").getAsString());
+                        paramDescriptorMap.addProperty("value", maskedValue);
+                        localJson.getAsJsonObject("parameters").add(e.getKey(), paramDescriptorMap);
+                    }
+                }
+            }
+        }
+
+        // If no params were changed, return the original JSON
+        if (!maskedParams) {
+            return json.toString();
+        } else {
+            return localJson.toString();
+        }
+    }
+
+    /**
+     * Returns a String containing the provided arguments, with any JSON objects having their
+     * sensitive values masked. Elements are joined with {@code ,}. If {@code isVerbose} is
+     * {@code false}, elides the JSON entirely.
+     *
+     * @param args      the list of arguments
+     * @param isVerbose if {@code true}, will print the complete JSON value
+     * @return the masked string response
+     */
+    public static String formatArgs(String[] args, boolean isVerbose) {
+        List<String> argsList = new ArrayList<>(Arrays.asList(args));
+        int jsonIndex = determineJsonIndex(argsList);
+
+        if (jsonIndex != -1) {
+            if (isVerbose) {
+                JsonObject json = new JsonParser().parse(argsList.get(jsonIndex)).getAsJsonObject();
+                String maskedJson = getLoggableRepresentationOfJsonObject(json);
+                argsList.add(jsonIndex, maskedJson);
+            } else {
+                argsList.add(jsonIndex, "{...json...}");
+            }
+            argsList.remove(jsonIndex + 1);
+        }
+
+        return String.join(",", argsList);
+    }
+
+    /**
+     * Returns a String containing the JSON object with any sensitive values masked.
+     *
+     * @param json the JSON object
+     * @return a masked string
+     */
+    public static String formatJson(JsonObject json) {
+        return StatelessSecurityUtility.getLoggableRepresentationOfJsonObject(json);
+    }
+
+    /**
+     * Returns the index of the JSON string in this list (checks {@link Program#JSON_FLAG} first, then {@link Program#YARN_JSON_FLAG}). Returns -1 if no JSON is present.
+     *
+     * @param argsList the list of arguments
+     * @return the index of the JSON element or -1
+     */
+    public static int determineJsonIndex(List<String> argsList) {
+        int jsonIndex = -1;
+        if (argsList.contains(JSON_FLAG)) {
+            jsonIndex = determineJsonIndex(argsList, JSON_FLAG);
+        } else if (argsList.contains(YARN_JSON_FLAG)) {
+            jsonIndex = determineJsonIndex(argsList, YARN_JSON_FLAG);
+        }
+        return jsonIndex;
+    }
+
+    /**
+     * Returns the index of the JSON string in this list for the given flag. Returns -1 if no JSON is present.
+     *
+     * @param argsList the list of arguments
+     * @param flag     either {@link Program#JSON_FLAG} or {@link Program#YARN_JSON_FLAG}
+     * @return the index of the JSON element or -1
+     */
+    public static int determineJsonIndex(List<String> argsList, String flag) {
+        // One of the arguments is a JSON string
+        int flagIndex = argsList.indexOf(flag);
+        return flagIndex >= 0 ? flagIndex + 1 : -1;
+    }
+
+    /**
+     * Returns a masked String result given the input if sensitive; the input intact if not.
+     *
+     * @param input the input string
+     * @return masked result if input is sensitive, input otherwise
+     */
+    public static String sanitizeString(String input) {
+        if (isSensitive(input)) {
+            return StatelessSecurityUtility.getLoggableRepresentationOfSensitiveValue(input);
+        } else {
+            return input;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the provided {@code input} is determined to be a sensitive value that
+     * needs masking before output. This method uses a series of regular expressions to define common
+     * keywords like {@code secret} or {@code password} that indicate a sensitive value.
+     *
+     * @param input the input string
+     * @return true if the value should be masked
+     */
+    public static boolean isSensitive(String input) {
+        return input != null && Program.SENSITIVE_INDICATORS.stream().anyMatch(indicator -> input.toLowerCase().matches(".*" + indicator + ".*"));
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/Program.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/Program.java
index 53feeee..616df37 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/Program.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/Program.java
@@ -18,20 +18,22 @@ package org.apache.nifi.stateless.runtimes;
 
 import com.google.gson.JsonObject;
 import com.google.gson.JsonParser;
-import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
-import org.apache.nifi.stateless.bootstrap.RunnableFlow;
-import org.apache.nifi.stateless.core.StatelessFlow;
-import org.apache.nifi.stateless.runtimes.openwhisk.StatelessNiFiOpenWhiskAction;
-import org.apache.nifi.stateless.runtimes.yarn.YARNServiceUtil;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
+import java.util.Arrays;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Queue;
+import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
+import org.apache.nifi.stateless.bootstrap.RunnableFlow;
+import org.apache.nifi.stateless.core.StatelessFlow;
+import org.apache.nifi.stateless.core.security.StatelessSecurityUtility;
+import org.apache.nifi.stateless.runtimes.openwhisk.StatelessNiFiOpenWhiskAction;
+import org.apache.nifi.stateless.runtimes.yarn.YARNServiceUtil;
 
 public class Program {
 
@@ -39,12 +41,26 @@ public class Program {
     public static final String RUN_YARN_SERVICE_FROM_REGISTRY = "RunYARNServiceFromRegistry";
     public static final String RUN_OPENWHISK_ACTION_SERVER = "RunOpenwhiskActionServer";
 
+    public static final String SENSITIVE_TRUE_JSON_SEGMENT = "\"sensitive\"\\s*:\\s*\"true\"";
+    public static final String PASSWORD_SEGMENT = "password";
+    public static final String TOKEN_SEGMENT = "token";
+    public static final String ACCESS_SEGMENT = "access";
+    public static final String SECRET_SEGMENT = "secret";
+    public static final String API_KEY_SEGMENT = "api_key";
+    public static final List<String> SENSITIVE_INDICATORS = Arrays.asList(SENSITIVE_TRUE_JSON_SEGMENT, PASSWORD_SEGMENT, TOKEN_SEGMENT, ACCESS_SEGMENT, SECRET_SEGMENT, API_KEY_SEGMENT);
+
+    public static final String JSON_FLAG = "--json";
+    public static final String FILE_FLAG = "--file";
+    public static final String YARN_JSON_FLAG = "--yarnjson";
+
+    private static boolean isVerbose = true;
+
     public static void launch(final String[] args, final ClassLoader systemClassLoader, final File narWorkingDirectory) throws Exception {
 
         //Workaround for YARN
         //TODO make configurable
         String hadoopTokenFileLocation = System.getenv("HADOOP_TOKEN_FILE_LOCATION");
-        if(hadoopTokenFileLocation != null && !hadoopTokenFileLocation.equals("")) {
+        if (hadoopTokenFileLocation != null && !hadoopTokenFileLocation.equals("")) {
             File targetFile = new File(hadoopTokenFileLocation);
             File parent = targetFile.getParentFile();
             if (!parent.exists() && !parent.mkdirs()) {
@@ -70,7 +86,7 @@ public class Program {
         } else if (args[0].equals(RUN_OPENWHISK_ACTION_SERVER) && args.length == 2) {
             runOnOpenWhisk(args, systemClassLoader, narWorkingDirectory);
         } else {
-            System.out.println("Invalid input: " + String.join(",", args));
+            System.out.println("Invalid input: " + formatArgs(args));
             printUsage();
             System.exit(1);
         }
@@ -88,12 +104,12 @@ public class Program {
         int numberOfContainers = Integer.parseInt(args[4]);
         String json;
 
-        if (args[5].equals("--file")) {
+        if (args[5].equals(FILE_FLAG)) {
             json = new String(Files.readAllBytes(Paths.get(args[6])));
-        } else if (args[5].equals("--json")) {
+        } else if (args[5].equals(JSON_FLAG)) {
             json = args[6];
         } else {
-            System.out.println("Invalid input: " + String.join(",", args));
+            System.out.println("Invalid input: " + formatArgs(args));
             printUsage();
             System.exit(1);
             return;
@@ -101,7 +117,7 @@ public class Program {
         String[] launchCommand = {
                 RUN_FROM_REGISTRY,
                 "Continuous",
-                "--json",
+                JSON_FLAG,
                 new JsonParser().parse(json).toString() //validate and minify
         };
 
@@ -115,21 +131,21 @@ public class Program {
         final boolean once = args[1].equalsIgnoreCase("Once");
 
         final String json;
-        if (args[2].equals("--file")) {
+        if (args[2].equals(FILE_FLAG)) {
             json = new String(Files.readAllBytes(Paths.get(args[3])));
-        } else if (args[2].equals("--json")) {
+        } else if (args[2].equals(JSON_FLAG)) {
             json = args[3];
-        }  else if (args[2].equals("--yarnjson")) {
-            json = args[3].replace(';',',');
+        } else if (args[2].equals(YARN_JSON_FLAG)) {
+            json = args[3].replace(';', ',');
         } else {
-            System.out.println("Invalid input: " + String.join(",", args));
+            System.out.println("Invalid input: " + formatArgs(args));
             printUsage();
             System.exit(1);
             return;
         }
+
         JsonObject jsonObject = new JsonParser().parse(json).getAsJsonObject();
-        System.out.println("Running from json:");
-        System.out.println(jsonObject.toString());
+        System.out.println("Running from json: " + StatelessSecurityUtility.formatJson(jsonObject));
         final RunnableFlow flow = StatelessFlow.createAndEnqueueFromJSON(jsonObject, systemClassLoader, narWorkingDirectory);
 
         // Run Flow
@@ -141,16 +157,36 @@ public class Program {
             successful = flow.run(outputFlowFiles);  //Run forever
         }
 
+        // TODO: Introduce verbose flag to determine if flowfiles should be printed on completion
         if (successful) {
             System.out.println("Flow Succeeded");
-            outputFlowFiles.forEach(f -> System.out.println(f.toStringFull()));
+            if (isVerbose) {
+                outputFlowFiles.forEach(f -> System.out.println(f.toStringFull()));
+            }
         } else {
             System.out.println("Flow Failed");
-            outputFlowFiles.forEach(f -> System.out.println(f.toStringFull()));
+            if (isVerbose) {
+                outputFlowFiles.forEach(f -> System.out.println(f.toStringFull()));
+            }
             System.exit(1);
         }
     }
 
+    public static boolean isVerbose() {
+        return isVerbose;
+    }
+
+    /**
+     * Returns a String containing the provided arguments, with any JSON objects having their
+     * sensitive values masked. Elements are joined with {@code ,}. If {@link #isVerbose()} is
+     * {@code false}, elides the JSON entirely.
+     *
+     * @param args the list of arguments
+     * @return the masked string response
+     */
+    static String formatArgs(String[] args) {
+        return StatelessSecurityUtility.formatArgs(args, isVerbose());
+    }
 
     private static void printUsage() {
         System.out.println("Usage:");
@@ -170,7 +206,7 @@ public class Program {
         System.out.println("Notes:");
         System.out.println("    1) The configuration file must be in JSON format. ");
         System.out.println("    2) When providing configurations via JSON, the following attributes must be provided: " + StatelessFlow.REGISTRY + ", " + StatelessFlow.BUCKETID
-            + ", " + StatelessFlow.FLOWID + ".");
+                + ", " + StatelessFlow.FLOWID + ".");
         System.out.println("          All other attributes will be passed to the flow using the variable registry interface");
         System.out.println();
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/openwhisk/StatelessNiFiOpenWhiskAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/openwhisk/StatelessNiFiOpenWhiskAction.java
index 7927d38..f1fe069 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/openwhisk/StatelessNiFiOpenWhiskAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/openwhisk/StatelessNiFiOpenWhiskAction.java
@@ -21,10 +21,6 @@ import com.google.gson.JsonParser;
 import com.sun.net.httpserver.HttpExchange;
 import com.sun.net.httpserver.HttpHandler;
 import com.sun.net.httpserver.HttpServer;
-import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
-import org.apache.nifi.stateless.bootstrap.RunnableFlow;
-import org.apache.nifi.stateless.core.StatelessFlow;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.IOException;
@@ -39,6 +35,10 @@ import java.util.LinkedList;
 import java.util.Map;
 import java.util.Queue;
 import java.util.stream.Collectors;
+import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
+import org.apache.nifi.stateless.bootstrap.RunnableFlow;
+import org.apache.nifi.stateless.core.StatelessFlow;
+import org.apache.nifi.stateless.core.security.StatelessSecurityUtility;
 
 public class StatelessNiFiOpenWhiskAction {
 
@@ -53,6 +53,7 @@ public class StatelessNiFiOpenWhiskAction {
         this.systemClassLoader = systemClassLoader;
         this.narWorkingDirectory = narWorkingDirectory;
 
+        // TODO: This runs a plaintext HTTP server
         this.server = HttpServer.create(new InetSocketAddress(port), -1);
 
         this.server.createContext("/init", new InitHandler());
@@ -156,18 +157,17 @@ public class StatelessNiFiOpenWhiskAction {
                 Queue<InMemoryFlowFile> output = new LinkedList<>();
                 boolean successful;
                 if (flow == null) {
-                    System.out.println(inputObject.toString());
+                    System.out.println(StatelessSecurityUtility.formatJson(inputObject));
 
                     final JsonObject config = new JsonParser().parse(inputObject.get("code").getAsJsonPrimitive().getAsString()).getAsJsonObject();
                     RunnableFlow tempFlow = StatelessFlow.createAndEnqueueFromJSON(config, systemClassLoader, narWorkingDirectory);
                     successful = tempFlow.runOnce(output);
                 } else {
-                    System.out.println("Input:");
-                    inputObject.entrySet().forEach(item -> System.out.println(item.getKey()+":"+item.getValue().getAsString()));
+                    System.out.println("Input: " + StatelessSecurityUtility.formatJson(inputObject));
 
                     Map<String,String> Attributes = inputObject.entrySet()
                             .stream()
-                            .collect(Collectors.toMap(item -> item.getKey(), item -> item.getValue().getAsString()));
+                            .collect(Collectors.toMap(Map.Entry::getKey, item -> item.getValue().getAsString()));
                     ((StatelessFlow)flow).enqueueFlowFile(new byte[0],Attributes);
                     successful = flow.runOnce(output);
                 }
@@ -185,6 +185,7 @@ public class StatelessNiFiOpenWhiskAction {
                 PrintWriter pw = new PrintWriter(sw);
                 e.printStackTrace(pw);
                 String sStackTrace = sw.toString();
+                // TODO: This leaks the stacktrace in the HTTP response
                 writeResponse(t, 500, "An error has occurred (see logs for details): " + e.getMessage()+"\n"+sStackTrace);
             } finally {
                 writeLogMarkers();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/yarn/YARNServiceUtil.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/yarn/YARNServiceUtil.java
index 0e8c410..e009c59 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/yarn/YARNServiceUtil.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/runtimes/yarn/YARNServiceUtil.java
@@ -18,14 +18,14 @@ package org.apache.nifi.stateless.runtimes.yarn;
 
 import com.google.gson.JsonArray;
 import com.google.gson.JsonObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.StringEntity;
 import org.apache.http.impl.client.BasicResponseHandler;
 import org.apache.http.impl.client.HttpClientBuilder;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
+import org.apache.nifi.stateless.core.security.StatelessSecurityUtility;
 
 public class YARNServiceUtil {
     private final String YARNUrl;
@@ -84,9 +84,9 @@ public class YARNServiceUtil {
             this.YARNUrl + "/app/v1/services?user.name=" + System.getProperty("user.name")
         );
         System.out.println("Running YARN service with the following definition:");
-        System.out.println(spec);
+        System.out.println(StatelessSecurityUtility.formatJson(spec));
         System.out.println("Launch Command");
-        System.out.println(String.join(",", updatedLaunchCommand));
+        System.out.println(StatelessSecurityUtility.formatArgs(updatedLaunchCommand, true));
 
         try {
             request.setEntity(new StringEntity(spec.toString(), " application/json", StandardCharsets.UTF_8.toString()));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/test/groovy/org/apache/nifi/stateless/core/security/StatelessSecurityUtilityTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/test/groovy/org/apache/nifi/stateless/core/security/StatelessSecurityUtilityTest.groovy
new file mode 100644
index 0000000..432983b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/test/groovy/org/apache/nifi/stateless/core/security/StatelessSecurityUtilityTest.groovy
@@ -0,0 +1,222 @@
+/*
+ * 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.stateless.core.security
+
+import com.google.gson.JsonObject
+import com.google.gson.JsonParser
+import groovy.json.JsonSlurper
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+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
+
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class StatelessSecurityUtilityTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessSecurityUtilityTest.class)
+
+    private static final String JSON_ARGS = """{
+  "registryUrl": "http://nifi-registry-service:18080",
+  "bucketId": "50ca47f9-b07a-4199-97cd-e2b519d397d1",
+  "flowId": "9fbe1d70-82ec-44de-b815-c7f838af181a",
+  "parameters": {
+    "DB_IP": "127.0.0.1",
+    "DB_NAME": "database",
+    "DB_PASS": {
+      "sensitive": "true",
+      "value": "password"
+    },
+    "DB_USER": "username"
+  }
+}"""
+    private final String MASKED_REGEX = /\[MASKED\] \([\w\/\+=]+\)/
+
+    @BeforeClass
+    static void setUpOnce() {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() {
+
+    }
+
+    @After
+    void tearDown() {
+
+    }
+
+    @Test
+    void testShouldMaskSensitiveParameterInJsonObject() {
+        // Arrange
+        JsonObject json = new JsonParser().parse(JSON_ARGS).getAsJsonObject()
+        def dbPass = json.getAsJsonObject("parameters").getAsJsonObject("DB_PASS")
+        logger.info("DB password: ${dbPass.toString()}")
+        def dbPassSensitive = dbPass.getAsJsonPrimitive("sensitive").getAsBoolean()
+        def dbPassword = dbPass.getAsJsonPrimitive("value").getAsString()
+
+        // Act
+        String output = StatelessSecurityUtility.getLoggableRepresentationOfJsonObject(json)
+        logger.info("Masked output: ${output}")
+
+        // Assert
+        assert !(output =~ dbPassword)
+        def masked = output =~ MASKED_REGEX
+        assert masked
+    }
+
+    @Test
+    void testShouldMaskMultipleSensitiveParametersInJsonObject() {
+        // Arrange
+        final String MULTIPLE_SENSITIVE_JSON_ARGS = """{
+  "registryUrl": "http://nifi-registry-service:18080",
+  "bucketId": "50ca47f9-b07a-4199-97cd-e2b519d397d1",
+  "flowId": "9fbe1d70-82ec-44de-b815-c7f838af181a",
+  "parameters": {
+    "DB_IP": "127.0.0.1",
+    "DB_NAME": "database",
+    "DB_PASS": {
+      "sensitive": "true",
+      "value": "password"
+    },
+    "DB_OTHER_PASS": {
+      "sensitive": "true",
+      "value": "otherPassword"
+    },
+    "DB_USER": "username"
+  }
+}"""
+
+        JsonObject json = new JsonParser().parse(MULTIPLE_SENSITIVE_JSON_ARGS).getAsJsonObject()
+        def dbPass = json.getAsJsonObject("parameters").getAsJsonObject("DB_PASS")
+        logger.info("DB password: ${dbPass.toString()}")
+        def dbPassword = dbPass.getAsJsonPrimitive("value").getAsString()
+        def dbOtherPassword = json.getAsJsonObject("parameters").getAsJsonObject("DB_OTHER_PASS").getAsJsonPrimitive("value").getAsString()
+
+        // Act
+        String output = StatelessSecurityUtility.getLoggableRepresentationOfJsonObject(json)
+        logger.info("Masked output: ${output}")
+
+        // Assert
+        assert !(output =~ dbPassword)
+        assert !(output =~ dbOtherPassword)
+
+        // Use Groovy JSON assertions
+        def groovyJson = new JsonSlurper().parseText(output)
+        assert groovyJson.parameters.DB_PASS.value =~ MASKED_REGEX
+        assert groovyJson.parameters.DB_OTHER_PASS.value =~ MASKED_REGEX
+    }
+
+    @Test
+    void testShouldNotMaskSensitiveFalseParameterInJsonObject() {
+        // Arrange
+        final String JSON_SENSITIVE_FALSE_ARGS = JSON_ARGS.replaceAll('"sensitive": "true"', '"sensitive": "false"')
+
+        JsonObject json = new JsonParser().parse(JSON_SENSITIVE_FALSE_ARGS).getAsJsonObject()
+        def dbPass = json.getAsJsonObject("parameters").getAsJsonObject("DB_PASS")
+        logger.info("DB password: ${dbPass.toString()}")
+        def dbPassSensitive = dbPass.getAsJsonPrimitive("sensitive").getAsBoolean()
+        assert !dbPassSensitive
+        def dbPassword = dbPass.getAsJsonPrimitive("value").getAsString()
+
+        // Act
+        String output = StatelessSecurityUtility.getLoggableRepresentationOfJsonObject(json)
+        logger.info("Masked output: ${output}")
+
+        // Assert
+        assert output =~ dbPassword
+        assert output == json.toString()
+    }
+
+    @Test
+    void testMaskSensitiveParameterInJsonObjectShouldNotHaveSideEffects() {
+        // Arrange
+        JsonObject json = new JsonParser().parse(JSON_ARGS).getAsJsonObject()
+        def dbPass = json.getAsJsonObject("parameters").getAsJsonObject("DB_PASS")
+        logger.info("DB password: ${dbPass.toString()}")
+        def dbPassSensitive = dbPass.getAsJsonPrimitive("sensitive").getAsBoolean()
+        def dbPassword = dbPass.getAsJsonPrimitive("value").getAsString()
+
+        // Act
+        String output = StatelessSecurityUtility.getLoggableRepresentationOfJsonObject(json)
+        logger.info("Masked output: ${output}")
+        logger.info("Original JSON object after masking: ${json.toString()}")
+
+        // Assert
+        assert json.getAsJsonObject("parameters").getAsJsonObject("DB_PASS").getAsJsonPrimitive("value").getAsString() == "password"
+    }
+
+    @Test
+    void testShouldDetectSensitiveStrings() {
+        // Arrange
+        def sensitiveStrings = [
+                '"sensitive": "true"',
+                '"sensitive":"true"'.toUpperCase(),
+                '"sensitive"\t:\t"true"',
+                '"sensitive"\n:\n\n"true"',
+                '{"parameter_name": {"sensitive": "true", "value": "password"} }',
+                '"password": ',
+                "token",
+                '"access": "some_key_value"',
+                '"secret": "my_secret"'.toUpperCase()
+        ]
+        def safeStrings = [
+                "regular_json",
+                '"sensitive": "false"'
+        ]
+
+        // Act
+        def sensitiveResults = sensitiveStrings.collectEntries {
+            [it, StatelessSecurityUtility.isSensitive(it)]
+        }
+        logger.info("Sensitive results: ${sensitiveResults}")
+
+        def safeResults = safeStrings.collectEntries {
+            [it, StatelessSecurityUtility.isSensitive(it)]
+        }
+        logger.info("Safe results: ${safeResults}")
+
+        // Assert
+        assert sensitiveResults.every { it.value }
+        assert safeResults.every { !it.value }
+    }
+
+
+    @Test
+    void testShouldFormatJson() {
+        // Arrange
+        final JsonObject JSON = new JsonParser().parse(JSON_ARGS.replaceAll("\n", "")).getAsJsonObject()
+
+        // Act
+        String output = StatelessSecurityUtility.formatJson(JSON)
+        logger.info("Masked output: ${output}")
+
+        // Assert
+        assert output =~ MASKED_REGEX
+        assert !(output =~ "password")
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/test/groovy/org/apache/nifi/stateless/runtimes/ProgramTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/test/groovy/org/apache/nifi/stateless/runtimes/ProgramTest.groovy
new file mode 100644
index 0000000..ba030fe
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/test/groovy/org/apache/nifi/stateless/runtimes/ProgramTest.groovy
@@ -0,0 +1,101 @@
+/*
+ * 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.stateless.runtimes
+
+
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+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
+
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class ProgramTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(ProgramTest.class)
+
+    private static final String JSON_ARGS = """{
+  "registryUrl": "http://nifi-registry-service:18080",
+  "bucketId": "50ca47f9-b07a-4199-97cd-e2b519d397d1",
+  "flowId": "9fbe1d70-82ec-44de-b815-c7f838af181a",
+  "parameters": {
+    "DB_IP": "127.0.0.1",
+    "DB_NAME": "database",
+    "DB_PASS": {
+      "sensitive": "true",
+      "value": "password"
+    },
+    "DB_USER": "username"
+  }
+}"""
+    private final String MASKED_REGEX = /\[MASKED\] \([\w\/\+=]+\)/
+
+    @BeforeClass
+    static void setUpOnce() {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() {
+
+    }
+
+    @After
+    void tearDown() {
+
+    }
+
+    @Test
+    void testShouldFormatArgs() {
+        // Arrange
+        final String[] ARGS = ["RunFromRegistry", "Once", "--json", JSON_ARGS] as String[]
+
+        // Act
+        String output = Program.formatArgs(ARGS)
+        logger.info("Masked output: ${output}")
+
+        // Assert
+        assert output =~ MASKED_REGEX
+        assert !(output =~ "password")
+    }
+
+    @Test
+    void testShouldFormatArgsWhenVerbosityDisabled() {
+        // Arrange
+        final String[] ARGS = ["RunFromRegistry", "Once", "--json", JSON_ARGS] as String[]
+        Program.isVerbose = false
+
+        // Act
+        String output = Program.formatArgs(ARGS)
+        logger.info("Masked output: ${output}")
+
+        // Assert
+        assert output.contains("{...json...}")
+        assert !(output =~ "password")
+
+        Program.isVerbose = true
+    }
+}