You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2022/05/16 21:55:12 UTC

[nifi] branch main updated: NIFI-9895 Allow parameters to reference controller services (#5958)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 2a8d07641e NIFI-9895 Allow parameters to reference controller services (#5958)
2a8d07641e is described below

commit 2a8d07641e63d84613286e63b68a25a5caf9b0ce
Author: tpalfy <53...@users.noreply.github.com>
AuthorDate: Mon May 16 23:55:03 2022 +0200

    NIFI-9895 Allow parameters to reference controller services (#5958)
    
    * NIFI-9895 Allow parameter to reference controller service. Check read and write authorization for both previous and newly set controller service. Authorization done for both property or parameter change. Import/export handled by switching between instance id and versioned id.
---
 .../StandardVersionedComponentSynchronizer.java    | 33 ++++++++++
 .../StandardParameterReferenceManager.java         | 73 ++++++++++++++++++++++
 .../flow/mapping/NiFiRegistryFlowMapper.java       | 53 +++++++++++++---
 .../parameter/TestStandardParameterContext.java    |  6 ++
 .../nifi/controller/AbstractComponentNode.java     | 25 +++-----
 .../nifi/parameter/ParameterReferenceManager.java  | 15 +++++
 .../ParameterReferencedControllerServiceData.java  | 63 +++++++++++++++++++
 .../nifi/integration/versioned/ImportFlowIT.java   | 32 ++++++++++
 .../flow/mapping/NiFiRegistryFlowMapperTest.java   |  7 ++-
 .../nifi/authorization/AuthorizableLookup.java     |  3 +-
 .../AuthorizeControllerServiceReference.java       | 57 ++++++++++++++---
 .../nifi/authorization/ComponentAuthorizable.java  |  3 +-
 .../authorization/StandardAuthorizableLookup.java  | 11 ++--
 .../nifi/web/api/ParameterContextResource.java     | 62 +++++++++++++++++-
 .../jquery/propertytable/jquery.propertytable.js   | 32 +++++-----
 15 files changed, 414 insertions(+), 61 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
index 98ab1b3dd0..4b42725211 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
@@ -78,6 +78,7 @@ import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
 import org.apache.nifi.parameter.ParameterReferenceManager;
+import org.apache.nifi.parameter.ParameterReferencedControllerServiceData;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.registry.ComponentVariableRegistry;
@@ -277,6 +278,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
             }
         }
 
+        boolean proposedParameterContextExistsBeforeSynchronize = getParameterContextByName(proposed.getParameterContextName()) != null;
+
         // Ensure that we create all Parameter Contexts before updating them. This is necessary in case the proposed incoming dataflow has
         // parameter contexts that inherit from one another and neither the inheriting nor inherited parameter context exists.
         if (versionedParameterContexts != null) {
@@ -412,6 +415,36 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
             removeTemporaryFunnel(group);
         }
 
+        Map<String, Parameter> newParameters = new HashMap<>();
+        if (!proposedParameterContextExistsBeforeSynchronize && this.context.getFlowMappingOptions().isMapControllerServiceReferencesToVersionedId()) {
+            Map<String, String> controllerServiceVersionedIdToId = group.getControllerServices(false)
+                .stream()
+                .filter(controllerServiceNode -> controllerServiceNode.getVersionedComponentId().isPresent())
+                .collect(Collectors.toMap(
+                    controllerServiceNode -> controllerServiceNode.getVersionedComponentId().get(),
+                    ComponentNode::getIdentifier
+                ));
+
+            ParameterContext parameterContext = group.getParameterContext();
+
+            if (parameterContext != null) {
+                parameterContext.getParameters().forEach((descriptor, parameter) -> {
+                    List<ParameterReferencedControllerServiceData> referencedControllerServiceData = parameterContext
+                        .getParameterReferenceManager()
+                        .getReferencedControllerServiceData(parameterContext, descriptor.getName());
+
+                    if (referencedControllerServiceData.isEmpty()) {
+                        newParameters.put(descriptor.getName(), parameter);
+                    } else {
+                        final Parameter adjustedParameter = new Parameter(parameter.getDescriptor(), controllerServiceVersionedIdToId.get(parameter.getValue()));
+                        newParameters.put(descriptor.getName(), adjustedParameter);
+                    }
+                });
+
+                parameterContext.setParameters(newParameters);
+            }
+        }
+
         // We can now add in any necessary connections, since all connectable components have now been created.
         synchronizeConnections(group, proposed, connectionsByVersionedId);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java
index 6f7c0946b1..5238285bc1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java
@@ -16,16 +16,21 @@
  */
 package org.apache.nifi.parameter;
 
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.VersionedComponent;
 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.flow.FlowManager;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.groups.ProcessGroup;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.function.Function;
 
@@ -48,6 +53,41 @@ public class StandardParameterReferenceManager implements ParameterReferenceMana
         return getComponentsReferencing(parameterContext, parameterName, group -> group.getControllerServices(false));
     }
 
+    @Override
+    public List<ParameterReferencedControllerServiceData> getReferencedControllerServiceData(final ParameterContext parameterContext, final String parameterName) {
+        final List<ParameterReferencedControllerServiceData> referencedControllerServiceData = new ArrayList<>();
+
+        final String versionedServiceId = parameterContext.getParameter(parameterName)
+            .map(Parameter::getValue)
+            .map(this.flowManager::getControllerServiceNode)
+            .flatMap(VersionedComponent::getVersionedComponentId)
+            .orElse(null);
+
+
+        final ProcessGroup rootGroup = flowManager.getRootGroup();
+        final List<ProcessGroup> referencingGroups = rootGroup.findAllProcessGroups(group -> group.referencesParameterContext(parameterContext));
+
+        for (final ProcessGroup group : referencingGroups) {
+            for (ProcessorNode processor : group.getProcessors()) {
+                referencedControllerServiceData.addAll(getReferencedControllerServiceData(
+                    processor,
+                    parameterName,
+                    versionedServiceId
+                ));
+            }
+
+            for (ControllerServiceNode controllerService : group.getControllerServices(false)) {
+                referencedControllerServiceData.addAll(getReferencedControllerServiceData(
+                    controllerService,
+                    parameterName,
+                    versionedServiceId
+                ));
+            }
+        }
+
+        return referencedControllerServiceData;
+    }
+
     @Override
     public Set<ProcessGroup> getProcessGroupsBound(final ParameterContext parameterContext) {
         final ProcessGroup rootGroup = flowManager.getRootGroup();
@@ -91,4 +131,37 @@ public class StandardParameterReferenceManager implements ParameterReferenceMana
         return false;
     }
 
+    private Set<ParameterReferencedControllerServiceData> getReferencedControllerServiceData(
+        final ComponentNode componentNode,
+        final String parameterName,
+        final String versionedServiceId
+    ) {
+        Set<ParameterReferencedControllerServiceData> referencedControllerServiceTypes = new HashSet<>();
+
+        for (Map.Entry<PropertyDescriptor, PropertyConfiguration> propertyDescriptorAndPropertyConfiguration : componentNode.getProperties().entrySet()) {
+            PropertyDescriptor descriptor = propertyDescriptorAndPropertyConfiguration.getKey();
+            PropertyConfiguration configuration = propertyDescriptorAndPropertyConfiguration.getValue();
+
+            Class<? extends ControllerService> referencedControllerServiceType = descriptor.getControllerServiceDefinition();
+
+            if (referencedControllerServiceType == null || configuration == null) {
+                continue;
+            }
+
+            for (final ParameterReference reference : configuration.getParameterReferences()) {
+                if (parameterName.equals(reference.getParameterName())) {
+                    referencedControllerServiceTypes.add(new ParameterReferencedControllerServiceData(
+                        parameterName,
+                        componentNode,
+                        descriptor,
+                        referencedControllerServiceType,
+                        versionedServiceId
+                    ));
+                }
+            }
+        }
+
+        return referencedControllerServiceTypes;
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
index 626544d202..20db633e06 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
@@ -65,6 +65,7 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterReferencedControllerServiceData;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.registry.VariableDescriptor;
 import org.apache.nifi.flow.ExternalControllerServiceReference;
@@ -760,11 +761,7 @@ public class NiFiRegistryFlowMapper {
     }
 
     public VersionedParameterContext mapParameterContext(final ParameterContext parameterContext) {
-        final Set<VersionedParameter> versionedParameters = new HashSet<>();
-        for (final Parameter parameter : parameterContext.getParameters().values()) {
-            final VersionedParameter versionedParameter = mapParameter(parameter);
-            versionedParameters.add(versionedParameter);
-        }
+        final Set<VersionedParameter> versionedParameters = mapParameters(parameterContext);
 
         final VersionedParameterContext versionedParameterContext = new VersionedParameterContext();
         versionedParameterContext.setDescription(parameterContext.getDescription());
@@ -807,9 +804,7 @@ public class NiFiRegistryFlowMapper {
 
     private void mapParameterContext(final ParameterContext parameterContext, final Map<String, VersionedParameterContext> parameterContexts) {
         // map this process group's parameter context and add to the collection
-        final Set<VersionedParameter> parameters = parameterContext.getParameters().values().stream()
-                .map(this::mapParameter)
-                .collect(Collectors.toSet());
+        final Set<VersionedParameter> parameters = mapParameters(parameterContext);
 
         final VersionedParameterContext versionedContext = new VersionedParameterContext();
         versionedContext.setName(parameterContext.getName());
@@ -822,7 +817,45 @@ public class NiFiRegistryFlowMapper {
         parameterContexts.put(versionedContext.getName(), versionedContext);
     }
 
+    private Set<VersionedParameter> mapParameters(ParameterContext parameterContext) {
+        final Set<VersionedParameter> parameters = parameterContext.getParameters().entrySet().stream()
+                .map(descriptorAndParameter -> mapParameter(
+                    parameterContext,
+                    descriptorAndParameter.getKey(),
+                    descriptorAndParameter.getValue())
+                )
+                .collect(Collectors.toSet());
+        return parameters;
+    }
+
+    private VersionedParameter mapParameter(ParameterContext parameterContext, ParameterDescriptor parameterDescriptor, Parameter parameter) {
+        VersionedParameter versionedParameter;
+
+        if (this.flowMappingOptions.isMapControllerServiceReferencesToVersionedId()) {
+            List<ParameterReferencedControllerServiceData> referencedControllerServiceData = parameterContext
+                .getParameterReferenceManager()
+                .getReferencedControllerServiceData(parameterContext, parameterDescriptor.getName());
+
+            if (referencedControllerServiceData.isEmpty()) {
+                versionedParameter = mapParameter(parameter);
+            } else {
+                versionedParameter = mapParameter(
+                    parameter,
+                    getId(Optional.ofNullable(referencedControllerServiceData.get(0).getVersionedServiceId()), parameter.getValue())
+                );
+            }
+        } else {
+            versionedParameter = mapParameter(parameter);
+        }
+
+        return versionedParameter;
+    }
+
     private VersionedParameter mapParameter(final Parameter parameter) {
+        return mapParameter(parameter, parameter.getValue());
+    }
+
+    private VersionedParameter mapParameter(final Parameter parameter, final String value) {
         if (parameter == null) {
             return null;
         }
@@ -838,9 +871,9 @@ public class NiFiRegistryFlowMapper {
         final String parameterValue;
         if (mapParameterValue) {
             if (descriptor.isSensitive()) {
-                parameterValue = encrypt(parameter.getValue());
+                parameterValue = encrypt(value);
             } else {
-                parameterValue = parameter.getValue();
+                parameterValue = value;
             }
         } else {
             parameterValue = null;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/parameter/TestStandardParameterContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/parameter/TestStandardParameterContext.java
index 0e826b7b5b..6032adf0d2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/parameter/TestStandardParameterContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/parameter/TestStandardParameterContext.java
@@ -30,6 +30,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -787,6 +788,11 @@ public class TestStandardParameterContext {
             return node == null ? Collections.emptySet() : Collections.singleton(node);
         }
 
+        @Override
+        public List<ParameterReferencedControllerServiceData> getReferencedControllerServiceData(ParameterContext parameterContext, String parameterName) {
+            return Collections.emptyList();
+        }
+
         @Override
         public Set<ProcessGroup> getProcessGroupsBound(final ParameterContext parameterContext) {
             return Collections.emptySet();
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 edf26e46da..59f9bd50c4 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
@@ -348,11 +348,6 @@ public abstract class AbstractComponentNode implements ComponentNode {
                     }
                 }
             }
-
-            if (descriptor.getControllerServiceDefinition() != null && !referenceList.isEmpty()) {
-                throw new IllegalArgumentException("The property '" + descriptor.getDisplayName() + "' cannot reference a Parameter because the property is a Controller Service reference. " +
-                    "Allowing Controller Service references to make use of Parameters could result in security issues and a poor user experience. As a result, this is not allowed.");
-            }
         }
     }
 
@@ -460,18 +455,14 @@ public abstract class AbstractComponentNode implements ComponentNode {
         // If it previously referenced a Controller Service, we need to also remove that reference.
         // It is okay if the new & old values are the same - we just unregister the component/descriptor and re-register it.
         if (descriptor.getControllerServiceDefinition() != null) {
-            if (oldConfiguration != null) {
-                final String oldEffectiveValue = oldConfiguration.getEffectiveValue(getParameterContext());
-                final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(oldEffectiveValue);
-                if (oldNode != null) {
-                    oldNode.removeReference(this, descriptor);
-                }
-            }
-
-            final ControllerServiceNode newNode = serviceProvider.getControllerServiceNode(effectiveValue);
-            if (newNode != null) {
-                newNode.addReference(this, descriptor);
-            }
+            Optional.ofNullable(oldConfiguration)
+                .map(_oldConfiguration -> _oldConfiguration.getEffectiveValue(getParameterContext()))
+                .map(oldEffectiveValue -> serviceProvider.getControllerServiceNode(oldEffectiveValue))
+                .ifPresent(oldNode -> oldNode.removeReference(this, descriptor));
+
+            Optional.ofNullable(effectiveValue)
+                .map(serviceProvider::getControllerServiceNode)
+                .ifPresent(newNode -> newNode.addReference(this, descriptor));
         }
 
         // In the case of a component "reload", we want to call onPropertyModified when the value is changed from the descriptor's default.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferenceManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferenceManager.java
index c1f724e771..fba0f241b3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferenceManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferenceManager.java
@@ -21,6 +21,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.groups.ProcessGroup;
 
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 
 /**
@@ -46,6 +47,15 @@ public interface ParameterReferenceManager {
      */
     Set<ControllerServiceNode> getControllerServicesReferencing(ParameterContext parameterContext, String parameterName);
 
+    /**
+     * Collects controller service reference data for a given parameter.
+     *
+     * @param  parameterContext the Parameter Context that the parameter belongs to
+     * @param parameterName the name of the parameter
+     * @return controller service reference data for the given parameter
+     */
+    List<ParameterReferencedControllerServiceData> getReferencedControllerServiceData(ParameterContext parameterContext, String parameterName);
+
     /**
      * Returns the set of all Process Groups that are bound to the given Parameter Context
      * @param parameterContext the Parameter Context
@@ -64,6 +74,11 @@ public interface ParameterReferenceManager {
             return Collections.emptySet();
         }
 
+        @Override
+        public List<ParameterReferencedControllerServiceData> getReferencedControllerServiceData(ParameterContext parameterContext, String parameterName) {
+            return Collections.emptyList();
+        }
+
         @Override
         public Set<ProcessGroup> getProcessGroupsBound(final ParameterContext parameterContext) {
             return Collections.emptySet();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferencedControllerServiceData.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferencedControllerServiceData.java
new file mode 100644
index 0000000000..c5ef27a4b3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterReferencedControllerServiceData.java
@@ -0,0 +1,63 @@
+/*
+ * 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.parameter;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.ControllerService;
+
+public class ParameterReferencedControllerServiceData {
+    private final String parameterName;
+    private final ComponentNode componentNode;
+    private final PropertyDescriptor descriptor;
+    private final Class<? extends ControllerService> referencedControllerServiceType;
+    private final String versionedServiceId;
+
+    public ParameterReferencedControllerServiceData(
+        String parameterName,
+        ComponentNode componentNode,
+        PropertyDescriptor descriptor,
+        Class<? extends ControllerService> referencedControllerServiceType,
+        String versionedServiceId
+    ) {
+        this.parameterName = parameterName;
+        this.componentNode = componentNode;
+        this.descriptor = descriptor;
+        this.referencedControllerServiceType = referencedControllerServiceType;
+        this.versionedServiceId = versionedServiceId;
+    }
+
+    public String getParameterName() {
+        return parameterName;
+    }
+
+    public ComponentNode getComponentNode() {
+        return componentNode;
+    }
+
+    public PropertyDescriptor getDescriptor() {
+        return descriptor;
+    }
+
+    public Class<? extends ControllerService> getReferencedControllerServiceType() {
+        return referencedControllerServiceType;
+    }
+
+    public String getVersionedServiceId() {
+        return versionedServiceId;
+    }
+}
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 27d6e4a2b1..6511360346 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,7 @@ import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.StandardSnippet;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.VersionedControllerService;
 import org.apache.nifi.flow.VersionedExternalFlow;
 import org.apache.nifi.flow.VersionedParameterContext;
 import org.apache.nifi.flow.VersionedProcessGroup;
@@ -34,6 +35,7 @@ import org.apache.nifi.groups.ProcessGroup;
 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.NopControllerService;
 import org.apache.nifi.integration.cs.NopServiceReferencingProcessor;
 import org.apache.nifi.integration.processors.GenerateProcessor;
 import org.apache.nifi.integration.processors.UsernamePasswordProcessor;
@@ -661,6 +663,36 @@ public class ImportFlowIT extends FrameworkIntegrationTest {
         assertEquals(inputPort.getName(), groupA.getInputPorts().stream().findFirst().get().getName());
     }
 
+    @Test
+    public void testExportImportFlowSwitchesVersionedIdToAndFromInstanceIdOfParameterReferencedControllerService() throws ExecutionException, InterruptedException {
+        ControllerServiceNode controllerService = createControllerServiceNode(NopControllerService.class);
+
+        ProcessorNode processor = createProcessorNode(NopServiceReferencingProcessor.class);
+        processor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS));
+        processor.setProperties(Collections.singletonMap(NopServiceReferencingProcessor.SERVICE.getName(), "#{service}"));
+
+        // Setting value to the instance id
+        Parameter parameter = new Parameter(new ParameterDescriptor.Builder()
+            .name("service")
+            .build(),
+            controllerService.getIdentifier()
+        );
+        setParameter(parameter);
+        VersionedExternalFlow flowSnapshot = createFlowSnapshot();
+
+        VersionedControllerService snapshotControllerService = flowSnapshot.getFlowContents().getControllerServices().stream().findAny().get();
+
+        assertEquals(controllerService.getIdentifier(), snapshotControllerService.getInstanceIdentifier());
+        // Exported flow contains versioned id instead of instance id
+        assertEquals(snapshotControllerService.getIdentifier(), flowSnapshot.getParameterContexts().get("unimportant").getParameters().stream().findAny().get().getValue());
+
+        getRootGroup().setParameterContext(null);
+        getRootGroup().updateFlow(flowSnapshot, null, false, true, true);
+
+        // Imported flow contains instance id again
+        assertEquals(snapshotControllerService.getInstanceIdentifier(), getRootGroup().getParameterContext().getParameter("service").get().getValue());
+    }
+
     private void setParameter(Parameter parameter) {
         ParameterContext rootParameterContext = getFlowController().getFlowManager().getParameterContextManager().getParameterContext("unimportant");
         if (rootParameterContext == null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java
index b5e52ae19e..e366635474 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java
@@ -76,6 +76,7 @@ import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+import org.mockito.Answers;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
@@ -95,6 +96,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -261,7 +263,10 @@ public class NiFiRegistryFlowMapperTest {
         final ProcessGroup processGroup = mock(ProcessGroup.class);
 
         if (includeParameterContext) {
-            final ParameterContext parameterContext = mock(ParameterContext.class);
+            final ParameterContext parameterContext = mock(ParameterContext.class, Answers.RETURNS_DEEP_STUBS);
+            when(parameterContext
+                .getParameterReferenceManager()
+                .getReferencedControllerServiceData(any(ParameterContext.class), anyString())).thenReturn(Collections.emptyList());
             when(processGroup.getParameterContext()).thenReturn(parameterContext);
             when(parameterContext.getName()).thenReturn("context" + (counter++));
             final Map<ParameterDescriptor, Parameter> parametersMap = new LinkedHashMap<>();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
index f3285847d6..eb9ed6e759 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
@@ -19,6 +19,7 @@ package org.apache.nifi.authorization;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.FlowSnippetDTO;
 
@@ -197,7 +198,7 @@ public interface AuthorizableLookup {
      * @param id the ID of the Parameter Context
      * @return authorizable
      */
-    Authorizable getParameterContext(String id);
+    ParameterContext getParameterContext(String id);
 
     /**
      * Get the authorizable for Parameter Contexts
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
index fb38782422..c4ba20e61d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
@@ -20,6 +20,8 @@ import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.PropertyConfigurationMapper;
 import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
 import org.apache.nifi.parameter.ParameterParser;
 import org.apache.nifi.parameter.ParameterTokenList;
@@ -27,6 +29,7 @@ import org.apache.nifi.web.ResourceNotFoundException;
 
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 
 /**
  * Authorizes references to Controller Services. Utilizes when Processors, Controller Services, and Reporting Tasks are created and updated.
@@ -88,9 +91,14 @@ public final class AuthorizeControllerServiceReference {
 
                 // if this descriptor identifies a controller service
                 if (propertyDescriptor.getControllerServiceDefinition() != null) {
-                    final String currentValue = authorizable.getValue(propertyDescriptor);
                     final String proposedValue = entry.getValue();
 
+                    String proposedEffectiveValue = new PropertyConfigurationMapper()
+                        .mapRawPropertyValuesToPropertyConfiguration(propertyDescriptor, proposedValue)
+                        .getEffectiveValue(authorizable.getParameterContext());
+
+                    final String currentValue = authorizable.getValue(propertyDescriptor);
+
                     // if the value is changing
                     if (!Objects.equals(currentValue, proposedValue)) {
                         // ensure access to the old service
@@ -109,17 +117,52 @@ public final class AuthorizeControllerServiceReference {
                             final ParameterTokenList tokenList = parser.parseTokens(proposedValue);
                             final boolean referencesParameter = !tokenList.toReferenceList().isEmpty();
                             if (referencesParameter) {
-                                throw new IllegalArgumentException("The property '" + propertyDescriptor.getDisplayName() + "' cannot reference a Parameter because the property is a " +
-                                    "Controller Service reference. Allowing Controller Service references to make use of Parameters could result in security issues and a poor user experience. " +
-                                    "As a result, this is not allowed.");
+                                authorizeControllerServiceReference(authorizable, authorizer, lookup, user, propertyDescriptor, proposedEffectiveValue);
+                            } else {
+                                final Authorizable newServiceAuthorizable = lookup.getControllerService(proposedValue).getAuthorizable();
+                                newServiceAuthorizable.authorize(authorizer, RequestAction.READ, user);
                             }
-
-                            final Authorizable newServiceAuthorizable = lookup.getControllerService(proposedValue).getAuthorizable();
-                            newServiceAuthorizable.authorize(authorizer, RequestAction.READ, user);
                         }
                     }
                 }
             }
         }
     }
+
+    /**
+     * Authorizes a proposed new controller service reference.
+     *
+     * @param authorizable authorizable that may reference a controller service
+     * @param authorizer authorizer
+     * @param lookup lookup
+     * @param user user
+     * @param propertyDescriptor the propertyDescriptor referencing a controller service
+     * @param proposedEffectiveValue the new proposed value (id of the controller service) to use as a reference
+     */
+    public static void authorizeControllerServiceReference(
+        ComponentAuthorizable authorizable,
+        Authorizer authorizer,
+        AuthorizableLookup lookup,
+        NiFiUser user,
+        PropertyDescriptor propertyDescriptor,
+        String proposedEffectiveValue
+    ) {
+        final String currentValue = authorizable.getValue(propertyDescriptor);
+
+        if (authorizable.getAuthorizable() instanceof ComponentNode) {
+            authorize(authorizer, lookup, user, currentValue);
+            authorize(authorizer, lookup, user, proposedEffectiveValue);
+        } else {
+            throw new IllegalArgumentException(authorizable.getAuthorizable().getResource().getSafeDescription() + " cannot reference Controller Services through Parameters.");
+        }
+    }
+
+    private static void authorize(Authorizer authorizer, AuthorizableLookup lookup, NiFiUser user, String serviceId) {
+        Optional.ofNullable(serviceId).map(lookup::getControllerService).ifPresent(service -> {
+            Authorizable serviceAuthorizable = service.getAuthorizable();
+
+            serviceAuthorizable.authorize(authorizer, RequestAction.READ, user);
+            serviceAuthorizable.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/authorization/ComponentAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java
index 46acc11155..61180da228 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java
@@ -18,6 +18,7 @@ package org.apache.nifi.authorization;
 
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.parameter.ParameterContext;
 
 import java.util.List;
 import java.util.Set;
@@ -50,7 +51,7 @@ public interface ComponentAuthorizable {
     /**
      * @return the Parameter Context that is bound to this component's Process Group, as an Authorizable, or <code>null</code> if no Parameter Context is bound
      */
-    Authorizable getParameterContext();
+    ParameterContext getParameterContext();
 
     /**
      * Returns the property descriptor for the specified property.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
index 4594ee05fa..41f9815a7e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
@@ -44,6 +44,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceReference;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.remote.PortAuthorizationResult;
 import org.apache.nifi.remote.PublicPort;
 import org.apache.nifi.util.BundleUtils;
@@ -273,7 +274,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     }
 
     @Override
-    public Authorizable getParameterContext(final String id) {
+    public ParameterContext getParameterContext(final String id) {
         return parameterContextDAO.getParameterContext(id);
     }
 
@@ -838,7 +839,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         @Override
-        public Authorizable getParameterContext() {
+        public ParameterContext getParameterContext() {
             return null;
         }
     }
@@ -871,7 +872,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         @Override
-        public Authorizable getParameterContext() {
+        public ParameterContext getParameterContext() {
             return processorNode.getProcessGroup().getParameterContext();
         }
 
@@ -929,7 +930,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         @Override
-        public Authorizable getParameterContext() {
+        public ParameterContext getParameterContext() {
             final ProcessGroup processGroup = controllerServiceNode.getProcessGroup();
             return processGroup == null ? null : processGroup.getParameterContext(); // will be null if Controller-level Controller Service.
         }
@@ -988,7 +989,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         @Override
-        public Authorizable getParameterContext() {
+        public ParameterContext getParameterContext() {
             return null;
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java
index de5d30c716..431082e42f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java
@@ -25,13 +25,19 @@ import io.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.controller.service.StandardControllerServiceNode;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterReferencedControllerServiceData;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.ResumeFlowException;
@@ -92,6 +98,7 @@ import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
@@ -390,12 +397,65 @@ public class ParameterContextResource extends ApplicationResource {
             requestRevision,
             lookup -> {
                 // Verify READ and WRITE permissions for user, for the Parameter Context itself
-                final Authorizable parameterContext = lookup.getParameterContext(contextId);
+                final ParameterContext parameterContext = lookup.getParameterContext(contextId);
                 parameterContext.authorize(authorizer, RequestAction.READ, user);
                 parameterContext.authorize(authorizer, RequestAction.WRITE, user);
 
                 // Verify READ and WRITE permissions for user, for every component that is affected
                 affectedComponents.forEach(component -> authorizeAffectedComponent(component, lookup, user, true, true));
+
+                Set<ParameterEntity> parametersEntities = requestEntity.getComponent().getParameters();
+                for (ParameterEntity parameterEntity : parametersEntities) {
+                    String parameterName = parameterEntity.getParameter().getName();
+                    List<ParameterReferencedControllerServiceData> referencedControllerServiceDataSet = parameterContext
+                        .getParameterReferenceManager()
+                        .getReferencedControllerServiceData(parameterContext, parameterName);
+
+                    Set<? extends Class<? extends ControllerService>> referencedControllerServiceTypes = referencedControllerServiceDataSet
+                        .stream()
+                        .map(ParameterReferencedControllerServiceData::getReferencedControllerServiceType)
+                        .collect(Collectors.toSet());
+
+                    if (referencedControllerServiceTypes.size() > 1) {
+                        throw new IllegalStateException("Parameter is used by multiple different types of controller service references");
+                    } else if (!referencedControllerServiceTypes.isEmpty()) {
+                        Optional<Parameter> parameterOptional = parameterContext.getParameter(parameterName);
+                        if (parameterOptional.isPresent()) {
+                            String currentParameterValue = parameterOptional.get().getValue();
+                            if (currentParameterValue != null) {
+                                ComponentAuthorizable currentControllerService = lookup.getControllerService(currentParameterValue);
+                                if (currentControllerService != null) {
+                                    Authorizable currentControllerServiceAuthorizable = currentControllerService.getAuthorizable();
+                                    if (currentControllerServiceAuthorizable != null) {
+                                        currentControllerServiceAuthorizable.authorize(authorizer, RequestAction.READ, user);
+                                        currentControllerServiceAuthorizable.authorize(authorizer, RequestAction.WRITE, user);
+                                    }
+                                }
+                            }
+                        }
+
+                        String newParameterValue = parameterEntity.getParameter().getValue();
+                        if (newParameterValue != null) {
+                            ComponentAuthorizable newControllerService = lookup.getControllerService(newParameterValue);
+                            if (newControllerService != null) {
+                                Authorizable newControllerServiceAuthorizable = newControllerService.getAuthorizable();
+                                if (newControllerServiceAuthorizable != null) {
+                                    newControllerServiceAuthorizable.authorize(authorizer, RequestAction.READ, user);
+                                    newControllerServiceAuthorizable.authorize(authorizer, RequestAction.WRITE, user);
+
+                                    if (
+                                        !referencedControllerServiceTypes.iterator().next()
+                                            .isAssignableFrom(
+                                                ((StandardControllerServiceNode) newControllerServiceAuthorizable).getComponent().getClass()
+                                            )
+                                    ) {
+                                        throw new IllegalArgumentException("New Parameter value attempts to reference an incompatible controller service");
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
             },
             () -> {
                 // Verify Request
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
index 7ffa885a1f..5ec357f7a6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
@@ -476,7 +476,7 @@
             };
             var CREATE_CONTROLLER_SERVICE_OPTION = {
                 text: 'Create new service...',
-                value: undefined,
+                value: 'createControllerService',
                 optionClass: 'unset'
             };
 
@@ -544,7 +544,7 @@
                 }
 
                 // if this does not represent an identify a controller service
-                if (parametersSupported && !nfCommon.isDefinedAndNotNull(propertyDescriptor.identifiesControllerService)) {
+                if (parametersSupported) {
                     allowableValueOptions.push(PARAMETER_REFERENCE_OPTION);
                 }
 
@@ -1320,23 +1320,19 @@
                     if (value === '') {
                         valueMarkup = '<span class="table-cell blank">Empty string set</span>';
                     } else {
-                        if (!resolvedAllowableValue && nfCommon.isDefinedAndNotNull(propertyDescriptor.identifiesControllerService)) {
-                            valueMarkup = '<span class="table-cell blank">Incompatible Controller Service Configured</div>';
-                        } else {
-                            valueWidthOffset = 10;
+                        valueWidthOffset = 10;
 
-                            // check for multi-line
-                            if (nfCommon.isMultiLine(value)) {
-                                valueMarkup = '<div class="table-cell value"><div class="ellipsis-white-space-pre multi-line-clamp-ellipsis">' + nfCommon.escapeHtml(value) + '</div></div>';
-                            } else {
-                                valueMarkup = '<div class="table-cell value"><div class="ellipsis-white-space-pre">' + nfCommon.escapeHtml(value) + '</div></div>';
-                            }
+                        // check for multi-line
+                        if (nfCommon.isMultiLine(value)) {
+                            valueMarkup = '<div class="table-cell value"><div class="ellipsis-white-space-pre multi-line-clamp-ellipsis">' + nfCommon.escapeHtml(value) + '</div></div>';
+                        } else {
+                            valueMarkup = '<div class="table-cell value"><div class="ellipsis-white-space-pre">' + nfCommon.escapeHtml(value) + '</div></div>';
+                        }
 
-                            // check for leading or trailing whitespace
-                            if (nfCommon.hasLeadTrailWhitespace(value)) {
-                                valueMarkup += '<div class="fa fa-info" alt="Info" style="float: right;"></div>';
-                                valueWidthOffset = 20;
-                            }
+                        // check for leading or trailing whitespace
+                        if (nfCommon.hasLeadTrailWhitespace(value)) {
+                            valueMarkup += '<div class="fa fa-info" alt="Info" style="float: right;"></div>';
+                            valueWidthOffset = 20;
                         }
                     }
                 }
@@ -1417,7 +1413,7 @@
             }
 
             if (options.readOnly !== true) {
-                if (canConvertPropertyToParam && !referencesParam && !identifiesControllerService) {
+                if (canConvertPropertyToParam && !referencesParam) {
                     markup += '<div title="Convert to parameter" class="convert-to-parameter pointer fa fa-level-up"></div>';
                 }