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 2021/10/23 23:42:46 UTC

[nifi] branch main updated: NIFI-9309: Include a uiOnly flag when requesting flow for a given pro… (#5462)

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 5766d33  NIFI-9309: Include a uiOnly flag when requesting flow for a given pro… (#5462)
5766d33 is described below

commit 5766d33fce1093fc4c532a684bc1d2e8410b09c4
Author: markap14 <ma...@hotmail.com>
AuthorDate: Sat Oct 23 19:42:31 2021 -0400

    NIFI-9309: Include a uiOnly flag when requesting flow for a given pro… (#5462)
    
    NIFI-9309: Include a uiOnly flag when requesting flow for a given process group. In that case, do not include the property descriptors, property values, etc. for Processors. When fetching Variable Registry, improved logic to cache the VariableImpact for each property value instead of parsing/recreating it every time. When fetching bulletins for a component, avoid filtering through all components' bulletins and instead only look at bulletins that might pertain to the appropriate component
---
 .../components/AbstractConfigurableComponent.java  |   2 +-
 .../apache/nifi/reporting/BulletinRepository.java  |  11 +
 .../apache/nifi/util/MockBulletinRepository.java   |   5 +
 .../apache/nifi/util/MockValidationContext.java    |   3 +-
 .../manager/StandardStateManagerProvider.java      |   8 +-
 .../nifi/events/VolatileBulletinRepository.java    |  78 +++++--
 .../apache/nifi/groups/StandardProcessGroup.java   |  84 ++++---
 .../processor/TestStandardValidationContext.java   |   3 +-
 .../nifi/controller/AbstractComponentNode.java     |   6 +-
 .../nifi/controller/PropertyConfiguration.java     |  11 +-
 .../controller/PropertyConfigurationMapper.java    |   8 +-
 .../org/apache/nifi/web/NiFiServiceFacade.java     |   3 +-
 .../apache/nifi/web/StandardNiFiServiceFacade.java |  10 +-
 .../java/org/apache/nifi/web/api/FlowResource.java |  10 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java    | 252 ++++++++++-----------
 .../src/main/webapp/js/nf/canvas/nf-canvas.js      |   5 +-
 .../stateless/engine/StandardStatelessEngine.java  |   3 +-
 17 files changed, 284 insertions(+), 218 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java b/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
index a14dcfc..2b0a061 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
@@ -229,7 +229,7 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
     @Override
     public final List<PropertyDescriptor> getPropertyDescriptors() {
         final List<PropertyDescriptor> supported = getSupportedPropertyDescriptors();
-        return supported == null ? Collections.emptyList() : new ArrayList<>(supported);
+        return supported == null ? Collections.emptyList() : Collections.unmodifiableList(supported);
     }
 
     @Override
diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java b/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
index 2d731fb..1bf5768 100644
--- a/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
+++ b/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
@@ -62,6 +62,17 @@ public interface BulletinRepository {
     List<Bulletin> findBulletinsForSource(String sourceId);
 
     /**
+     * Finds all bulletins for the specified source component that resides in the given group. While the
+     * {@link #findBulletinsForSource(String)} method may be used, this method is preferred when the ID of the group
+     * is known, as this is far more efficient.
+     *
+     * @param sourceId the id of the source component
+     * @param groupId the id of the process group
+     * @return bulletins for the given source
+     */
+    List<Bulletin> findBulletinsForSource(String sourceId, String groupId);
+
+    /**
      * Finds all bulletins for the specified group.
      *
      * @param groupId id of the group
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java
index 0ffd0f4..bafdfdb 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java
@@ -55,6 +55,11 @@ public class MockBulletinRepository implements BulletinRepository {
     }
 
     @Override
+    public List<Bulletin> findBulletinsForSource(final String sourceId, final String groupId) {
+        return null;
+    }
+
+    @Override
     public List<Bulletin> findBulletinsForGroupBySource(String groupId) {
         // TODO: Implement
         return null;
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
index a73d6eb..61bdd69 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.util;
 
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.components.validation.AbstractValidationContext;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
@@ -81,7 +82,7 @@ public class MockValidationContext extends MockControllerServiceLookup implement
             final PropertyDescriptor descriptor = processContext.getPropertyDescriptor(entry.getKey());
             final ParameterTokenList tokenList = new StandardParameterTokenList(entry.getValue(), Collections.emptyList());
             final List<ParameterReference> parameterReferences = Collections.emptyList();
-            final PropertyConfiguration configuration = new PropertyConfiguration(entry.getValue(), tokenList, parameterReferences);
+            final PropertyConfiguration configuration = new PropertyConfiguration(entry.getValue(), tokenList, parameterReferences, VariableImpact.NEVER_IMPACTED);
             configurationMap.put(descriptor, configuration);
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
index 99652b8..35c3180 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
@@ -18,7 +18,9 @@
 package org.apache.nifi.controller.state.manager;
 
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
@@ -232,7 +234,8 @@ public class StandardStateManagerProvider implements StateManagerProvider {
             propertyMap.put(descriptor, new StandardPropertyValue(resourceContext, descriptor.getDefaultValue(),null, parameterLookup, variableRegistry));
 
             final ParameterTokenList references = parser.parseTokens(descriptor.getDefaultValue());
-            final PropertyConfiguration configuration = new PropertyConfiguration(descriptor.getDefaultValue(), references, references.toReferenceList());
+            final VariableImpact variableImpact = Query.prepare(descriptor.getDefaultValue()).getVariableImpact();
+            final PropertyConfiguration configuration = new PropertyConfiguration(descriptor.getDefaultValue(), references, references.toReferenceList(), variableImpact);
 
             propertyStringMap.put(descriptor, configuration);
         }
@@ -242,7 +245,8 @@ public class StandardStateManagerProvider implements StateManagerProvider {
             final PropertyDescriptor descriptor = provider.getPropertyDescriptor(entry.getKey());
 
             final ParameterTokenList references = parser.parseTokens(entry.getValue());
-            final PropertyConfiguration configuration = new PropertyConfiguration(entry.getValue(), references, references.toReferenceList());
+            final VariableImpact variableImpact = Query.prepare(entry.getValue()).getVariableImpact();
+            final PropertyConfiguration configuration = new PropertyConfiguration(entry.getValue(), references, references.toReferenceList(), variableImpact);
 
             propertyStringMap.put(descriptor, configuration);
             final ResourceContext resourceContext = new StandardResourceContext(resourceReferenceFactory, descriptor);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
index d532625..9bdb119 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
@@ -24,6 +24,7 @@ import org.apache.nifi.util.RingBuffer;
 import org.apache.nifi.util.RingBuffer.Filter;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -32,6 +33,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
 
 public class VolatileBulletinRepository implements BulletinRepository {
 
@@ -70,14 +72,31 @@ public class VolatileBulletinRepository implements BulletinRepository {
 
     @Override
     public List<Bulletin> findBulletins(final BulletinQuery bulletinQuery) {
-        final Filter<Bulletin> filter = new Filter<Bulletin>() {
-            @Override
-            public boolean select(final Bulletin bulletin) {
-                final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
-                if (bulletin.getTimestamp().getTime() < fiveMinutesAgo) {
-                    return false;
+        final Filter<Bulletin> filter = createFilter(bulletinQuery);
+
+        final Set<Bulletin> selected = new TreeSet<>();
+        int max = bulletinQuery.getLimit() == null ? Integer.MAX_VALUE : bulletinQuery.getLimit();
+
+        for (final ConcurrentMap<String, RingBuffer<Bulletin>> componentMap : bulletinStoreMap.values()) {
+            for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) {
+                final List<Bulletin> bulletinsForComponent = ringBuffer.getSelectedElements(filter, max);
+                selected.addAll(bulletinsForComponent);
+                max -= bulletinsForComponent.size();
+                if (max <= 0) {
+                    break;
                 }
+            }
+        }
+
+        return new ArrayList<>(selected);
+    }
 
+    private Filter<Bulletin> createFilter(final BulletinQuery bulletinQuery) {
+        final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
+
+        return new Filter<Bulletin>() {
+            @Override
+            public boolean select(final Bulletin bulletin) {
                 // only include bulletins after the specified id
                 if (bulletinQuery.getAfter() != null && bulletin.getId() <= bulletinQuery.getAfter()) {
                     return false;
@@ -107,6 +126,10 @@ public class VolatileBulletinRepository implements BulletinRepository {
                     }
                 }
 
+                if (bulletin.getTimestamp().getTime() < fiveMinutesAgo) {
+                    return false;
+                }
+
                 // if a source id was specified see if it should be excluded
                 if (bulletinQuery.getSourceIdPattern() != null) {
                     // exclude if this bulletin doesn't have a source id or if it doesn't match
@@ -126,27 +149,40 @@ public class VolatileBulletinRepository implements BulletinRepository {
                 return true;
             }
         };
+    }
 
-        final Set<Bulletin> selected = new TreeSet<>();
-        int max = bulletinQuery.getLimit() == null ? Integer.MAX_VALUE : bulletinQuery.getLimit();
-
-        for (final ConcurrentMap<String, RingBuffer<Bulletin>> componentMap : bulletinStoreMap.values()) {
-            for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) {
-                final List<Bulletin> bulletinsForComponent = ringBuffer.getSelectedElements(filter, max);
-                selected.addAll(bulletinsForComponent);
-                max -= bulletinsForComponent.size();
-                if (max <= 0) {
-                    break;
-                }
-            }
+    @Override
+    public List<Bulletin> findBulletinsForSource(final String sourceId, final String groupId) {
+        final BulletinQuery bulletinQuery = new BulletinQuery.Builder().sourceIdMatches(Pattern.quote(sourceId)).groupIdMatches(Pattern.quote(groupId)).limit(COMPONENT_BUFFER_SIZE).build();
+        final ConcurrentMap<String, RingBuffer<Bulletin>> componentMap = bulletinStoreMap.get(groupId);
+        if (componentMap == null) {
+            return Collections.emptyList();
         }
 
-        return new ArrayList<>(selected);
+        return findBulletinsForSource(sourceId, bulletinQuery, Collections.singleton(componentMap));
     }
 
     @Override
-    public List<Bulletin> findBulletinsForSource(String sourceId) {
-        return findBulletins(new BulletinQuery.Builder().sourceIdMatches(sourceId).limit(COMPONENT_BUFFER_SIZE).build());
+    public List<Bulletin> findBulletinsForSource(final String sourceId) {
+        final BulletinQuery bulletinQuery = new BulletinQuery.Builder().sourceIdMatches(Pattern.quote(sourceId)).limit(COMPONENT_BUFFER_SIZE).build();
+        return findBulletinsForSource(sourceId, bulletinQuery, this.bulletinStoreMap.values());
+    }
+
+    private List<Bulletin> findBulletinsForSource(final String sourceId, final BulletinQuery bulletinQuery, final Collection<ConcurrentMap<String, RingBuffer<Bulletin>>> bulletinStoreMaps) {
+        final Filter<Bulletin> filter = createFilter(bulletinQuery);
+
+        final int max = bulletinQuery.getLimit() == null ? Integer.MAX_VALUE : bulletinQuery.getLimit();
+        for (final ConcurrentMap<String, RingBuffer<Bulletin>> componentMap : bulletinStoreMaps) {
+            final RingBuffer<Bulletin> ringBuffer = componentMap.get(sourceId);
+            if (ringBuffer == null) {
+                continue;
+            }
+
+            final List<Bulletin> bulletinsForComponent = ringBuffer.getSelectedElements(filter, max);
+            return bulletinsForComponent;
+        }
+
+        return Collections.emptyList();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 9c8d264..880c6d2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -22,8 +22,6 @@ import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
-import org.apache.nifi.attribute.expression.language.Query;
-import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
@@ -71,6 +69,22 @@ import org.apache.nifi.controller.service.ControllerServiceReference;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.flow.BatchSize;
+import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.ComponentType;
+import org.apache.nifi.flow.ConnectableComponent;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.flow.VersionedConnection;
+import org.apache.nifi.flow.VersionedControllerService;
+import org.apache.nifi.flow.VersionedFlowCoordinates;
+import org.apache.nifi.flow.VersionedFunnel;
+import org.apache.nifi.flow.VersionedLabel;
+import org.apache.nifi.flow.VersionedPort;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flow.VersionedPropertyDescriptor;
+import org.apache.nifi.flow.VersionedRemoteGroupPort;
+import org.apache.nifi.flow.VersionedRemoteProcessGroup;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
@@ -89,32 +103,16 @@ import org.apache.nifi.processor.StandardProcessContext;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.VariableDescriptor;
 import org.apache.nifi.registry.client.NiFiRegistryException;
-import org.apache.nifi.flow.BatchSize;
-import org.apache.nifi.flow.Bundle;
-import org.apache.nifi.flow.ComponentType;
-import org.apache.nifi.flow.ConnectableComponent;
 import org.apache.nifi.registry.flow.FlowRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.flow.StandardVersionControlInformation;
 import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.flow.VersionedComponent;
-import org.apache.nifi.flow.VersionedConnection;
-import org.apache.nifi.flow.VersionedControllerService;
 import org.apache.nifi.registry.flow.VersionedFlow;
-import org.apache.nifi.flow.VersionedFlowCoordinates;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowState;
 import org.apache.nifi.registry.flow.VersionedFlowStatus;
-import org.apache.nifi.flow.VersionedFunnel;
-import org.apache.nifi.flow.VersionedLabel;
 import org.apache.nifi.registry.flow.VersionedParameter;
 import org.apache.nifi.registry.flow.VersionedParameterContext;
-import org.apache.nifi.flow.VersionedPort;
-import org.apache.nifi.flow.VersionedProcessGroup;
-import org.apache.nifi.flow.VersionedProcessor;
-import org.apache.nifi.flow.VersionedPropertyDescriptor;
-import org.apache.nifi.flow.VersionedRemoteGroupPort;
-import org.apache.nifi.flow.VersionedRemoteProcessGroup;
 import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
 import org.apache.nifi.registry.flow.diff.DifferenceType;
 import org.apache.nifi.registry.flow.diff.EvolvingDifferenceDescriptor;
@@ -3316,11 +3314,9 @@ public final class StandardProcessGroup implements ProcessGroup {
                     continue;
                 }
 
-                for (final VariableImpact impact : getVariableImpact(processor)) {
-                    for (final String variableName : updatedVariableNames) {
-                        if (impact.isImpacted(variableName)) {
-                            throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + processor + ", which is currently running");
-                        }
+                for (final String variableName : updatedVariableNames) {
+                    if (isComponentImpactedByVariable(processor, variableName)) {
+                        throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + processor + ", which is currently running");
                     }
                 }
             }
@@ -3331,11 +3327,9 @@ public final class StandardProcessGroup implements ProcessGroup {
                     continue;
                 }
 
-                for (final VariableImpact impact : getVariableImpact(service)) {
-                    for (final String variableName : updatedVariableNames) {
-                        if (impact.isImpacted(variableName)) {
-                            throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + service + ", which is currently running");
-                        }
+                for (final String variableName : updatedVariableNames) {
+                    if (isComponentImpactedByVariable(service, variableName)) {
+                        throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + service + ", which is currently running");
                     }
                 }
             }
@@ -3408,10 +3402,8 @@ public final class StandardProcessGroup implements ProcessGroup {
 
         // Determine any Processors that references the variable
         for (final ProcessorNode processor : getProcessors()) {
-            for (final VariableImpact impact : getVariableImpact(processor)) {
-                if (impact.isImpacted(variableName)) {
-                    affected.add(processor);
-                }
+            if (isComponentImpactedByVariable(processor, variableName)) {
+                affected.add(processor);
             }
         }
 
@@ -3419,13 +3411,11 @@ public final class StandardProcessGroup implements ProcessGroup {
         // then that means that any other component that references that service is also affected, so recursively
         // find any references to that service and add it.
         for (final ControllerServiceNode service : getControllerServices(false)) {
-            for (final VariableImpact impact : getVariableImpact(service)) {
-                if (impact.isImpacted(variableName)) {
-                    affected.add(service);
+            if (isComponentImpactedByVariable(service, variableName)) {
+                affected.add(service);
 
-                    final ControllerServiceReference reference = service.getReferences();
-                    affected.addAll(reference.findRecursiveReferences(ComponentNode.class));
-                }
+                final ControllerServiceReference reference = service.getReferences();
+                affected.addAll(reference.findRecursiveReferences(ComponentNode.class));
             }
         }
 
@@ -3461,14 +3451,16 @@ public final class StandardProcessGroup implements ProcessGroup {
         return updatedVariableNames;
     }
 
-    private List<VariableImpact> getVariableImpact(final ComponentNode component) {
-        return component.getEffectivePropertyValues().keySet().stream()
-            .map(descriptor -> {
-                final String configuredVal = component.getEffectivePropertyValue(descriptor);
-                return configuredVal == null ? descriptor.getDefaultValue() : configuredVal;
-            })
-            .map(propVal -> Query.prepare(propVal).getVariableImpact())
-            .collect(Collectors.toList());
+    private boolean isComponentImpactedByVariable(final ComponentNode component, final String variableName) {
+        final List<PropertyDescriptor> propertyDescriptors = component.getPropertyDescriptors();
+        for (final PropertyDescriptor descriptor : propertyDescriptors) {
+            final PropertyConfiguration propertyConfiguration = component.getProperty(descriptor);
+            if (propertyConfiguration.getVariableImpact().isImpacted(variableName)) {
+                return true;
+            }
+        }
+
+        return false;
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/processor/TestStandardValidationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/processor/TestStandardValidationContext.java
index 70f6b3f..cb761fa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/processor/TestStandardValidationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/processor/TestStandardValidationContext.java
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.processor;
 
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.PropertyConfiguration;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
@@ -67,7 +68,7 @@ public class TestStandardValidationContext {
         // Property A's dependency is always satisfied b/c no dependency
         assertTrue(context.isDependencySatisfied(descriptorA, propertyLookup));
 
-        properties.put(descriptorA, new PropertyConfiguration("xyz", new StandardParameterTokenList("xyz", Collections.emptyList()), Collections.emptyList()));
+        properties.put(descriptorA, new PropertyConfiguration("xyz", new StandardParameterTokenList("xyz", Collections.emptyList()), Collections.emptyList(), VariableImpact.NEVER_IMPACTED));
         context = new StandardValidationContext(csProvider, properties, null, "1234", "12345", VariableRegistry.EMPTY_REGISTRY, null, false);
 
         // Should not be satisfied because A = "xyz".
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 399a484..7c5165f 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
@@ -19,6 +19,7 @@ package org.apache.nifi.controller;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigVerificationResult;
@@ -363,7 +364,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
                 final String rawValue = entry.getValue();
                 final String propertyValue = rawValue == null ? descriptor.getDefaultValue() : rawValue;
 
-                final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList());
+                final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList(), VariableImpact.NEVER_IMPACTED);
                 descriptorToConfigMap.put(descriptor, propertyConfiguration);
             }
 
@@ -609,7 +610,8 @@ public abstract class AbstractComponentNode implements ComponentNode {
     private PropertyConfiguration createPropertyConfiguration(final String value, final boolean supportsEL) {
         final ParameterParser parser = new ExpressionLanguageAwareParameterParser();
         final ParameterTokenList references = parser.parseTokens(value);
-        return new PropertyConfiguration(value, references, references.toReferenceList());
+        final VariableImpact variableImpact = Query.prepare(value).getVariableImpact();
+        return new PropertyConfiguration(value, references, references.toReferenceList(), variableImpact);
     }
 
     /**
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java
index d25a981..2e0761e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller;
 
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.parameter.ParameterReference;
 import org.apache.nifi.parameter.ParameterTokenList;
@@ -27,17 +28,23 @@ import java.util.Objects;
 import java.util.concurrent.atomic.AtomicReference;
 
 public class PropertyConfiguration {
-    public static PropertyConfiguration EMPTY = new PropertyConfiguration(null, new StandardParameterTokenList(null, Collections.emptyList()), Collections.emptyList());
+    public static PropertyConfiguration EMPTY = new PropertyConfiguration(null, new StandardParameterTokenList(null, Collections.emptyList()), Collections.emptyList(), VariableImpact.NEVER_IMPACTED);
 
     private final String rawValue;
     private final ParameterTokenList parameterTokenList;
     private final List<ParameterReference> parameterReferences;
+    private final VariableImpact variableImpact;
     private final AtomicReference<ComputedEffectiveValue> effectiveValue = new AtomicReference<>();
 
-    public PropertyConfiguration(final String rawValue, final ParameterTokenList tokenList, final List<ParameterReference> parameterReferences) {
+    public PropertyConfiguration(final String rawValue, final ParameterTokenList tokenList, final List<ParameterReference> parameterReferences, final VariableImpact variableImpact) {
         this.rawValue = rawValue;
         this.parameterTokenList = tokenList;
         this.parameterReferences = parameterReferences;
+        this.variableImpact = variableImpact;
+    }
+
+    public VariableImpact getVariableImpact() {
+        return variableImpact;
     }
 
     public String getRawValue() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfigurationMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfigurationMapper.java
index 9e10ebc..61d658a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfigurationMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfigurationMapper.java
@@ -17,6 +17,8 @@
 
 package org.apache.nifi.controller;
 
+import org.apache.nifi.attribute.expression.language.Query;
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
 import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
@@ -57,12 +59,12 @@ public class PropertyConfigurationMapper {
         final ParameterTokenList updatedValueReferences = elAgnosticParameterParser.parseTokens(updatedValue);
         final List<ParameterReference> parameterReferences = updatedValueReferences.toReferenceList();
 
-        final PropertyConfiguration propertyConfiguration;
         final boolean supportsEL = propertyDescriptor.isExpressionLanguageSupported();
         if (supportsEL) {
-            return new PropertyConfiguration(updatedValue, elAwareParameterParser.parseTokens(updatedValue), parameterReferences);
+            final VariableImpact variableImpact = Query.prepare(propertyValue).getVariableImpact();
+            return new PropertyConfiguration(updatedValue, elAwareParameterParser.parseTokens(updatedValue), parameterReferences, variableImpact);
         } else {
-            return new PropertyConfiguration(updatedValue, updatedValueReferences, parameterReferences);
+            return new PropertyConfiguration(updatedValue, updatedValueReferences, parameterReferences, VariableImpact.NEVER_IMPACTED);
         }
 
     }
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 9ad65a3..10f13cd 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
@@ -1009,9 +1009,10 @@ public interface NiFiServiceFacade {
      * Returns the flow.
      *
      * @param groupId group
+     * @param uiOnly whether or not the entity should be populated only with UI-related fields
      * @return the flow
      */
-    ProcessGroupFlowEntity getProcessGroupFlow(String groupId);
+    ProcessGroupFlowEntity getProcessGroupFlow(String groupId, boolean uiOnly);
 
     // ----------------------------------------
     // ProcessGroup methods
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 0d0ebe6..b22a417 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
@@ -4388,7 +4388,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
-    public ProcessGroupFlowEntity getProcessGroupFlow(final String groupId) {
+    public ProcessGroupFlowEntity getProcessGroupFlow(final String groupId, final boolean uiOnly) {
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
 
         // Get the Process Group Status but we only need a status depth of one because for any child process group,
@@ -4397,7 +4397,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         // doesn't include that anyway. So we can avoid including the information in the status that is returned.
         final ProcessGroupStatus groupStatus = controllerFacade.getProcessGroupStatus(groupId, 1);
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup);
-        return entityFactory.createProcessGroupFlowEntity(dtoFactory.createProcessGroupFlowDto(processGroup, groupStatus, revisionManager, this::getProcessGroupBulletins), permissions);
+        return entityFactory.createProcessGroupFlowEntity(dtoFactory.createProcessGroupFlowDto(processGroup, groupStatus, revisionManager, this::getProcessGroupBulletins, uiOnly), permissions);
     }
 
     @Override
@@ -4432,18 +4432,22 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     private VariableRegistryEntity createVariableRegistryEntity(final ProcessGroup processGroup, final boolean includeAncestorGroups) {
+        final Set<String> variablesToIgnore = new HashSet<>();
+
         final VariableRegistryDTO registryDto = dtoFactory.createVariableRegistryDto(processGroup, revisionManager);
         final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier()));
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup);
+        registryDto.getVariables().forEach(entity -> variablesToIgnore.add(entity.getVariable().getName()));
 
         if (includeAncestorGroups) {
             ProcessGroup parent = processGroup.getParent();
             while (parent != null) {
                 final PermissionsDTO parentPerms = dtoFactory.createPermissionsDto(parent);
                 if (Boolean.TRUE.equals(parentPerms.getCanRead())) {
-                    final VariableRegistryDTO parentRegistryDto = dtoFactory.createVariableRegistryDto(parent, revisionManager);
+                    final VariableRegistryDTO parentRegistryDto = dtoFactory.createVariableRegistryDto(parent, revisionManager, variablesToIgnore);
                     final Set<VariableEntity> parentVariables = parentRegistryDto.getVariables();
                     registryDto.getVariables().addAll(parentVariables);
+                    registryDto.getVariables().forEach(entity -> variablesToIgnore.add(entity.getVariable().getName()));
                 }
 
                 parent = parent.getParent();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index 0816c42..35a7c81 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -346,7 +346,6 @@ public class FlowResource extends ApplicationResource {
      *
      * @param groupId The id of the process group.
      * @return A processGroupEntity.
-     * @throws InterruptedException if interrupted
      */
     @GET
     @Consumes(MediaType.WILDCARD)
@@ -357,7 +356,9 @@ public class FlowResource extends ApplicationResource {
             response = ProcessGroupFlowEntity.class,
             authorizations = {
                     @Authorization(value = "Read - /flow")
-            }
+            },
+            notes = "If the uiOnly query parameter is provided with a value of true, the returned entity may only contain fields that are necessary for rendering the NiFi User Interface. As such, " +
+                "the selected fields may change at any time, even during incremental releases, without warning. As a result, this parameter should not be provided by any client other than the UI."
     )
     @ApiResponses(
             value = {
@@ -373,7 +374,8 @@ public class FlowResource extends ApplicationResource {
                     value = "The process group id.",
                     required = false
             )
-            @PathParam("id") String groupId) throws InterruptedException {
+            @PathParam("id") final String groupId,
+            @QueryParam("uiOnly") @DefaultValue("false") final boolean uiOnly) {
 
         authorizeFlow();
 
@@ -382,7 +384,7 @@ public class FlowResource extends ApplicationResource {
         }
 
         // get this process group flow
-        final ProcessGroupFlowEntity entity = serviceFacade.getProcessGroupFlow(groupId);
+        final ProcessGroupFlowEntity entity = serviceFacade.getProcessGroupFlow(groupId, uiOnly);
         populateRemainingFlowContent(entity.getProcessGroupFlow());
         return generateOkResponse(entity).build();
     }
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 6408055..6a6ba16 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
@@ -116,6 +116,8 @@ import org.apache.nifi.diagnostics.GarbageCollection;
 import org.apache.nifi.diagnostics.StorageUsage;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.groups.ProcessGroup;
@@ -142,10 +144,8 @@ import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.VariableDescriptor;
 import org.apache.nifi.registry.flow.FlowRegistry;
 import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.flow.VersionedComponent;
 import org.apache.nifi.registry.flow.VersionedFlowState;
 import org.apache.nifi.registry.flow.VersionedFlowStatus;
-import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.registry.flow.diff.DifferenceType;
 import org.apache.nifi.registry.flow.diff.FlowComparison;
 import org.apache.nifi.registry.flow.diff.FlowDifference;
@@ -267,7 +267,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
-import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 public final class DtoFactory {
@@ -2235,13 +2234,13 @@ public final class DtoFactory {
     }
 
     public ProcessGroupFlowDTO createProcessGroupFlowDto(final ProcessGroup group, final ProcessGroupStatus groupStatus, final RevisionManager revisionManager,
-                                                         final Function<ProcessGroup, List<BulletinEntity>> getProcessGroupBulletins) {
+                                                         final Function<ProcessGroup, List<BulletinEntity>> getProcessGroupBulletins, final boolean uiOnly) {
 
         final ProcessGroupFlowDTO dto = new ProcessGroupFlowDTO();
         dto.setId(group.getIdentifier());
         dto.setLastRefreshed(new Date());
         dto.setBreadcrumb(createBreadcrumbEntity(group));
-        dto.setFlow(createFlowDto(group, groupStatus, revisionManager, getProcessGroupBulletins));
+        dto.setFlow(createFlowDto(group, groupStatus, revisionManager, getProcessGroupBulletins, uiOnly));
 
         final ProcessGroup parent = group.getParent();
         if (parent != null) {
@@ -2275,6 +2274,7 @@ public final class DtoFactory {
 
         final FlowDTO flow = new FlowDTO();
 
+        final Map<String, ConnectionStatus> connectionStatuses = groupStatus.getConnectionStatus().stream().collect(Collectors.toMap(ConnectionStatus::getId, status -> status));
         for (final ConnectionDTO snippetConnection : snippet.getConnections()) {
             final Connection connection = group.getConnection(snippetConnection.getId());
 
@@ -2282,11 +2282,9 @@ public final class DtoFactory {
             final ConnectionDTO dto = createConnectionDto(connection);
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(connection.getIdentifier()));
             final PermissionsDTO accessPolicy = createPermissionsDto(connection);
-            final ConnectionStatusDTO status = getComponentStatus(
-                () -> groupStatus.getConnectionStatus().stream().filter(connectionStatus -> connection.getIdentifier().equals(connectionStatus.getId())).findFirst().orElse(null),
-                this::createConnectionStatusDto
-            );
-            flow.getConnections().add(entityFactory.createConnectionEntity(dto, revision, accessPolicy, status));
+            final ConnectionStatus connectionStatus = connectionStatuses.get(connection.getIdentifier());
+            final ConnectionStatusDTO statusDto = connectionStatus == null ? null : createConnectionStatusDto(connectionStatus);
+            flow.getConnections().add(entityFactory.createConnectionEntity(dto, revision, accessPolicy, statusDto));
         }
 
         for (final FunnelDTO snippetFunnel : snippet.getFunnels()) {
@@ -2299,6 +2297,7 @@ public final class DtoFactory {
             flow.getFunnels().add(entityFactory.createFunnelEntity(dto, revision, accessPolicy));
         }
 
+        final Map<String, PortStatus> inputPortStatuses = groupStatus.getInputPortStatus().stream().collect(Collectors.toMap(PortStatus::getId, status -> status));
         for (final PortDTO snippetInputPort : snippet.getInputPorts()) {
             final Port inputPort = group.getInputPort(snippetInputPort.getId());
 
@@ -2307,15 +2306,14 @@ public final class DtoFactory {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(inputPort.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(inputPort);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(inputPort));
-            final PortStatusDTO status = getComponentStatus(
-                () -> groupStatus.getInputPortStatus().stream().filter(inputPortStatus -> inputPort.getIdentifier().equals(inputPortStatus.getId())).findFirst().orElse(null),
-                inputPortStatus -> createPortStatusDto(inputPortStatus)
-            );
+            final PortStatus portStatus = inputPortStatuses.get(inputPort.getIdentifier());
+            final PortStatusDTO statusDto = portStatus == null ? null : createPortStatusDto(portStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(inputPort.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            flow.getInputPorts().add(entityFactory.createPortEntity(dto, revision, permissions, operatePermissions, status, bulletinEntities));
+            flow.getInputPorts().add(entityFactory.createPortEntity(dto, revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
+        final Map<String, PortStatus> outputPortStatuses = groupStatus.getOutputPortStatus().stream().collect(Collectors.toMap(PortStatus::getId, status -> status));
         for (final PortDTO snippetOutputPort : snippet.getOutputPorts()) {
             final Port outputPort = group.getOutputPort(snippetOutputPort.getId());
 
@@ -2324,13 +2322,11 @@ public final class DtoFactory {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(outputPort.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(outputPort);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(outputPort));
-            final PortStatusDTO status = getComponentStatus(
-                () -> groupStatus.getOutputPortStatus().stream().filter(outputPortStatus -> outputPort.getIdentifier().equals(outputPortStatus.getId())).findFirst().orElse(null),
-                outputPortStatus -> createPortStatusDto(outputPortStatus)
-            );
+            final PortStatus portStatus = outputPortStatuses.get(outputPort.getIdentifier());
+            final PortStatusDTO statusDto = portStatus == null ? null : createPortStatusDto(portStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(outputPort.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            flow.getOutputPorts().add(entityFactory.createPortEntity(dto, revision, permissions, operatePermissions, status, bulletinEntities));
+            flow.getOutputPorts().add(entityFactory.createPortEntity(dto, revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
         for (final LabelDTO snippetLabel : snippet.getLabels()) {
@@ -2343,6 +2339,7 @@ public final class DtoFactory {
             flow.getLabels().add(entityFactory.createLabelEntity(dto, revision, accessPolicy));
         }
 
+        final Map<String, ProcessGroupStatus> groupStatuses = groupStatus.getProcessGroupStatus().stream().collect(Collectors.toMap(ProcessGroupStatus::getId, status -> status));
         for (final ProcessGroupDTO snippetProcessGroup : snippet.getProcessGroups()) {
             final ProcessGroup processGroup = group.getProcessGroup(snippetProcessGroup.getId());
 
@@ -2350,14 +2347,13 @@ public final class DtoFactory {
             final ProcessGroupDTO dto = createProcessGroupDto(processGroup);
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(processGroup);
-            final ProcessGroupStatusDTO status = getComponentStatus(
-                () -> groupStatus.getProcessGroupStatus().stream().filter(processGroupStatus -> processGroup.getIdentifier().equals(processGroupStatus.getId())).findFirst().orElse(null),
-                processGroupStatus -> createConciseProcessGroupStatusDto(processGroupStatus)
-            );
+            final ProcessGroupStatus childGroupStatus = groupStatuses.get(processGroup.getIdentifier());
+            final ProcessGroupStatusDTO statusDto = childGroupStatus == null ? null : createConciseProcessGroupStatusDto(childGroupStatus);
             final List<BulletinEntity> bulletins = getProcessGroupBulletins.apply(processGroup);
-            flow.getProcessGroups().add(entityFactory.createProcessGroupEntity(dto, revision, permissions, status, bulletins));
+            flow.getProcessGroups().add(entityFactory.createProcessGroupEntity(dto, revision, permissions, statusDto, bulletins));
         }
 
+        final Map<String, ProcessorStatus> processorStatuses = groupStatus.getProcessorStatus().stream().collect(Collectors.toMap(ProcessorStatus::getId, status -> status));
         for (final ProcessorDTO snippetProcessor : snippet.getProcessors()) {
             final ProcessorNode processor = group.getProcessor(snippetProcessor.getId());
 
@@ -2366,15 +2362,14 @@ public final class DtoFactory {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(processor.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(processor);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(processor));
-            final ProcessorStatusDTO status = getComponentStatus(
-                () -> groupStatus.getProcessorStatus().stream().filter(processorStatus -> processor.getIdentifier().equals(processorStatus.getId())).findFirst().orElse(null),
-                processorStatus -> createProcessorStatusDto(processorStatus)
-            );
+            final ProcessorStatus processorStatus = processorStatuses.get(processor.getIdentifier());
+            final ProcessorStatusDTO statusDto = processorStatus == null ? null : createProcessorStatusDto(processorStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(processor.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            flow.getProcessors().add(entityFactory.createProcessorEntity(dto, revision, permissions, operatePermissions, status, bulletinEntities));
+            flow.getProcessors().add(entityFactory.createProcessorEntity(dto, revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
+        final Map<String, RemoteProcessGroupStatus> rpgStatuses = groupStatus.getRemoteProcessGroupStatus().stream().collect(Collectors.toMap(RemoteProcessGroupStatus::getId, status -> status));
         for (final RemoteProcessGroupDTO snippetRemoteProcessGroup : snippet.getRemoteProcessGroups()) {
             final RemoteProcessGroup remoteProcessGroup = group.getRemoteProcessGroup(snippetRemoteProcessGroup.getId());
 
@@ -2383,54 +2378,40 @@ public final class DtoFactory {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(remoteProcessGroup.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(remoteProcessGroup);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(remoteProcessGroup));
-            final RemoteProcessGroupStatusDTO status = getComponentStatus(
-                () -> groupStatus.getRemoteProcessGroupStatus().stream().filter(rpgStatus -> remoteProcessGroup.getIdentifier().equals(rpgStatus.getId())).findFirst().orElse(null),
-                remoteProcessGroupStatus -> createRemoteProcessGroupStatusDto(remoteProcessGroup, remoteProcessGroupStatus)
-            );
+            final RemoteProcessGroupStatus rpgStatus = rpgStatuses.get(remoteProcessGroup.getIdentifier());
+            final RemoteProcessGroupStatusDTO statusDto = rpgStatus == null ? null : createRemoteProcessGroupStatusDto(remoteProcessGroup, rpgStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(remoteProcessGroup.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            flow.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(dto, revision, permissions, operatePermissions, status, bulletinEntities));
+            flow.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(dto, revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
         return flow;
     }
 
-    private <T, S> T getComponentStatus(final Supplier<S> getComponentStatus, final Function<S, T> convertToDto) {
-        final T statusDTO;
-        final S status = getComponentStatus.get();
-        if (status != null) {
-            statusDTO = convertToDto.apply(status);
-        } else {
-            statusDTO = null;
-        }
-        return statusDTO;
-    }
 
     public FlowDTO createFlowDto(final ProcessGroup group, final ProcessGroupStatus groupStatus, final RevisionManager revisionManager,
-                                 final Function<ProcessGroup, List<BulletinEntity>> getProcessGroupBulletins) {
+                                 final Function<ProcessGroup, List<BulletinEntity>> getProcessGroupBulletins, final boolean uiOnly) {
         final FlowDTO dto = new FlowDTO();
 
+        final Map<String, ProcessorStatus> processorStatuses = groupStatus.getProcessorStatus().stream().collect(Collectors.toMap(ProcessorStatus::getId, status -> status));
         for (final ProcessorNode procNode : group.getProcessors()) {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(procNode.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(procNode);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(procNode));
-            final ProcessorStatusDTO status = getComponentStatus(
-                () -> groupStatus.getProcessorStatus().stream().filter(processorStatus -> procNode.getIdentifier().equals(processorStatus.getId())).findFirst().orElse(null),
-                processorStatus -> createProcessorStatusDto(processorStatus)
-            );
+            final ProcessorStatus processorStatus = processorStatuses.get(procNode.getIdentifier());
+            final ProcessorStatusDTO statusDto = processorStatus == null ? null : createProcessorStatusDto(processorStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(procNode.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            dto.getProcessors().add(entityFactory.createProcessorEntity(createProcessorDto(procNode), revision, permissions, operatePermissions, status, bulletinEntities));
+            dto.getProcessors().add(entityFactory.createProcessorEntity(createProcessorDto(procNode, uiOnly), revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
+        final Map<String, ConnectionStatus> connectionStatuses = groupStatus.getConnectionStatus().stream().collect(Collectors.toMap(ConnectionStatus::getId, status -> status));
         for (final Connection connNode : group.getConnections()) {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(connNode.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(connNode);
-            final ConnectionStatusDTO status = getComponentStatus(
-                () -> groupStatus.getConnectionStatus().stream().filter(connectionStatus -> connNode.getIdentifier().equals(connectionStatus.getId())).findFirst().orElse(null),
-                connectionStatus -> createConnectionStatusDto(connectionStatus)
-            );
-            dto.getConnections().add(entityFactory.createConnectionEntity(createConnectionDto(connNode), revision, permissions, status));
+            final ConnectionStatus connectionStatus = connectionStatuses.get(connNode.getIdentifier());
+            final ConnectionStatusDTO statusDto = connectionStatus == null ? null : createConnectionStatusDto(connectionStatus);
+            dto.getConnections().add(entityFactory.createConnectionEntity(createConnectionDto(connNode), revision, permissions, statusDto));
         }
 
         for (final Label label : group.getLabels()) {
@@ -2445,54 +2426,50 @@ public final class DtoFactory {
             dto.getFunnels().add(entityFactory.createFunnelEntity(createFunnelDto(funnel), revision, permissions));
         }
 
+        final Map<String, ProcessGroupStatus> groupStatuses = groupStatus.getProcessGroupStatus().stream().collect(Collectors.toMap(ProcessGroupStatus::getId, status -> status));
         for (final ProcessGroup childGroup : group.getProcessGroups()) {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(childGroup.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(childGroup);
-            final ProcessGroupStatusDTO status = getComponentStatus(
-                () -> groupStatus.getProcessGroupStatus().stream().filter(processGroupStatus -> childGroup.getIdentifier().equals(processGroupStatus.getId())).findFirst().orElse(null),
-                processGroupStatus -> createConciseProcessGroupStatusDto(processGroupStatus)
-            );
+            final ProcessGroupStatus childGroupStatus = groupStatuses.get(childGroup.getIdentifier());
+            final ProcessGroupStatusDTO statusDto = childGroupStatus == null ? null : createConciseProcessGroupStatusDto(childGroupStatus);
             final List<BulletinEntity> bulletins = getProcessGroupBulletins.apply(childGroup);
-            dto.getProcessGroups().add(entityFactory.createProcessGroupEntity(createProcessGroupDto(childGroup), revision, permissions, status, bulletins));
+            dto.getProcessGroups().add(entityFactory.createProcessGroupEntity(createProcessGroupDto(childGroup), revision, permissions, statusDto, bulletins));
         }
 
+        final Map<String, RemoteProcessGroupStatus> rpgStatuses = groupStatus.getRemoteProcessGroupStatus().stream().collect(Collectors.toMap(RemoteProcessGroupStatus::getId, status -> status));
         for (final RemoteProcessGroup rpg : group.getRemoteProcessGroups()) {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(rpg.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(rpg);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(rpg));
-            final RemoteProcessGroupStatusDTO status = getComponentStatus(
-                () -> groupStatus.getRemoteProcessGroupStatus().stream().filter(remoteProcessGroupStatus -> rpg.getIdentifier().equals(remoteProcessGroupStatus.getId())).findFirst().orElse(null),
-                remoteProcessGroupStatus -> createRemoteProcessGroupStatusDto(rpg, remoteProcessGroupStatus)
-            );
+            final RemoteProcessGroupStatus rpgStatus = rpgStatuses.get(rpg.getIdentifier());
+            final RemoteProcessGroupStatusDTO statusDto = rpgStatus == null ? null : createRemoteProcessGroupStatusDto(rpg, rpgStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(rpg.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            dto.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(createRemoteProcessGroupDto(rpg), revision, permissions, operatePermissions, status, bulletinEntities));
+            dto.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(createRemoteProcessGroupDto(rpg), revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
+        final Map<String, PortStatus> inputPortStatuses = groupStatus.getInputPortStatus().stream().collect(Collectors.toMap(PortStatus::getId, status -> status));
         for (final Port inputPort : group.getInputPorts()) {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(inputPort.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(inputPort);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(inputPort));
-            final PortStatusDTO status = getComponentStatus(
-                () -> groupStatus.getInputPortStatus().stream().filter(inputPortStatus -> inputPort.getIdentifier().equals(inputPortStatus.getId())).findFirst().orElse(null),
-                inputPortStatus -> createPortStatusDto(inputPortStatus)
-            );
+            final PortStatus portStatus = inputPortStatuses.get(inputPort.getIdentifier());
+            final PortStatusDTO statusDto = portStatus == null ? null : createPortStatusDto(portStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(inputPort.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            dto.getInputPorts().add(entityFactory.createPortEntity(createPortDto(inputPort), revision, permissions, operatePermissions, status, bulletinEntities));
+            dto.getInputPorts().add(entityFactory.createPortEntity(createPortDto(inputPort), revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
+        final Map<String, PortStatus> outputPortStatuses = groupStatus.getOutputPortStatus().stream().collect(Collectors.toMap(PortStatus::getId, status -> status));
         for (final Port outputPort : group.getOutputPorts()) {
             final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(outputPort.getIdentifier()));
             final PermissionsDTO permissions = createPermissionsDto(outputPort);
             final PermissionsDTO operatePermissions = createPermissionsDto(new OperationAuthorizable(outputPort));
-            final PortStatusDTO status = getComponentStatus(
-                () -> groupStatus.getOutputPortStatus().stream().filter(outputPortStatus -> outputPort.getIdentifier().equals(outputPortStatus.getId())).findFirst().orElse(null),
-                outputPortStatus -> createPortStatusDto(outputPortStatus)
-            );
+            final PortStatus portStatus = outputPortStatuses.get(outputPort.getIdentifier());
+            final PortStatusDTO statusDto = portStatus == null ? null : createPortStatusDto(portStatus);
             final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(outputPort.getIdentifier()));
             final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
-            dto.getOutputPorts().add(entityFactory.createPortEntity(createPortDto(outputPort), revision, permissions, operatePermissions, status, bulletinEntities));
+            dto.getOutputPorts().add(entityFactory.createPortEntity(createPortDto(outputPort), revision, permissions, operatePermissions, statusDto, bulletinEntities));
         }
 
         return dto;
@@ -2884,11 +2861,15 @@ public final class DtoFactory {
     }
 
     private List<BulletinDTO> createBulletins(final ComponentNode componentNode) {
-        final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(componentNode.getIdentifier()));
+        final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(componentNode.getIdentifier(), componentNode.getProcessGroupIdentifier()));
         return bulletins;
     }
 
     public VariableRegistryDTO createVariableRegistryDto(final ProcessGroup processGroup, final RevisionManager revisionManager) {
+        return createVariableRegistryDto(processGroup, revisionManager, Collections.emptySet());
+    }
+
+    public VariableRegistryDTO createVariableRegistryDto(final ProcessGroup processGroup, final RevisionManager revisionManager, final Set<String> variablesToIgnore) {
         final ComponentVariableRegistry variableRegistry = processGroup.getVariableRegistry();
 
         final List<String> variableNames = variableRegistry.getVariableMap().keySet().stream()
@@ -2898,6 +2879,10 @@ public final class DtoFactory {
         final Set<VariableEntity> variableEntities = new LinkedHashSet<>();
 
         for (final String variableName : variableNames) {
+            if (variablesToIgnore.contains(variableName)) {
+                continue;
+            }
+
             final VariableDTO variableDto = new VariableDTO();
             variableDto.setName(variableName);
             variableDto.setValue(variableRegistry.getVariableValue(variableName));
@@ -3144,21 +3129,31 @@ public final class DtoFactory {
         return fromDocumentedTypes(classBundles, bundleGroupFilter, bundleArtifactFilter, typeFilter);
     }
 
+    public ProcessorDTO createProcessorDto(final ProcessorNode node) {
+        return createProcessorDto(node, false);
+    }
+
     /**
      * Creates a ProcessorDTO from the specified ProcessorNode.
      * @param node node
      * @return dto
      */
-    public ProcessorDTO createProcessorDto(final ProcessorNode node) {
+    private ProcessorDTO createProcessorDto(final ProcessorNode node, final boolean uiOnly) {
         if (node == null) {
             return null;
         }
 
         final BundleCoordinate bundleCoordinate = node.getBundleCoordinate();
-        final List<Bundle> compatibleBundles = extensionManager.getBundles(node.getCanonicalClassName()).stream().filter(bundle -> {
+
+        final List<Bundle> availableBundles = extensionManager.getBundles(node.getCanonicalClassName());
+        int compatibleBundleCount = 0;
+        for (final Bundle bundle : availableBundles) {
             final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
-            return bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
-        }).collect(Collectors.toList());
+            final boolean compatible = bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
+            if (compatible) {
+                compatibleBundleCount++;
+            }
+        }
 
         final ProcessorDTO dto = new ProcessorDTO();
         dto.setId(node.getIdentifier());
@@ -3171,7 +3166,7 @@ public final class DtoFactory {
         dto.setDeprecated(node.isDeprecated());
         dto.setExecutionNodeRestricted(node.isExecutionNodeRestricted());
         dto.setExtensionMissing(node.isExtensionMissing());
-        dto.setMultipleVersionsAvailable(compatibleBundles.size() > 1);
+        dto.setMultipleVersionsAvailable(compatibleBundleCount > 1);
         dto.setVersionedComponentId(node.getVersionedComponentId().orElse(null));
 
         dto.setType(node.getCanonicalClassName());
@@ -3205,7 +3200,7 @@ public final class DtoFactory {
         dto.setSupportsEventDriven(node.isEventDrivenSupported());
         dto.setSupportsBatching(node.isSessionBatchingSupported());
 
-        dto.setConfig(createProcessorConfigDto(node));
+        dto.setConfig(createProcessorConfigDto(node, uiOnly));
 
         final ValidationStatus validationStatus = node.getValidationStatus();
         dto.setValidationStatus(validationStatus.name());
@@ -3954,52 +3949,64 @@ public final class DtoFactory {
      * @param procNode node
      * @return dto
      */
-    public ProcessorConfigDTO createProcessorConfigDto(final ProcessorNode procNode) {
+    public ProcessorConfigDTO createProcessorConfigDto(final ProcessorNode procNode, final boolean uiOnly) {
         if (procNode == null) {
             return null;
         }
 
         final ProcessorConfigDTO dto = new ProcessorConfigDTO();
 
-        // sort a copy of the properties
-        final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>(new Comparator<PropertyDescriptor>() {
-            @Override
-            public int compare(final PropertyDescriptor o1, final PropertyDescriptor o2) {
-                return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName());
-            }
-        });
-        sortedProperties.putAll(procNode.getRawPropertyValues());
-
         // get the property order from the processor
         final Processor processor = procNode.getProcessor();
-        final Map<PropertyDescriptor, String> orderedProperties = new LinkedHashMap<>();
-        final List<PropertyDescriptor> descriptors = processor.getPropertyDescriptors();
-        if (descriptors != null && !descriptors.isEmpty()) {
-            for (final PropertyDescriptor descriptor : descriptors) {
-                orderedProperties.put(descriptor, null);
-            }
-        }
-        orderedProperties.putAll(sortedProperties);
 
-        // build the descriptor and property dtos
-        dto.setDescriptors(new LinkedHashMap<String, PropertyDescriptorDTO>());
-        dto.setProperties(new LinkedHashMap<String, String>());
-        for (final Map.Entry<PropertyDescriptor, String> entry : orderedProperties.entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
+        if (!uiOnly) {
+            // sort a copy of the properties
+            final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>((o1, o2) -> Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()));
+            sortedProperties.putAll(procNode.getRawPropertyValues());
 
-            // store the property descriptor
-            dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor, procNode.getProcessGroup().getIdentifier()));
+            final Map<PropertyDescriptor, String> orderedProperties = new LinkedHashMap<>();
+            final List<PropertyDescriptor> descriptors = processor.getPropertyDescriptors();
+            if (descriptors != null && !descriptors.isEmpty()) {
+                for (final PropertyDescriptor descriptor : descriptors) {
+                    orderedProperties.put(descriptor, null);
+                }
+            }
+            orderedProperties.putAll(sortedProperties);
+
+            // build the descriptor and property dtos
+            dto.setDescriptors(new LinkedHashMap<>());
+            dto.setProperties(new LinkedHashMap<>());
+            for (final Map.Entry<PropertyDescriptor, String> entry : orderedProperties.entrySet()) {
+                final PropertyDescriptor descriptor = entry.getKey();
+
+                // store the property descriptor
+                dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor, procNode.getProcessGroup().getIdentifier()));
+
+                // determine the property value - don't include sensitive properties
+                String propertyValue = entry.getValue();
+                if (propertyValue != null && descriptor.isSensitive()) {
+                    propertyValue = SENSITIVE_VALUE_MASK;
+                } else if (propertyValue == null && descriptor.getDefaultValue() != null) {
+                    propertyValue = descriptor.getDefaultValue();
+                }
 
-            // determine the property value - don't include sensitive properties
-            String propertyValue = entry.getValue();
-            if (propertyValue != null && descriptor.isSensitive()) {
-                propertyValue = SENSITIVE_VALUE_MASK;
-            } else if (propertyValue == null && descriptor.getDefaultValue() != null) {
-                propertyValue = descriptor.getDefaultValue();
+                // set the property value
+                dto.getProperties().put(descriptor.getName(), propertyValue);
             }
 
-            // set the property value
-            dto.getProperties().put(descriptor.getName(), propertyValue);
+            dto.setAnnotationData(procNode.getAnnotationData());
+
+            // set up the default values for concurrent tasks and scheduling period
+            final Map<String, String> defaultConcurrentTasks = new HashMap<>();
+            defaultConcurrentTasks.put(SchedulingStrategy.TIMER_DRIVEN.name(), String.valueOf(SchedulingStrategy.TIMER_DRIVEN.getDefaultConcurrentTasks()));
+            defaultConcurrentTasks.put(SchedulingStrategy.EVENT_DRIVEN.name(), String.valueOf(SchedulingStrategy.EVENT_DRIVEN.getDefaultConcurrentTasks()));
+            defaultConcurrentTasks.put(SchedulingStrategy.CRON_DRIVEN.name(), String.valueOf(SchedulingStrategy.CRON_DRIVEN.getDefaultConcurrentTasks()));
+            dto.setDefaultConcurrentTasks(defaultConcurrentTasks);
+
+            final Map<String, String> defaultSchedulingPeriod = new HashMap<>();
+            defaultSchedulingPeriod.put(SchedulingStrategy.TIMER_DRIVEN.name(), SchedulingStrategy.TIMER_DRIVEN.getDefaultSchedulingPeriod());
+            defaultSchedulingPeriod.put(SchedulingStrategy.CRON_DRIVEN.name(), SchedulingStrategy.CRON_DRIVEN.getDefaultSchedulingPeriod());
+            dto.setDefaultSchedulingPeriod(defaultSchedulingPeriod);
         }
 
         dto.setSchedulingPeriod(procNode.getSchedulingPeriod());
@@ -4012,19 +4019,6 @@ public final class DtoFactory {
         dto.setBulletinLevel(procNode.getBulletinLevel().name());
         dto.setSchedulingStrategy(procNode.getSchedulingStrategy().name());
         dto.setExecutionNode(procNode.getExecutionNode().name());
-        dto.setAnnotationData(procNode.getAnnotationData());
-
-        // set up the default values for concurrent tasks and scheduling period
-        final Map<String, String> defaultConcurrentTasks = new HashMap<>();
-        defaultConcurrentTasks.put(SchedulingStrategy.TIMER_DRIVEN.name(), String.valueOf(SchedulingStrategy.TIMER_DRIVEN.getDefaultConcurrentTasks()));
-        defaultConcurrentTasks.put(SchedulingStrategy.EVENT_DRIVEN.name(), String.valueOf(SchedulingStrategy.EVENT_DRIVEN.getDefaultConcurrentTasks()));
-        defaultConcurrentTasks.put(SchedulingStrategy.CRON_DRIVEN.name(), String.valueOf(SchedulingStrategy.CRON_DRIVEN.getDefaultConcurrentTasks()));
-        dto.setDefaultConcurrentTasks(defaultConcurrentTasks);
-
-        final Map<String, String> defaultSchedulingPeriod = new HashMap<>();
-        defaultSchedulingPeriod.put(SchedulingStrategy.TIMER_DRIVEN.name(), SchedulingStrategy.TIMER_DRIVEN.getDefaultSchedulingPeriod());
-        defaultSchedulingPeriod.put(SchedulingStrategy.CRON_DRIVEN.name(), SchedulingStrategy.CRON_DRIVEN.getDefaultSchedulingPeriod());
-        dto.setDefaultSchedulingPeriod(defaultSchedulingPeriod);
 
         return dto;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
index 11c8d96..a569a29 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
@@ -152,7 +152,10 @@
         return $.ajax({
             type: 'GET',
             url: config.urls.api + '/flow/process-groups/' + encodeURIComponent(processGroupId),
-            dataType: 'json'
+            dataType: 'json',
+            data: {
+                uiOnly: true
+            }
         }).done(function (flowResponse) {
             // get the controller and its contents
             var processGroupFlow = flowResponse.processGroupFlow;
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
index 24c1ddd..a728b29 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.stateless.engine;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.attribute.expression.language.VariableImpact;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.AllowableValue;
@@ -257,7 +258,7 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
             final String propertyValue = property.getValue();
 
             final PropertyDescriptor descriptor = component.getPropertyDescriptor(propertyName);
-            final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList());
+            final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList(), VariableImpact.NEVER_IMPACTED);
 
             explicitlyConfiguredPropertyMap.put(descriptor, propertyConfiguration);
         }