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 2019/08/16 20:48:40 UTC

[nifi] 02/02: NIFI-6382: On flow import and version change, create parameter context where necessary and add parameters where necessary

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

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

commit bd8342c5e06e71846d1e5976bfe9de45690b974d
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Thu Aug 8 15:54:41 2019 -0400

    NIFI-6382: On flow import and version change, create parameter context where necessary and add parameters where necessary
    
    NIFI-6382: Fixed bugs reported in code review: sensitive properties referencing parameters should be exported to registry; when importing a versioned flow, any new parameters should be added to the parameter context
    
    NIFI-6382: Updated logic for handling parameter references with sensitive properties when integrating with nifi registry
    
    NIFI-6382: If user does not have permissions to READ a parameter context with a given name when importing a flow or changing flow version, ensure that NiFi properly generates an AccessDenied messages instead of throwing NullPointerException
    
    This closes #3642.
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../nifi/controller/AbstractComponentNode.java     |   2 +
 .../serialization/StandardFlowSerializer.java      |   3 +-
 .../apache/nifi/groups/StandardProcessGroup.java   | 181 ++++++++---
 .../nifi/groups/StandardVersionedFlowStatus.java   |   4 +-
 .../nifi/registry/flow/RestBasedFlowRegistry.java  |   2 +
 .../flow/mapping/NiFiRegistryFlowMapper.java       |  19 +-
 .../nifi/integration/FrameworkIntegrationTest.java |  11 +-
 .../integration/MockSingleFlowRegistryClient.java  | 205 ++++++++++++
 .../processors/UsernamePasswordProcessor.java      |  62 ++++
 .../nifi/integration/versioned/ImportFlowIT.java   | 343 ++++++++++++++++++++-
 .../authorization/AuthorizeParameterReference.java |  50 +++
 .../org/apache/nifi/web/NiFiServiceFacade.java     |  16 +-
 .../apache/nifi/web/StandardNiFiServiceFacade.java |  28 +-
 .../apache/nifi/web/api/ProcessGroupResource.java  |   9 +-
 .../org/apache/nifi/web/api/VersionsResource.java  |  17 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |   4 +-
 16 files changed, 901 insertions(+), 55 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index 48e359f..ea29d9e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -658,6 +658,8 @@ public abstract class AbstractComponentNode implements ComponentNode {
                         .valid(false)
                         .explanation("Property references Parameter '" + paramName + "' but the currently selected Parameter Context does not have a Parameter with that name")
                         .build());
+
+                    continue;
                 }
 
                 if (!validationContext.isParameterSet(paramName)) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
index a5c31b7..adedf3c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
@@ -180,7 +180,8 @@ public class StandardFlowSerializer implements FlowSerializer<Document> {
         addStringElement(parameterElement, "sensitive", String.valueOf(descriptor.isSensitive()));
 
         if (descriptor.isSensitive()) {
-            addStringElement(parameterElement, "value", ENC_PREFIX + encryptor.encrypt(parameter.getValue()) + ENC_SUFFIX);
+            final String parameterValue = parameter.getValue();
+            addStringElement(parameterElement, "value", parameterValue == null ? null : ENC_PREFIX + encryptor.encrypt(parameterValue) + ENC_SUFFIX);
         } else {
             addStringElement(parameterElement, "value", parameter.getValue());
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 526e172..52c5cbe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -72,6 +72,7 @@ import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
 import org.apache.nifi.parameter.ParameterReference;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.StandardProcessContext;
@@ -96,6 +97,8 @@ import org.apache.nifi.registry.flow.VersionedFlowState;
 import org.apache.nifi.registry.flow.VersionedFlowStatus;
 import org.apache.nifi.registry.flow.VersionedFunnel;
 import org.apache.nifi.registry.flow.VersionedLabel;
+import org.apache.nifi.registry.flow.VersionedParameter;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.registry.flow.VersionedPort;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
 import org.apache.nifi.registry.flow.VersionedProcessor;
@@ -3547,7 +3550,8 @@ public final class StandardProcessGroup implements ProcessGroup {
 
             final StandardVersionControlInformation originalVci = this.versionControlInfo.get();
             try {
-                updateProcessGroup(this, proposedSnapshot.getFlowContents(), componentIdSeed, updatedVersionedComponentIds, false, updateSettings, updateDescendantVersionedFlows, knownVariables);
+                updateProcessGroup(this, proposedSnapshot.getFlowContents(), componentIdSeed, updatedVersionedComponentIds, false, updateSettings, updateDescendantVersionedFlows, knownVariables,
+                    proposedSnapshot.getParameterContexts());
             } catch (final Throwable t) {
                 // The proposed snapshot may not have any Versioned Flow Coordinates. As a result, the call to #updateProcessGroup may
                 // set this PG's Version Control Info to null. During the normal flow of control,
@@ -3627,7 +3631,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
     private void updateProcessGroup(final ProcessGroup group, final VersionedProcessGroup proposed, final String componentIdSeed,
                                     final Set<String> updatedVersionedComponentIds, final boolean updatePosition, final boolean updateName, final boolean updateDescendantVersionedGroups,
-                                    final Set<String> variablesToSkip) throws ProcessorInstantiationException {
+                                    final Set<String> variablesToSkip, final Map<String, VersionedParameterContext> versionedParameterContexts) throws ProcessorInstantiationException {
 
         // During the flow update, we will use temporary names for process group ports. This is because port names must be
         // unique within a process group, but during an update we might temporarily be in a state where two ports have the same name.
@@ -3647,24 +3651,8 @@ public final class StandardProcessGroup implements ProcessGroup {
             group.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
         }
 
-        // Determine which variables have been added/removed and add/remove them from this group's variable registry.
-        // We don't worry about if a variable value has changed, because variables are designed to be 'environment specific.'
-        // As a result, once imported, we won't update variables to match the remote flow, but we will add any missing variables
-        // and remove any variables that are no longer part of the remote flow.
-        final Set<String> existingVariableNames = group.getVariableRegistry().getVariableMap().keySet().stream()
-                .map(VariableDescriptor::getName)
-                .collect(Collectors.toSet());
-
-        final Map<String, String> updatedVariableMap = new HashMap<>();
-
-        // If any new variables exist in the proposed flow, add those to the variable registry.
-        for (final Map.Entry<String, String> entry : proposed.getVariables().entrySet()) {
-            if (!existingVariableNames.contains(entry.getKey()) && !variablesToSkip.contains(entry.getKey())) {
-                updatedVariableMap.put(entry.getKey(), entry.getValue());
-            }
-        }
-
-        group.setVariables(updatedVariableMap);
+        updateParameterContext(group, proposed, versionedParameterContexts, componentIdSeed);
+        updateVariableRegistry(group, proposed, variablesToSkip);
 
         final VersionedFlowCoordinates remoteCoordinates = proposed.getVersionedFlowCoordinates();
         if (remoteCoordinates == null) {
@@ -3743,12 +3731,13 @@ public final class StandardProcessGroup implements ProcessGroup {
             final VersionedFlowCoordinates childCoordinates = proposedChildGroup.getVersionedFlowCoordinates();
 
             if (childGroup == null) {
-                final ProcessGroup added = addProcessGroup(group, proposedChildGroup, componentIdSeed, variablesToSkip);
+                final ProcessGroup added = addProcessGroup(group, proposedChildGroup, componentIdSeed, variablesToSkip, versionedParameterContexts);
                 flowManager.onProcessGroupAdded(added);
                 added.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
                 LOG.info("Added {} to {}", added, this);
             } else if (childCoordinates == null || updateDescendantVersionedGroups) {
-                updateProcessGroup(childGroup, proposedChildGroup, componentIdSeed, updatedVersionedComponentIds, true, true, updateDescendantVersionedGroups, variablesToSkip);
+                updateProcessGroup(childGroup, proposedChildGroup, componentIdSeed, updatedVersionedComponentIds, true, true, updateDescendantVersionedGroups,
+                    variablesToSkip, versionedParameterContexts);
                 LOG.info("Updated {}", childGroup);
             }
 
@@ -4017,6 +4006,102 @@ public final class StandardProcessGroup implements ProcessGroup {
         }
     }
 
+    private ParameterContext createParameterContext(final VersionedParameterContext versionedParameterContext, final String parameterContextId) {
+        final Map<String, Parameter> parameters = new HashMap<>();
+        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
+            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
+                .name(versionedParameter.getName())
+                .description(versionedParameter.getDescription())
+                .sensitive(versionedParameter.isSensitive())
+                .build();
+
+            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
+            parameters.put(versionedParameter.getName(), parameter);
+        }
+
+        return flowController.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters);
+    }
+
+    private void addMissingParameters(final VersionedParameterContext versionedParameterContext, final ParameterContext currentParameterContext) {
+        final Map<String, Parameter> parameters = new HashMap<>();
+        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
+            final Optional<Parameter> parameterOption = currentParameterContext.getParameter(versionedParameter.getName());
+            if (parameterOption.isPresent()) {
+                // Skip this parameter, since it is already defined. We only want to add missing parameters
+                continue;
+            }
+
+            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
+                .name(versionedParameter.getName())
+                .description(versionedParameter.getDescription())
+                .sensitive(versionedParameter.isSensitive())
+                .build();
+
+            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
+            parameters.put(versionedParameter.getName(), parameter);
+        }
+
+        currentParameterContext.setParameters(parameters);
+    }
+
+    private ParameterContext getParameterContextByName(final String contextName) {
+        return flowController.getFlowManager().getParameterContextManager().getParameterContexts().stream()
+            .filter(context -> context.getName().equals(contextName))
+            .findAny()
+            .orElse(null);
+    }
+
+    private void updateParameterContext(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
+                                        final String componentIdSeed) {
+        // Update the Parameter Context
+        final ParameterContext currentParamContext = group.getParameterContext();
+        final String proposedParameterContextName = proposed.getParameterContextName();
+        if (proposedParameterContextName != null) {
+            if (currentParamContext == null) {
+                // Create a new Parameter Context based on the parameters provided
+                final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(proposedParameterContextName);
+
+                final ParameterContext contextByName = getParameterContextByName(versionedParameterContext.getName());
+                final ParameterContext selectedParameterContext;
+                if (contextByName == null) {
+                    final String parameterContextId = generateUuid(versionedParameterContext.getName(), versionedParameterContext.getName(), componentIdSeed);
+                    selectedParameterContext = createParameterContext(versionedParameterContext, parameterContextId);
+                } else {
+                    selectedParameterContext = contextByName;
+                    addMissingParameters(versionedParameterContext, selectedParameterContext);
+                }
+
+                group.setParameterContext(selectedParameterContext);
+            } else {
+                // Update the current Parameter Context so that it has any Parameters included in the proposed context
+                final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(proposedParameterContextName);
+                addMissingParameters(versionedParameterContext, currentParamContext);
+            }
+        }
+    }
+
+    private void updateVariableRegistry(final ProcessGroup group, final VersionedProcessGroup proposed, final Set<String> variablesToSkip) {
+        // Determine which variables have been added/removed and add/remove them from this group's variable registry.
+        // We don't worry about if a variable value has changed, because variables are designed to be 'environment specific.'
+        // As a result, once imported, we won't update variables to match the remote flow, but we will add any missing variables
+        // and remove any variables that are no longer part of the remote flow.
+        final Set<String> existingVariableNames = group.getVariableRegistry().getVariableMap().keySet().stream()
+            .map(VariableDescriptor::getName)
+            .collect(Collectors.toSet());
+
+        final Map<String, String> updatedVariableMap = new HashMap<>();
+
+        // If any new variables exist in the proposed flow, add those to the variable registry.
+        for (final Map.Entry<String, String> entry : proposed.getVariables().entrySet()) {
+            if (!existingVariableNames.contains(entry.getKey()) && !variablesToSkip.contains(entry.getKey())) {
+                updatedVariableMap.put(entry.getKey(), entry.getValue());
+            }
+        }
+
+        group.setVariables(updatedVariableMap);
+    }
+
+
     private String getPublicPortFinalName(final PublicPort publicPort, final String proposedFinalName) {
         final Optional<Port> existingPublicPort;
         if (TransferDirection.RECEIVE == publicPort.getDirection()) {
@@ -4074,12 +4159,13 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
 
-    private ProcessGroup addProcessGroup(final ProcessGroup destination, final VersionedProcessGroup proposed, final String componentIdSeed, final Set<String> variablesToSkip)
+    private ProcessGroup addProcessGroup(final ProcessGroup destination, final VersionedProcessGroup proposed, final String componentIdSeed, final Set<String> variablesToSkip,
+                                         final Map<String, VersionedParameterContext> versionedParameterContexts)
             throws ProcessorInstantiationException {
         final ProcessGroup group = flowManager.createProcessGroup(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed));
         group.setVersionedComponentId(proposed.getIdentifier());
         group.setParent(destination);
-        updateProcessGroup(group, proposed, componentIdSeed, Collections.emptySet(), true, true, true, variablesToSkip);
+        updateProcessGroup(group, proposed, componentIdSeed, Collections.emptySet(), true, true, true, variablesToSkip, versionedParameterContexts);
         destination.addProcessGroup(group);
         return group;
     }
@@ -4308,7 +4394,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             service.setComments(proposed.getComments());
             service.setName(proposed.getName());
 
-            final Map<String, String> properties = populatePropertiesMap(service.getEffectivePropertyValues(), proposed.getProperties(), proposed.getPropertyDescriptors(), service.getProcessGroup());
+            final Map<String, String> properties = populatePropertiesMap(service, proposed.getProperties(), proposed.getPropertyDescriptors(), service.getProcessGroup());
             service.setProperties(properties, true);
 
             if (!isEqual(service.getBundleCoordinate(), proposed.getBundle())) {
@@ -4447,7 +4533,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             processor.setName(proposed.getName());
             processor.setPenalizationPeriod(proposed.getPenaltyDuration());
 
-            final Map<String, String> properties = populatePropertiesMap(processor.getRawPropertyValues(), proposed.getProperties(), proposed.getPropertyDescriptors(), processor.getProcessGroup());
+            final Map<String, String> properties = populatePropertiesMap(processor, proposed.getProperties(), proposed.getPropertyDescriptors(), processor.getProcessGroup());
             processor.setProperties(properties, true);
             processor.setRunDuration(proposed.getRunDurationMillis(), TimeUnit.MILLISECONDS);
             processor.setSchedulingStrategy(SchedulingStrategy.valueOf(proposed.getSchedulingStrategy()));
@@ -4470,7 +4556,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
 
-    private Map<String, String> populatePropertiesMap(final Map<PropertyDescriptor, String> currentProperties, final Map<String, String> proposedProperties,
+    private Map<String, String> populatePropertiesMap(final ComponentNode componentNode, final Map<String, String> proposedProperties,
                                                       final Map<String, VersionedPropertyDescriptor> proposedDescriptors, final ProcessGroup group) {
 
         // since VersionedPropertyDescriptor currently doesn't know if it is sensitive or not,
@@ -4478,7 +4564,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         final Set<String> sensitiveProperties = new HashSet<>();
 
         final Map<String, String> fullPropertyMap = new HashMap<>();
-        for (final PropertyDescriptor property : currentProperties.keySet()) {
+        for (final PropertyDescriptor property : componentNode.getRawPropertyValues().keySet()) {
             if (property.isSensitive()) {
                 sensitiveProperties.add(property.getName());
             } else {
@@ -4487,25 +4573,46 @@ public final class StandardProcessGroup implements ProcessGroup {
         }
 
         if (proposedProperties != null) {
-            for (final Map.Entry<String, String> entry : proposedProperties.entrySet()) {
-                final String propertyName = entry.getKey();
+            // Build a Set of all properties that are included in either the currently configured property values or the proposed values.
+            final Set<String> updatedPropertyNames = new HashSet<>();
+            updatedPropertyNames.addAll(proposedProperties.keySet());
+            componentNode.getProperties().keySet().stream()
+                .map(PropertyDescriptor::getName)
+                .forEach(updatedPropertyNames::add);
+
+            for (final String propertyName : updatedPropertyNames) {
                 final VersionedPropertyDescriptor descriptor = proposedDescriptors.get(propertyName);
 
-                // skip any sensitive properties so we can retain whatever is currently set
-                if (sensitiveProperties.contains(propertyName)) {
-                    continue;
-                }
-
                 String value;
                 if (descriptor != null && descriptor.getIdentifiesControllerService()) {
                     // Property identifies a Controller Service. So the value that we want to assign is not the value given.
                     // The value given is instead the Versioned Component ID of the Controller Service. We want to resolve this
                     // to the instance ID of the Controller Service.
-                    final String serviceVersionedComponentId = entry.getValue();
+                    final String serviceVersionedComponentId = proposedProperties.get(propertyName);
                     String instanceId = getServiceInstanceId(serviceVersionedComponentId, group);
                     value = instanceId == null ? serviceVersionedComponentId : instanceId;
                 } else {
-                    value = entry.getValue();
+                    value = proposedProperties.get(propertyName);
+                }
+
+                // skip any sensitive properties that are not populated so we can retain whatever is currently set. We do this because sensitive properties are not stored in the registry
+                // unless the value is a reference to a Parameter. If the value in the registry is null, it indicates that the sensitive value was removed, so we want to keep the currently
+                // populated value. The exception to this rule is if the currently configured value is a Parameter Reference and the Versioned Flow is empty. In this case, it implies
+                // that the Versioned Flow has changed from a Parameter Reference to an explicit value. In this case, we do in fact want to change the value of the Sensitive Property from
+                // the current parameter reference to an unset value.
+                if (sensitiveProperties.contains(propertyName) && value == null) {
+                    final PropertyConfiguration propertyConfiguration = componentNode.getProperty(componentNode.getPropertyDescriptor(propertyName));
+                    if (propertyConfiguration == null) {
+                        continue;
+                    }
+
+                    // No parameter references. Property currently is set to an explicit value. We don't want to change it.
+                    if (propertyConfiguration.getParameterReferences().isEmpty()) {
+                        continue;
+                    }
+
+                    // Once we reach this point, the property is configured to reference a Parameter, and the value in the Versioned Flow is an explicit value,
+                    // so we want to continue on and update the value to null.
                 }
 
                 fullPropertyMap.put(propertyName, value);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardVersionedFlowStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardVersionedFlowStatus.java
index 4be9898..604ec42 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardVersionedFlowStatus.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardVersionedFlowStatus.java
@@ -20,11 +20,11 @@ package org.apache.nifi.groups;
 import org.apache.nifi.registry.flow.VersionedFlowState;
 import org.apache.nifi.registry.flow.VersionedFlowStatus;
 
-class StandardVersionedFlowStatus implements VersionedFlowStatus {
+public class StandardVersionedFlowStatus implements VersionedFlowStatus {
     private final VersionedFlowState state;
     private final String explanation;
 
-    StandardVersionedFlowStatus(final VersionedFlowState state, final String explanation) {
+    public StandardVersionedFlowStatus(final VersionedFlowState state, final String explanation) {
         this.state = state;
         this.explanation = explanation;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/RestBasedFlowRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/RestBasedFlowRegistry.java
index 08168b1..a7f87d2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/RestBasedFlowRegistry.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/RestBasedFlowRegistry.java
@@ -36,6 +36,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 public class RestBasedFlowRegistry implements FlowRegistry {
+    private static final String FLOW_ENCODING_VERSION = "1.0";
 
     private final FlowRegistryClient flowRegistryClient;
     private final String identifier;
@@ -189,6 +190,7 @@ public class RestBasedFlowRegistry implements FlowRegistry {
         versionedFlowSnapshot.setFlowContents(snapshot);
         versionedFlowSnapshot.setExternalControllerServices(externalControllerServices);
         versionedFlowSnapshot.setParameterContexts(parameterContextMap);
+        versionedFlowSnapshot.setFlowEncodingVersion(FLOW_ENCODING_VERSION);
 
         final VersionedFlowSnapshotMetadata metadata = new VersionedFlowSnapshotMetadata();
         metadata.setBucketIdentifier(flow.getBucketIdentifier());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
index 3c0e8f2..fdfc143 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
@@ -27,6 +27,7 @@ import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.ComponentNode;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.PropertyConfiguration;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.service.ControllerServiceNode;
@@ -362,7 +363,7 @@ public class NiFiRegistryFlowMapper {
         final Map<String, String> mapped = new HashMap<>();
 
         component.getProperties().keySet().stream()
-            .filter(property -> !property.isSensitive())
+            .filter(property -> isMappable(property, component.getProperty(property)))
             .forEach(property -> {
                 String value = component.getRawPropertyValue(property);
                 if (value == null) {
@@ -384,6 +385,21 @@ public class NiFiRegistryFlowMapper {
         return mapped;
     }
 
+    private boolean isMappable(final PropertyDescriptor propertyDescriptor, final PropertyConfiguration propertyConfiguration) {
+        if (!propertyDescriptor.isSensitive()) { // If the property is not sensitive, it can be mapped.
+            return true;
+        }
+
+        if (propertyConfiguration == null) {
+            return false;
+        }
+
+        // Sensitive properties can be mapped if and only if they reference a Parameter. If a sensitive property references a parameter, it cannot contain any other value around it.
+        // For example, for a non-sensitive property, a value of "hello#{param}123" is valid, but for a sensitive property, it is invalid. Only something like "hello123" or "#{param}" is valid.
+        // Thus, we will map sensitive properties only if they reference a parameter.
+        return !propertyConfiguration.getParameterReferences().isEmpty();
+    }
+
     private Map<String, VersionedPropertyDescriptor> mapPropertyDescriptors(final ComponentNode component, final ControllerServiceProvider serviceProvider, final Set<String> includedGroupIds,
                                                                             final Map<String, ExternalControllerServiceReference> externalControllerServiceReferences) {
         final Map<String, VersionedPropertyDescriptor> descriptors = new HashMap<>();
@@ -391,6 +407,7 @@ public class NiFiRegistryFlowMapper {
             final VersionedPropertyDescriptor versionedDescriptor = new VersionedPropertyDescriptor();
             versionedDescriptor.setName(descriptor.getName());
             versionedDescriptor.setDisplayName(descriptor.getDisplayName());
+            versionedDescriptor.setSensitive(descriptor.isSensitive());
 
             final Class<?> referencedServiceType = descriptor.getControllerServiceDefinition();
             versionedDescriptor.setIdentifiesControllerService(referencedServiceType != null);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
index 6826b59..8a01779 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
@@ -76,6 +76,7 @@ import org.apache.nifi.integration.processors.GenerateProcessor;
 import org.apache.nifi.integration.processors.NopProcessor;
 import org.apache.nifi.integration.processors.TerminateAll;
 import org.apache.nifi.integration.processors.TerminateOnce;
+import org.apache.nifi.integration.processors.UsernamePasswordProcessor;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.SystemBundle;
@@ -143,7 +144,7 @@ public class FrameworkIntegrationTest {
 
     private FlowEngine flowEngine;
     private FlowController flowController;
-    private FlowRegistryClient flowRegistryClient = new StandardFlowRegistryClient();
+    private FlowRegistryClient flowRegistryClient = createFlowRegistryClient();
     private ProcessorNode nopProcessor;
     private ProcessorNode terminateProcessor;
     private ProcessorNode terminateAllProcessor;
@@ -193,6 +194,13 @@ public class FrameworkIntegrationTest {
         initialize(nifiProperties);
     }
 
+    /**
+     * This method exists for subclasses to override and return a different implementation.
+     */
+    protected FlowRegistryClient createFlowRegistryClient() {
+        return new StandardFlowRegistryClient();
+    }
+
     protected final void initialize(final NiFiProperties nifiProperties) throws IOException {
         this.nifiProperties = nifiProperties;
 
@@ -214,6 +222,7 @@ public class FrameworkIntegrationTest {
         extensionManager.injectExtensionType(Processor.class, TerminateOnce.class);
         extensionManager.injectExtensionType(Processor.class, TerminateAll.class);
         extensionManager.injectExtensionType(Processor.class, NopProcessor.class);
+        extensionManager.injectExtensionType(Processor.class, UsernamePasswordProcessor.class);
 
         injectExtensionTypes(extensionManager);
         systemBundle = SystemBundle.create(nifiProperties);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/MockSingleFlowRegistryClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/MockSingleFlowRegistryClient.java
new file mode 100644
index 0000000..0a6c932
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/MockSingleFlowRegistryClient.java
@@ -0,0 +1,205 @@
+/*
+ * 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.integration;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.flow.ExternalControllerServiceReference;
+import org.apache.nifi.registry.flow.FlowRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.registry.flow.VersionedFlow;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
+import org.apache.nifi.registry.flow.VersionedProcessGroup;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class MockSingleFlowRegistryClient implements FlowRegistryClient {
+    private final MockFlowRegistry flowRegistry = new MockFlowRegistry();
+
+    @Override
+    public FlowRegistry getFlowRegistry(final String registryId) {
+        return flowRegistry;
+    }
+
+    @Override
+    public Set<String> getRegistryIdentifiers() {
+        return Collections.singleton("unit-test-registry-client-id");
+    }
+
+    @Override
+    public void addFlowRegistry(final FlowRegistry registry) {
+
+    }
+
+    @Override
+    public FlowRegistry addFlowRegistry(final String registryId, final String registryName, final String registryUrl, final String description) {
+        return null;
+    }
+
+    @Override
+    public FlowRegistry removeFlowRegistry(final String registryId) {
+        return null;
+    }
+
+    public void addFlow(final String bucketId, final String flowId, final int version, final VersionedFlowSnapshot snapshot) {
+        flowRegistry.addFlow(bucketId, flowId, version, snapshot);
+    }
+
+
+    private static class FlowCoordinates {
+        private final String bucketId;
+        private final String flowId;
+        private final int version;
+
+        public FlowCoordinates(final String bucketId, final String flowId, final int version) {
+            this.bucketId = bucketId;
+            this.flowId = flowId;
+            this.version = version;
+        }
+
+        public String getBucketId() {
+            return bucketId;
+        }
+
+        public String getFlowId() {
+            return flowId;
+        }
+
+        public int getVersion() {
+            return version;
+        }
+    }
+
+
+    public static class MockFlowRegistry implements FlowRegistry {
+        private final Map<FlowCoordinates, VersionedFlowSnapshot> snapshots = new ConcurrentHashMap<>();
+
+        public void addFlow(final String bucketId, final String flowId, final int version, final VersionedFlowSnapshot snapshot) {
+            final FlowCoordinates coordinates = new FlowCoordinates(bucketId, flowId, version);
+            snapshots.put(coordinates, snapshot);
+        }
+
+
+        @Override
+        public String getIdentifier() {
+            return "int-test-flow-registry";
+        }
+
+        @Override
+        public String getDescription() {
+            return null;
+        }
+
+        @Override
+        public void setDescription(final String description) {
+
+        }
+
+        @Override
+        public String getURL() {
+            return "http://localhost:18080/integration-test";
+        }
+
+        @Override
+        public void setURL(final String url) {
+
+        }
+
+        @Override
+        public String getName() {
+            return "Integration Test Registry";
+        }
+
+        @Override
+        public void setName(final String name) {
+
+        }
+
+        @Override
+        public Set<Bucket> getBuckets(final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public Bucket getBucket(final String bucketId, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public Set<VersionedFlow> getFlows(final String bucketId, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public Set<VersionedFlowSnapshotMetadata> getFlowVersions(final String bucketId, final String flowId, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public VersionedFlow registerVersionedFlow(final VersionedFlow flow, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public VersionedFlow deleteVersionedFlow(final String bucketId, final String flowId, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public VersionedFlowSnapshot registerVersionedFlowSnapshot(final VersionedFlow flow, final VersionedProcessGroup snapshot,
+                                                                   final Map<String, ExternalControllerServiceReference> externalControllerServices,
+                                                                   final Collection<VersionedParameterContext> parameterContexts, final String comments,
+                                                                   final int expectedVersion, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public int getLatestVersion(final String bucketId, final String flowId, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return 0;
+        }
+
+        @Override
+        public VersionedFlowSnapshot getFlowContents(final String bucketId, final String flowId, final int version, final boolean fetchRemoteFlows, final NiFiUser user)
+                        throws IOException, NiFiRegistryException {
+            return getFlowContents(bucketId, flowId, version, fetchRemoteFlows);
+        }
+
+        @Override
+        public VersionedFlowSnapshot getFlowContents(final String bucketId, final String flowId, final int version, final boolean fetchRemoteFlows) throws IOException, NiFiRegistryException {
+            final FlowCoordinates coordinates = new FlowCoordinates(bucketId, flowId, version);
+            return snapshots.get(coordinates);
+        }
+
+        @Override
+        public VersionedFlow getVersionedFlow(final String bucketId, final String flowId, final NiFiUser user) throws IOException, NiFiRegistryException {
+            return null;
+        }
+
+        @Override
+        public VersionedFlow getVersionedFlow(final String bucketId, final String flowId) throws IOException, NiFiRegistryException {
+            return null;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processors/UsernamePasswordProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processors/UsernamePasswordProcessor.java
new file mode 100644
index 0000000..09378e0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processors/UsernamePasswordProcessor.java
@@ -0,0 +1,62 @@
+/*
+ * 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.integration.processors;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class UsernamePasswordProcessor extends AbstractProcessor {
+    public static final PropertyDescriptor USERNAME = new Builder()
+        .name("username")
+        .displayName("username")
+        .description("username")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        .defaultValue("")
+        .build();
+
+    public static final PropertyDescriptor PASSWORD = new Builder()
+        .name("password")
+        .displayName("password")
+        .description("password")
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        .defaultValue("")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(USERNAME, PASSWORD);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
index 99f91b8..996d363 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
@@ -27,6 +27,13 @@ import org.apache.nifi.integration.DirectInjectionExtensionManager;
 import org.apache.nifi.integration.FrameworkIntegrationTest;
 import org.apache.nifi.integration.cs.LongValidatingControllerService;
 import org.apache.nifi.integration.cs.NopServiceReferencingProcessor;
+import org.apache.nifi.integration.processors.UsernamePasswordProcessor;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterReferenceManager;
+import org.apache.nifi.parameter.StandardParameterContext;
+import org.apache.nifi.parameter.StandardParameterReferenceManager;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.registry.bucket.Bucket;
 import org.apache.nifi.registry.flow.Bundle;
@@ -38,21 +45,36 @@ import org.apache.nifi.registry.flow.VersionedParameter;
 import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
 import org.apache.nifi.registry.flow.VersionedProcessor;
+import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
+import org.apache.nifi.registry.flow.diff.ConciseEvolvingDifferenceDescriptor;
+import org.apache.nifi.registry.flow.diff.DifferenceType;
+import org.apache.nifi.registry.flow.diff.FlowComparator;
+import org.apache.nifi.registry.flow.diff.FlowComparison;
+import org.apache.nifi.registry.flow.diff.FlowDifference;
+import org.apache.nifi.registry.flow.diff.StandardComparableDataFlow;
+import org.apache.nifi.registry.flow.diff.StandardFlowComparator;
 import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
+import org.apache.nifi.util.FlowDifferenceFilters;
 import org.junit.Test;
 
+import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
 
 import static junit.framework.TestCase.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.AssertJUnit.assertNull;
 
 public class ImportFlowIT extends FrameworkIntegrationTest {
 
@@ -110,7 +132,312 @@ public class ImportFlowIT extends FrameworkIntegrationTest {
     }
 
 
-    private VersionedFlowSnapshot createFlowSnapshot(final List<ControllerServiceNode> controllerServices, final List<ProcessorNode> processors, final Map<String, String> parameters) {
+    @Test
+    public void testLocalModificationWhenSensitivePropReferencesParameter() {
+        // Create a processor with a sensitive property
+        final ProcessorNode processor = createProcessorNode(UsernamePasswordProcessor.class);
+        processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "password"));
+
+        // Create a VersionedFlowSnapshot that contains the processor
+        final VersionedFlowSnapshot versionedFlowWithExplicitValue = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(processor), null);
+
+        // Create child group
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        // Move processor into the child group
+        moveProcessor(processor, innerGroup);
+
+        // Verify that there are no differences between the versioned flow and the Process Group
+        Set<FlowDifference> differences = getLocalModifications(innerGroup, versionedFlowWithExplicitValue);
+        assertEquals(0, differences.size());
+
+        // Change the value of the sensitive property from one explicit value to another. Verify no local modifications.
+        processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "secret"));
+        differences = getLocalModifications(innerGroup, versionedFlowWithExplicitValue);
+        assertEquals(0, differences.size());
+
+        // Change the value of the sensitive property to now reference a parameter. There should be one local modification.
+        processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-parameter}"));
+        differences = getLocalModifications(innerGroup, versionedFlowWithExplicitValue);
+        assertEquals(1, differences.size());
+        assertEquals(DifferenceType.PROPERTY_ADDED, differences.iterator().next().getDifferenceType());
+
+        // Create a Versioned Flow that contains the Parameter Reference.
+        final VersionedFlowSnapshot versionedFlowWithParameterReference = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(processor), null);
+
+        // Ensure no difference between the current configuration and the versioned flow
+        differences = getLocalModifications(innerGroup, versionedFlowWithParameterReference);
+        assertEquals(0, differences.size());
+
+        processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "secret"));
+        differences = getLocalModifications(innerGroup, versionedFlowWithParameterReference);
+        assertEquals(1, differences.size());
+        assertEquals(DifferenceType.PROPERTY_REMOVED, differences.iterator().next().getDifferenceType());
+    }
+
+    @Test
+    public void testParameterCreatedWithNullValueOnImportWithSensitivePropertyReference() {
+        // Create a processor with a sensitive property
+        final ProcessorNode processor = createProcessorNode(UsernamePasswordProcessor.class);
+        processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-param}"));
+
+        // Create a VersionedFlowSnapshot that contains the processor
+        final Parameter parameter = new Parameter(new ParameterDescriptor.Builder().name("secret-param").sensitive(true).build(), null);
+        final VersionedFlowSnapshot versionedFlowWithParameterReference = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(processor), Collections.singleton(parameter));
+
+        // Create child group
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(getFlowController().getFlowManager());
+        final ParameterContext parameterContext = new StandardParameterContext("param-context-id", "parameter-context", parameterReferenceManager, null);
+        innerGroup.setParameterContext(parameterContext);
+
+        assertTrue(parameterContext.getParameters().isEmpty());
+
+        innerGroup.updateFlow(versionedFlowWithParameterReference, null, true, true, true);
+
+        final Collection<Parameter> parameters = parameterContext.getParameters().values();
+        assertEquals(1, parameters.size());
+
+        final Parameter firstParameter = parameters.iterator().next();
+        assertEquals("secret-param", firstParameter.getDescriptor().getName());
+        assertTrue(firstParameter.getDescriptor().isSensitive());
+        assertNull(firstParameter.getValue());
+    }
+
+    @Test
+    public void testParameterContextCreatedOnImportWithSensitivePropertyReference() {
+        // Create a processor with a sensitive property
+        final ProcessorNode processor = createProcessorNode(UsernamePasswordProcessor.class);
+        processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-param}"));
+
+        // Create a VersionedFlowSnapshot that contains the processor
+        final Parameter parameter = new Parameter(new ParameterDescriptor.Builder().name("secret-param").sensitive(true).build(), null);
+        final VersionedFlowSnapshot versionedFlowWithParameterReference = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(processor), Collections.singleton(parameter));
+
+        // Create child group
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        innerGroup.updateFlow(versionedFlowWithParameterReference, null, true, true, true);
+
+        final ParameterContext parameterContext = innerGroup.getParameterContext();
+        assertNotNull(parameterContext);
+
+        final Collection<Parameter> parameters = parameterContext.getParameters().values();
+        assertEquals(1, parameters.size());
+
+        final Parameter firstParameter = parameters.iterator().next();
+        assertEquals("secret-param", firstParameter.getDescriptor().getName());
+        assertTrue(firstParameter.getDescriptor().isSensitive());
+        assertNull(firstParameter.getValue());
+    }
+
+
+    @Test
+    public void testChangeVersionFromParameterToExplicitValueSensitiveProperty() {
+        // Create a processor with a sensitive property
+        final ProcessorNode initialProcessor = createProcessorNode(UsernamePasswordProcessor.class);
+        initialProcessor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-param}"));
+
+        // Create a VersionedFlowSnapshot that contains the processor
+        final Parameter parameter = new Parameter(new ParameterDescriptor.Builder().name("secret-param").sensitive(true).build(), null);
+        final VersionedFlowSnapshot versionedFlowWithParameterReference = createFlowSnapshot(Collections.emptyList(),
+            Collections.singletonList(initialProcessor), Collections.singleton(parameter));
+
+
+        // Update processor to have an explicit value for the second version of the flow.
+        initialProcessor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "secret-value"));
+        final VersionedFlowSnapshot versionedFlowExplicitValue = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(initialProcessor), null);
+
+        // Create child group and update to the first version of the flow, with parameter ref
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        innerGroup.updateFlow(versionedFlowWithParameterReference, null, true, true, true);
+
+        final ProcessorNode nodeInGroupWithRef = innerGroup.getProcessors().iterator().next();
+        assertNotNull(nodeInGroupWithRef.getProperty(UsernamePasswordProcessor.PASSWORD).getRawValue());
+
+        // Update the flow to new version that uses explicit value.
+        innerGroup.updateFlow(versionedFlowExplicitValue, null, true, true, true);
+
+        // Updated flow has sensitive property that no longer references parameter. Now is an explicit value, so it should be unset
+        final ProcessorNode nodeInGroupWithNoValue = innerGroup.getProcessors().iterator().next();
+        assertNull(nodeInGroupWithNoValue.getProperty(UsernamePasswordProcessor.PASSWORD).getRawValue());
+    }
+
+    @Test
+    public void testChangeVersionFromExplicitToExplicitValueDoesNotChangeSensitiveProperty() {
+        // Create a processor with a sensitive property and create a versioned flow for it.
+        final ProcessorNode initialProcessor = createProcessorNode(UsernamePasswordProcessor.class);
+        final Map<String, String> initialProperties = new HashMap<>();
+        initialProperties.put(UsernamePasswordProcessor.USERNAME.getName(), "user");
+        initialProperties.put(UsernamePasswordProcessor.PASSWORD.getName(), "pass");
+        initialProcessor.setProperties(initialProperties);
+
+        final VersionedFlowSnapshot initialVersionSnapshot = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(initialProcessor), null);
+
+        // Update processor to have a different explicit value for both sensitive and non-sensitive properties and create a versioned flow for it.
+        final Map<String, String> updatedProperties = new HashMap<>();
+        updatedProperties.put(UsernamePasswordProcessor.USERNAME.getName(), "other");
+        updatedProperties.put(UsernamePasswordProcessor.PASSWORD.getName(), "pass");
+        initialProcessor.setProperties(updatedProperties);
+
+        final VersionedFlowSnapshot updatedVersionSnapshot = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(initialProcessor), null);
+
+        // Create child group and update to the first version of the flow, with parameter ref
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        // Import the flow into our newly created group
+        innerGroup.updateFlow(initialVersionSnapshot, null, true, true, true);
+
+        final ProcessorNode initialImportedProcessor = innerGroup.getProcessors().iterator().next();
+        assertEquals("user", initialImportedProcessor.getProperty(UsernamePasswordProcessor.USERNAME).getRawValue());
+        assertNull("pass", initialImportedProcessor.getProperty(UsernamePasswordProcessor.PASSWORD).getRawValue());
+
+        // Update the sensitive property to "pass"
+        initialImportedProcessor.setProperties(initialProperties);
+        assertEquals("pass", initialImportedProcessor.getProperty(UsernamePasswordProcessor.PASSWORD).getRawValue());
+
+        // Update the flow to new version
+        innerGroup.updateFlow(updatedVersionSnapshot, null, true, true, true);
+
+        // Updated flow has sensitive property that no longer references parameter. Now is an explicit value, so it should be unset
+        final ProcessorNode updatedImportedProcessor = innerGroup.getProcessors().iterator().next();
+        assertEquals("other", updatedImportedProcessor.getProperty(UsernamePasswordProcessor.USERNAME).getRawValue());
+        assertEquals("pass", updatedImportedProcessor.getProperty(UsernamePasswordProcessor.PASSWORD).getRawValue());
+    }
+
+
+    @Test
+    public void testChangeVersionFromParamReferenceToAnotherParamReferenceIsLocalModification() {
+        // Create a processor with a sensitive property and create a versioned flow for it.
+        final ProcessorNode initialProcessor = createProcessorNode(UsernamePasswordProcessor.class);
+        final Map<String, String> initialProperties = new HashMap<>();
+        initialProperties.put(UsernamePasswordProcessor.USERNAME.getName(), "user");
+        initialProperties.put(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-param}");
+        initialProcessor.setProperties(initialProperties);
+
+        final VersionedFlowSnapshot initialVersionSnapshot = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(initialProcessor), null);
+
+        // Update processor to have a different explicit value for both sensitive and non-sensitive properties and create a versioned flow for it.
+        final Map<String, String> updatedProperties = new HashMap<>();
+        updatedProperties.put(UsernamePasswordProcessor.USERNAME.getName(), "user");
+        updatedProperties.put(UsernamePasswordProcessor.PASSWORD.getName(), "#{other-param}");
+        initialProcessor.setProperties(updatedProperties);
+
+        final VersionedFlowSnapshot updatedVersionSnapshot = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(initialProcessor), null);
+
+        // Create child group and update to the first version of the flow, with parameter ref
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        // Import the flow into our newly created group
+        innerGroup.updateFlow(initialVersionSnapshot, null, true, true, true);
+
+        final Set<FlowDifference> localModifications = getLocalModifications(innerGroup, updatedVersionSnapshot);
+        assertEquals(1, localModifications.size());
+        assertEquals(DifferenceType.PROPERTY_CHANGED, localModifications.iterator().next().getDifferenceType());
+    }
+
+
+    @Test
+    public void testChangeVersionFromExplicitValueToParameterSensitiveProperty() {
+        // Create a processor with a sensitive property
+        final ProcessorNode processorWithParamRef = createProcessorNode(UsernamePasswordProcessor.class);
+        processorWithParamRef.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-param}"));
+
+        final ProcessorNode processorWithExplicitValue = createProcessorNode(UsernamePasswordProcessor.class);
+        processorWithExplicitValue.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "secret-value"));
+
+
+        // Create a VersionedFlowSnapshot that contains the processor
+        final Parameter parameter = new Parameter(new ParameterDescriptor.Builder().name("secret-param").sensitive(true).build(), null);
+        final VersionedFlowSnapshot versionedFlowWithParameterReference = createFlowSnapshot(Collections.emptyList(),
+            Collections.singletonList(processorWithParamRef), Collections.singleton(parameter));
+
+        final VersionedFlowSnapshot versionedFlowExplicitValue = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(processorWithExplicitValue), null);
+
+        // Create child group and update to the first version of the flow, with parameter ref
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        innerGroup.updateFlow(versionedFlowExplicitValue, null, true, true, true);
+
+        final ProcessorNode nodeInGroupWithRef = innerGroup.getProcessors().iterator().next();
+        assertNotNull(nodeInGroupWithRef.getProperty(UsernamePasswordProcessor.PASSWORD));
+
+
+        // Update the flow to new version that uses explicit value.
+        innerGroup.updateFlow(versionedFlowWithParameterReference, null, true, true, true);
+
+        // Updated flow has sensitive property that no longer references parameter. Now is an explicit value, so it should be unset
+        final ProcessorNode nodeInGroupWithNoValue = innerGroup.getProcessors().iterator().next();
+        assertEquals("#{secret-param}", nodeInGroupWithNoValue.getProperty(UsernamePasswordProcessor.PASSWORD).getRawValue());
+    }
+
+
+
+
+    private Set<FlowDifference> getLocalModifications(final ProcessGroup processGroup, final VersionedFlowSnapshot versionedFlowSnapshot) {
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(getFlowController().getExtensionManager());
+        final VersionedProcessGroup localGroup = mapper.mapProcessGroup(processGroup, getFlowController().getControllerServiceProvider(), getFlowController().getFlowRegistryClient(), true);
+        final VersionedProcessGroup registryGroup = versionedFlowSnapshot.getFlowContents();
+
+        final ComparableDataFlow localFlow = new StandardComparableDataFlow("Local Flow", localGroup);
+        final ComparableDataFlow registryFlow = new StandardComparableDataFlow("Versioned Flow", registryGroup);
+
+        final Set<String> ancestorServiceIds = getAncestorGroupServiceIds(processGroup);
+        final FlowComparator flowComparator = new StandardFlowComparator(registryFlow, localFlow, ancestorServiceIds, new ConciseEvolvingDifferenceDescriptor());
+        final FlowComparison flowComparison = flowComparator.compare();
+        final Set<FlowDifference> differences = flowComparison.getDifferences().stream()
+            .filter(difference -> difference.getDifferenceType() != DifferenceType.BUNDLE_CHANGED)
+            .filter(FlowDifferenceFilters.FILTER_ADDED_REMOVED_REMOTE_PORTS)
+            .filter(FlowDifferenceFilters.FILTER_PUBLIC_PORT_NAME_CHANGES)
+            .filter(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES)
+            .collect(Collectors.toCollection(HashSet::new));
+
+        return differences;
+    }
+
+    private Set<String> getAncestorGroupServiceIds(final ProcessGroup processGroup) {
+        final Set<String> ancestorServiceIds;
+        ProcessGroup parentGroup = processGroup.getParent();
+
+        if (parentGroup == null) {
+            ancestorServiceIds = Collections.emptySet();
+        } else {
+            ancestorServiceIds = parentGroup.getControllerServices(true).stream()
+                .map(cs -> {
+                    // We want to map the Controller Service to its Versioned Component ID, if it has one.
+                    // If it does not have one, we want to generate it in the same way that our Flow Mapper does
+                    // because this allows us to find the Controller Service when doing a Flow Diff.
+                    final Optional<String> versionedId = cs.getVersionedComponentId();
+                    if (versionedId.isPresent()) {
+                        return versionedId.get();
+                    }
+
+                    return UUID.nameUUIDFromBytes(cs.getIdentifier().getBytes(StandardCharsets.UTF_8)).toString();
+                })
+                .collect(Collectors.toSet());
+        }
+
+        return ancestorServiceIds;
+    }
+
+
+    private VersionedFlowSnapshot createFlowSnapshot(final List<ControllerServiceNode> controllerServices, final List<ProcessorNode> processors, final Set<Parameter> parameters) {
         final VersionedFlowSnapshotMetadata snapshotMetadata = new VersionedFlowSnapshotMetadata();
         snapshotMetadata.setAuthor("unit-test");
         snapshotMetadata.setBucketIdentifier("unit-test-bucket");
@@ -142,12 +469,14 @@ public class ImportFlowIT extends FrameworkIntegrationTest {
         for (final ProcessorNode processor : processors) {
             final VersionedProcessor versionedProcessor = flowMapper.mapProcessor(processor, getFlowController().getControllerServiceProvider(), Collections.emptySet(), new HashMap<>());
             versionedProcessors.add(versionedProcessor);
+            processor.setVersionedComponentId(versionedProcessor.getIdentifier());
         }
 
         final Set<VersionedControllerService> services = new HashSet<>();
         for (final ControllerServiceNode serviceNode : controllerServices) {
             final VersionedControllerService service = flowMapper.mapControllerService(serviceNode, getFlowController().getControllerServiceProvider(), Collections.emptySet(), new HashMap<>());
             services.add(service);
+            serviceNode.setVersionedComponentId(service.getIdentifier());
         }
 
         final VersionedProcessGroup flowContents = new VersionedProcessGroup();
@@ -164,17 +493,21 @@ public class ImportFlowIT extends FrameworkIntegrationTest {
 
         if (parameters != null) {
             final Set<VersionedParameter> versionedParameters = new HashSet<>();
-            for (final Map.Entry<String, String> entry : parameters.entrySet()) {
+            for (final Parameter parameter : parameters) {
                 final VersionedParameter versionedParameter = new VersionedParameter();
-                versionedParameter.setName(entry.getKey());
-                versionedParameter.setValue(entry.getValue());
+                versionedParameter.setName(parameter.getDescriptor().getName());
+                versionedParameter.setValue(parameter.getValue());
+                versionedParameter.setSensitive(parameter.getDescriptor().isSensitive());
+
                 versionedParameters.add(versionedParameter);
             }
 
             final VersionedParameterContext versionedParameterContext = new VersionedParameterContext();
             versionedParameterContext.setName("Unit Test Context");
             versionedParameterContext.setParameters(versionedParameters);
-            versionedFlowSnapshot.setParameterContexts(Collections.singletonMap("unit-test-context", versionedParameterContext));
+            versionedFlowSnapshot.setParameterContexts(Collections.singletonMap(versionedParameterContext.getName(), versionedParameterContext));
+
+            flowContents.setParameterContextName("Unit Test Context");
         }
 
         return versionedFlowSnapshot;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeParameterReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeParameterReference.java
index fd7f644..48cbec5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeParameterReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeParameterReference.java
@@ -22,14 +22,20 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
 import org.apache.nifi.parameter.ParameterParser;
 import org.apache.nifi.parameter.ParameterTokenList;
+import org.apache.nifi.registry.flow.VersionedParameter;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
+import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.FlowSnippetDTO;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
 
 import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 public class AuthorizeParameterReference {
 
@@ -124,4 +130,48 @@ public class AuthorizeParameterReference {
             }
         }
     }
+
+    /**
+     * Ensures that any Parameter Context that is referenced by the given VersionedParameterContext is readable by the given user. If the Versioned Parameter Context references a Parameter Context
+     * (by name) that does not exist in the current flow, ensures that the user has persmissions to create a new Parameter Context. If the Versioned Parameter Context contains any Parameters that
+     * do not currently exist in the Parameter Context that is referenced, ensures that the usre has permissions to WRITE to the Parameter Context so that the additional Parameter can be added.
+     *
+     * @param versionedParameterContext the Versioned Parameter Context
+     * @param serviceFacade the Service Facade
+     * @param authorizer the authorizer
+     * @param lookup the authorizable lookup
+     * @param user the user
+     */
+    public static void authorizeParameterContextAddition(final VersionedParameterContext versionedParameterContext, final NiFiServiceFacade serviceFacade, final Authorizer authorizer,
+                                                         final AuthorizableLookup lookup, final NiFiUser user) {
+        final ParameterContext parameterContext = serviceFacade.getParameterContextByName(versionedParameterContext.getName(), user);
+
+        if (parameterContext == null) {
+            // If Parameter Context does not yet exist, authorize that the user is allowed to create it.
+            lookup.getParameterContexts().authorize(authorizer, RequestAction.WRITE, user);
+            return;
+        }
+
+        // User must have READ permissions to the Parameter Context in order to use it
+        parameterContext.authorize(authorizer, RequestAction.READ, user);
+
+        // Parameter Context exists. Check if there are any new parameters that must be added.
+        final Set<String> existingParameterNames = parameterContext.getParameters().keySet().stream()
+            .map(ParameterDescriptor::getName)
+            .collect(Collectors.toSet());
+
+        boolean requiresAddition = false;
+        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
+            final String versionedParameterName = versionedParameter.getName();
+            if (!existingParameterNames.contains(versionedParameterName)) {
+                requiresAddition = true;
+                break;
+            }
+        }
+
+        if (requiresAddition) {
+            // User is required to have WRITE permission to the Parameter Context in order to add one or more parameters.
+            parameterContext.authorize(authorizer, RequestAction.WRITE, user);
+        }
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index 15720d8..f62bec2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -25,6 +25,7 @@ import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.repository.claim.ContentDirection;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.registry.flow.ExternalControllerServiceReference;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
@@ -84,6 +85,7 @@ import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.AffectedComponentEntity;
 import org.apache.nifi.web.api.entity.BucketEntity;
 import org.apache.nifi.web.api.entity.BulletinEntity;
+import org.apache.nifi.web.api.entity.ComponentValidationResultEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerBulletinsEntity;
@@ -96,7 +98,6 @@ import org.apache.nifi.web.api.entity.FlowConfigurationEntity;
 import org.apache.nifi.web.api.entity.FlowEntity;
 import org.apache.nifi.web.api.entity.FunnelEntity;
 import org.apache.nifi.web.api.entity.LabelEntity;
-import org.apache.nifi.web.api.entity.ComponentValidationResultEntity;
 import org.apache.nifi.web.api.entity.ParameterContextEntity;
 import org.apache.nifi.web.api.entity.PortEntity;
 import org.apache.nifi.web.api.entity.PortStatusEntity;
@@ -132,6 +133,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.function.Function;
+import java.util.function.Supplier;
 
 /**
  * Defines the NiFiServiceFacade interface.
@@ -1047,6 +1049,15 @@ public interface NiFiServiceFacade {
     Set<ParameterContextEntity> getParameterContexts();
 
     /**
+     * Returns the Parameter Context with the given name
+     * @param parameterContextName the name of the Parameter Context
+     * @return the Parameter Context with the given name, or <code>null</code> if no Parameter Context exists with that name
+     *
+     * @throws org.apache.nifi.authorization.AccessDeniedException if a Parameter Context exists with the given name but the user does not have READ permissions to it
+     */
+    ParameterContext getParameterContextByName(String parameterContextName, NiFiUser user);
+
+    /**
      * Returns the ParameterContextEntity for the ParameterContext with the given ID
      * @param parameterContextId the ID of the Parameter Context
      * @param user the user on whose behalf the Parameter Context is being retrieved
@@ -1547,10 +1558,11 @@ public interface NiFiServiceFacade {
      * @param updateSettings whether or not the process group's name and position should be updated
      * @param updateDescendantVersionedFlows if a child/descendant Process Group is under Version Control, specifies whether or not to
      *            update the contents of that Process Group
+     * @param  idGenerator the id generator
      * @return the Process Group
      */
     ProcessGroupEntity updateProcessGroupContents(Revision revision, String groupId, VersionControlInformationDTO versionControlInfo, VersionedFlowSnapshot snapshot,
-                                                  String componentIdSeed, boolean verifyNotModified, boolean updateSettings, boolean updateDescendantVersionedFlows);
+                                                  String componentIdSeed, boolean verifyNotModified, boolean updateSettings, boolean updateDescendantVersionedFlows, Supplier<String> idGenerator);
 
 
     /**
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 7594fa8..9f4744f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -1032,6 +1032,28 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return entities;
     }
 
+    @Override
+    public ParameterContext getParameterContextByName(final String parameterContextName, final NiFiUser user) {
+        final ParameterContext parameterContext = parameterContextDAO.getParameterContexts().stream()
+            .filter(context -> context.getName().equals(parameterContextName))
+            .findAny()
+            .orElse(null);
+
+        if (parameterContext == null) {
+            return null;
+        }
+
+        final boolean authorized = parameterContext.isAuthorized(authorizer, RequestAction.READ, user);
+        if (!authorized) {
+            // Note that we do not call ParameterContext.authorize() because doing so would result in an error message indicating that the user does not have permission
+            // to READ Parameter Context with ID ABC123, which tells the user that the Parameter Context ABC123 has the same name as the requested name. Instead, we simply indicate
+            // that the user is unable to read the Parameter Context and provide the name, rather than the ID, so that information about which ID corresponds to the given name is not provided.
+            throw new AccessDeniedException("Unable to read Parameter Context with name '" + parameterContextName + "'.");
+        }
+
+        return parameterContext;
+    }
+
     private ParameterContextEntity createParameterContextEntity(final ParameterContext parameterContext, final NiFiUser user) {
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterContext, user);
         final RevisionDTO revisionDto = dtoFactory.createRevisionDTO(revisionManager.getRevision(parameterContext.getIdentifier()));
@@ -4854,7 +4876,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     @Override
     public ProcessGroupEntity updateProcessGroupContents(final Revision revision, final String groupId, final VersionControlInformationDTO versionControlInfo,
-        final VersionedFlowSnapshot proposedFlowSnapshot, final String componentIdSeed, final boolean verifyNotModified, final boolean updateSettings, final boolean updateDescendantVersionedFlows) {
+                                                         final VersionedFlowSnapshot proposedFlowSnapshot, final String componentIdSeed, final boolean verifyNotModified,
+                                                         final boolean updateSettings, final boolean updateDescendantVersionedFlows, final Supplier<String> idGenerator) {
 
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
 
@@ -4868,7 +4891,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             @Override
             public RevisionUpdate<ProcessGroupDTO> update() {
                 // update the Process Group
-                processGroupDAO.updateProcessGroupFlow(groupId, proposedFlowSnapshot, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings, updateDescendantVersionedFlows);
+                final ProcessGroup updatedProcessGroup = processGroupDAO.updateProcessGroupFlow(groupId, proposedFlowSnapshot, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings,
+                    updateDescendantVersionedFlows);
 
                 // update the revisions
                 final Set<Revision> updatedRevisions = revisions.stream()
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index 6144ab2..af2bec0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -53,6 +53,7 @@ import org.apache.nifi.registry.flow.FlowRegistryUtils;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowState;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.registry.variable.VariableRegistryUpdateRequest;
 import org.apache.nifi.registry.variable.VariableRegistryUpdateStep;
 import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
@@ -1789,6 +1790,11 @@ public class ProcessGroupResource extends ApplicationResource {
                             final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
                             authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
                         });
+
+                        final Map<String, VersionedParameterContext> parameterContexts = versionedFlowSnapshot.getParameterContexts();
+                        if (parameterContexts != null) {
+                            parameterContexts.values().forEach(context -> AuthorizeParameterReference.authorizeParameterContextAddition(context, serviceFacade, authorizer, lookup, user));
+                        }
                     }
                 },
                 () -> {
@@ -1823,7 +1829,7 @@ public class ProcessGroupResource extends ApplicationResource {
                         // To accomplish this, we call updateProcessGroupContents() passing 'true' for the updateSettings flag but null out the position.
                         flowSnapshot.getFlowContents().setPosition(null);
                         entity = serviceFacade.updateProcessGroupContents(newGroupRevision, newGroupId, versionControlInfo, flowSnapshot,
-                                getIdGenerationSeed().orElse(null), false, true, true);
+                                getIdGenerationSeed().orElse(null), false, true, true, this::generateUuid);
                     }
 
                     populateRemainingProcessGroupEntityContent(entity);
@@ -1836,6 +1842,7 @@ public class ProcessGroupResource extends ApplicationResource {
     }
 
 
+
     private VersionedFlowSnapshot getFlowFromRegistry(final VersionControlInformationDTO versionControlInfo) {
         final VersionedFlowSnapshot flowSnapshot = serviceFacade.getVersionedFlowSnapshot(versionControlInfo, true);
         final Bucket bucket = flowSnapshot.getBucket();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
index 867467f..84744e2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
@@ -25,6 +25,7 @@ import io.swagger.annotations.ApiResponses;
 import io.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.AuthorizeParameterReference;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.ProcessGroupAuthorizable;
@@ -43,6 +44,7 @@ import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.flow.VersionedFlowState;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.ResourceNotFoundException;
@@ -832,7 +834,7 @@ public class VersionsResource extends ApplicationResource {
                 versionControlInfoDto.setState(flowState.name());
 
                 final ProcessGroupEntity updatedGroup = serviceFacade.updateProcessGroupContents(rev, groupId, versionControlInfoDto, flowSnapshot, getIdGenerationSeed().orElse(null), false,
-                    false, entity.getUpdateDescendantVersionedFlows());
+                    false, entity.getUpdateDescendantVersionedFlows(), this::generateUuid);
                 final VersionControlInformationDTO updatedVci = updatedGroup.getComponent().getVersionControlInformation();
 
                 final VersionControlInformationEntity responseEntity = new VersionControlInformationEntity();
@@ -1180,6 +1182,11 @@ public class VersionsResource extends ApplicationResource {
                     final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
                     authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
                 });
+
+                final Map<String, VersionedParameterContext> parameterContexts = flowSnapshot.getParameterContexts();
+                if (parameterContexts != null) {
+                    parameterContexts.values().forEach(context -> AuthorizeParameterReference.authorizeParameterContextAddition(context, serviceFacade, authorizer, lookup, user));
+                }
             },
             () -> {
                 // Step 3: Verify that all components in the snapshot exist on all nodes
@@ -1354,6 +1361,11 @@ public class VersionsResource extends ApplicationResource {
                     final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
                     authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
                 });
+
+                final Map<String, VersionedParameterContext> parameterContexts = flowSnapshot.getParameterContexts();
+                if (parameterContexts != null) {
+                    parameterContexts.values().forEach(context -> AuthorizeParameterReference.authorizeParameterContextAddition(context, serviceFacade, authorizer, lookup, user));
+                }
             },
             () -> {
                 // Step 3: Verify that all components in the snapshot exist on all nodes
@@ -1552,7 +1564,8 @@ public class VersionsResource extends ApplicationResource {
                 vci.setVersion(metadata.getVersion());
                 vci.setState(flowSnapshot.isLatest() ? VersionedFlowState.UP_TO_DATE.name() : VersionedFlowState.STALE.name());
 
-                serviceFacade.updateProcessGroupContents(revision, groupId, vci, flowSnapshot, idGenerationSeed, verifyNotModified, false, updateDescendantVersionedFlows);
+                serviceFacade.updateProcessGroupContents(revision, groupId, vci, flowSnapshot, idGenerationSeed, verifyNotModified, false, updateDescendantVersionedFlows,
+                    this::generateUuid);
             }
         } finally {
             if (!asyncRequest.isCancelled()) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 4211707..97b8c5e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -1405,7 +1405,9 @@ public final class DtoFactory {
         dto.setName(descriptor.getName());
         dto.setDescription(descriptor.getDescription());
         dto.setSensitive(descriptor.isSensitive());
-        dto.setValue(descriptor.isSensitive() ? SENSITIVE_VALUE_MASK : parameter.getValue());
+        if (parameter.getValue() != null) {
+            dto.setValue(descriptor.isSensitive() ? SENSITIVE_VALUE_MASK : parameter.getValue());
+        }
 
         final ParameterReferenceManager parameterReferenceManager = parameterContext.getParameterReferenceManager();