You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/09/06 11:16:39 UTC

[GitHub] [nifi] gresockj opened a new pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

gresockj opened a new pull request #5369:
URL: https://github.com/apache/nifi/pull/5369


   #### Implements ParameterProviders
   
   This PR implements the back-end functionality described in the [Parameter Providers Feature Proposal](https://cwiki.apache.org/confluence/display/NIFI/Parameter+Providers%3A+Fetching+Parameters+from+External+Sources).  A NiFi CLI script has been added to the [NIFI-9003 JIRA](https://issues.apache.org/jira/browse/NIFI-9003) for running a basic test of the functionality.
   
   Included features:
   - Added `ParameterProvider` interface to framework
   - Added `EnvironmentVariableSensitiveParameterProvider` and `EnvironmentVariableNonSensitiveParameterProvider` to `nifi-standard-nar`
   - Added CLI commands for interacting with parameter providers
   - UI search bar and documentation integration for parameter providers
   
   Not included:
   - UI implementation of creating/displaying/updating/deleting parameter providers, or integrating them into parameter contexts
   - Automatic periodic fetch-and-apply
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [x] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [x] Have you written or updated unit tests to verify your changes?
   - [x] Have you verified that the full build is successful on JDK 8?
   - [x] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [x] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#discussion_r719262030



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AbstractParameterResource.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.web.api;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.util.LifecycleManagementException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.HttpMethod;
+import java.net.URI;
+import java.util.Map;
+
+public abstract class AbstractParameterResource extends ApplicationResource {

Review comment:
       This method needs access to the protected methods `getReplicationTarget()` and `getRequestReplicator()`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#discussion_r719389091



##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/SensitiveParameterProvider.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+/**
+ * A base interface for all sensitive <code>ParameterProvider</code>s
+ */
+public interface SensitiveParameterProvider extends ParameterProvider {

Review comment:
       I believe I was initially going for strong type enforcement of parameter provider sensitivity here, but on reconsideration I agree.  Further, I'd like to go in the direction of making `ParameterProviderNode` responsible for enforcing/setting sensitivity based on how a ParameterProvider is referenced rather than exposing a method like this to the `ParameterProvider` interface.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#discussion_r719372458



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
##########
@@ -405,6 +421,66 @@ public void onReportingTaskAdded(final ReportingTaskNode taskNode) {
         allReportingTasks.put(taskNode.getIdentifier(), taskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProvider(final String id) {
+        if (id == null) {

Review comment:
       Yes, `ConcurrentHashMap` throws NPE if the key is null.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#discussion_r717933999



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/AbstractParameterProviderNode.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterProvider;
+import org.apache.nifi.parameter.SensitiveParameterProvider;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {

Review comment:
       Perhaps it just hasn't clicked yet, but I don't think we need to have an `AbstractParameterProviderNode`, a `StandardParameterProviderNode`, and a `StatelessParameterProviderNode`. Instead, I think we can have just a `StandardParameterProviderNode`. The only thing that I think may have caused an issue with that is that `StandardParameterProviderNode` as it stands takes the `FlowController` in its constructor. But that `FlowController` shouldn't be provided at all, I don't think. The only way it's used is to return it as the parent. But the parent could instead just be defined as an `Authorizable` so any `Authorizable` could be provided in the constructor instead. So in standard nifi it would take in the FlowController (referenced though as an `Authorizable`) and in stateless the parent would be `null`.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/src/main/java/org/apache/nifi/parameter/AbstractEnvironmentVariableParameterProvider.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public abstract class AbstractEnvironmentVariableParameterProvider extends AbstractParameterProvider {

Review comment:
       Again, I think we should really have just an EnvironmentVariableParameterProvider and not an abstract with children. Instead, the interface can have a method, if necessary, to determine whether provided parameters are sensitive or not.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardParameterProviderDAO.java
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.web.dao.impl;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.exception.ValidationException;
+import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
+import org.apache.nifi.controller.parameter.ParameterProviderProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.logging.LogRepository;
+import org.apache.nifi.logging.repository.NopLogRepository;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.processor.SimpleProcessLogger;
+import org.apache.nifi.util.BundleUtils;
+import org.apache.nifi.web.NiFiCoreException;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
+import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderDTO;
+import org.apache.nifi.web.dao.ComponentStateDAO;
+import org.apache.nifi.web.dao.ParameterProviderDAO;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class StandardParameterProviderDAO extends ComponentDAO implements ParameterProviderDAO {
+
+    private ParameterProviderProvider parameterProviderProvider;
+    private ComponentStateDAO componentStateDAO;
+    private ReloadComponent reloadComponent;
+    private FlowController flowController;
+
+    private ParameterProviderNode locateParameterProvider(final String parameterProviderId) {
+        // get the parameter provider
+        final ParameterProviderNode parameterProvider = parameterProviderProvider.getParameterProviderNode(parameterProviderId);
+
+        // ensure the parameter provider exists
+        if (parameterProvider == null) {
+            throw new ResourceNotFoundException(String.format("Unable to locate parameter provider with id '%s'.", parameterProviderId));
+        }
+
+        return parameterProvider;
+    }
+
+    @Override
+    public void verifyCreate(final ParameterProviderDTO parameterProviderDTO) {
+        verifyCreate(parameterProviderProvider.getExtensionManager(), parameterProviderDTO.getType(), parameterProviderDTO.getBundle());
+    }
+
+    @Override
+    public ParameterProviderNode createParameterProvider(final ParameterProviderDTO parameterProviderDTO) {
+        // ensure the type is specified
+        if (parameterProviderDTO.getType() == null) {
+            throw new IllegalArgumentException("The parameter provider type must be specified.");
+        }
+
+        try {
+            // create the parameter provider
+            final ExtensionManager extensionManager = parameterProviderProvider.getExtensionManager();
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, parameterProviderDTO.getType(), parameterProviderDTO.getBundle());
+            final ParameterProviderNode parameterProvider = parameterProviderProvider.createParameterProvider(
+                    parameterProviderDTO.getType(), parameterProviderDTO.getId(), bundleCoordinate, true);
+
+            // ensure we can perform the update
+            verifyUpdate(parameterProvider, parameterProviderDTO);
+
+            // perform the update
+            configureParameterProvider(parameterProvider, parameterProviderDTO);
+
+            return parameterProvider;
+        } catch (final ParameterProviderInstantiationException e) {
+            throw new NiFiCoreException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public ParameterProviderNode getParameterProvider(final String parameterProviderId) {
+        return locateParameterProvider(parameterProviderId);
+    }
+
+    @Override
+    public boolean hasParameterProvider(final String parameterProviderId) {
+        return parameterProviderProvider.getParameterProviderNode(parameterProviderId) != null;
+    }
+
+    @Override
+    public Set<ParameterProviderNode> getParameterProviders() {
+        return parameterProviderProvider.getAllParameterProviders();
+    }
+
+    @Override
+    public ParameterProviderNode updateParameterProvider(final ParameterProviderDTO parameterProviderDTO) {
+        // get the parameter provider
+        final ParameterProviderNode parameterProvider = locateParameterProvider(parameterProviderDTO.getId());
+
+        // ensure we can perform the update
+        verifyUpdate(parameterProvider, parameterProviderDTO);
+
+        // perform the update
+        configureParameterProvider(parameterProvider, parameterProviderDTO);
+
+        // attempt to change the underlying processor if an updated bundle is specified
+        // updating the bundle must happen after configuring so that any additional classpath resources are set first
+        updateBundle(parameterProvider, parameterProviderDTO);
+
+        return parameterProvider;
+    }
+
+    @Override
+    public ParameterProviderNode fetchParameters(final String parameterProviderId) {
+        final ParameterProviderNode parameterProviderNode = locateParameterProvider(parameterProviderId);
+
+        parameterProviderNode.fetchParameters();
+        return parameterProviderNode;
+    }
+
+    @Override
+    public void verifyCanApplyParameters(final String parameterProviderId, final Set<String> parameterNames) {
+        final ParameterProviderNode parameterProviderNode = locateParameterProvider(parameterProviderId);
+        parameterProviderNode.verifyCanApplyParameters(parameterNames);
+    }
+
+    @Override
+    public Map<ParameterContext, Map<String, Parameter>> getFetchedParametersToApply(final String parameterProviderId, final Set<String> parameterNames) {
+        final ParameterProviderNode parameterProviderNode = locateParameterProvider(parameterProviderId);
+
+        return parameterProviderNode.getFetchedParametersToApply(parameterNames);
+    }
+
+    private void updateBundle(final ParameterProviderNode parameterProvider, final ParameterProviderDTO parameterProviderDTO) {
+        final BundleDTO bundleDTO = parameterProviderDTO.getBundle();
+        if (bundleDTO != null) {
+            final ExtensionManager extensionManager = parameterProviderProvider.getExtensionManager();
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(extensionManager, parameterProvider.getCanonicalClassName(), bundleDTO);
+            final BundleCoordinate existingCoordinate = parameterProvider.getBundleCoordinate();
+            if (!existingCoordinate.getCoordinate().equals(incomingCoordinate.getCoordinate())) {
+                try {
+                    // we need to use the property descriptors from the temp component here in case we are changing from a ghost component to a real component
+                    final ConfigurableComponent tempComponent = extensionManager.getTempComponent(parameterProvider.getCanonicalClassName(), incomingCoordinate);
+                    final Set<URL> additionalUrls = parameterProvider.getAdditionalClasspathResources(tempComponent.getPropertyDescriptors());
+                    reloadComponent.reload(parameterProvider, parameterProvider.getCanonicalClassName(), incomingCoordinate, additionalUrls);
+                } catch (final ParameterProviderInstantiationException e) {
+                    throw new NiFiCoreException(String.format("Unable to update parameter provider %s from %s to %s due to: %s",
+                            parameterProviderDTO.getId(), parameterProvider.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
+                }
+            }
+        }
+    }
+
+    // A placeholder in case validation is required in the future
+    private List<String> validateProposedConfiguration(final ParameterProviderNode parameterProvider, final ParameterProviderDTO parameterProviderDTO) {
+        final List<String> validationErrors = new ArrayList<>();

Review comment:
       Should probably just `return Collections.emptyList();`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextDTO.java
##########
@@ -88,8 +91,42 @@ public void setInheritedParameterContexts(final List<ParameterContextReferenceEn
         return boundProcessGroups;
     }
 
+    @ApiModelProperty("An optional Parameter Provider for sensitive Parameters")
+    public ComponentReferenceEntity getSensitiveParameterProviderRef() {
+        return sensitiveParameterProviderRef;
+    }
+
+    public void setSensitiveParameterProviderRef(final ComponentReferenceEntity sensitiveParameterProviderRef) {
+        this.sensitiveParameterProviderRef = sensitiveParameterProviderRef;
+    }
+
+    @ApiModelProperty("An optional Parameter Provider for non-sensitive Parameters")
+    public ComponentReferenceEntity getNonSensitiveParameterProviderRef() {
+        return nonSensitiveParameterProviderRef;
+    }
+
+    public void setNonSensitiveParameterProviderRef(final ComponentReferenceEntity nonSensitiveParameterProviderRef) {
+        this.nonSensitiveParameterProviderRef = nonSensitiveParameterProviderRef;
+    }
+
     @Override
     public String toString() {
         return "ParameterContext[id=" + identifier + ", name=" + name + ", parameters=" + parameters + "]";
     }
+
+    /**
+     * A utility method for safely returning the identifier of a ComponentReferenceEntity.
+     * @param referenceEntity A ComponentReferenceEntity
+     * @return The <code>referenceEntity.getComponent().getId()</code> (may be null)
+     */
+    public static String getReferenceId(final ComponentReferenceEntity referenceEntity) {

Review comment:
       I haven't seen where this is used yet but it seems a bit odd to have a static get method like this in a DTO. Especially when it's not even used by the DTO.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java
##########
@@ -99,12 +99,12 @@ public void setType(String type) {
     }
 
     /**
-     * The details of the artifact that bundled this processor type.
+     * The details of the artifact that bundled this reporting task type.

Review comment:
       Ha nice catch, good to know someone reads the documentation :)

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ParameterProviderProviderFactoryBean.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.web.spring;
+
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.parameter.ParameterProviderProvider;
+import org.springframework.beans.BeansException;
+import org.springframework.beans.factory.FactoryBean;
+import org.springframework.context.ApplicationContext;
+import org.springframework.context.ApplicationContextAware;
+
+/**
+ *
+ */
+public class ParameterProviderProviderFactoryBean implements FactoryBean<ParameterProviderProvider>, ApplicationContextAware {

Review comment:
       If we eliminate the ParameterProviderProvider class, we can eliminate this as well.

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProviderInitializationContext.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.kerberos.KerberosContext;
+import org.apache.nifi.logging.ComponentLog;
+
+public interface ParameterProviderInitializationContext extends KerberosContext {
+
+    /**
+     * @return the identifier associated with the {@link ParameterProvider} with
+     * which this context is associated
+     */
+    String getIdentifier();
+
+    /**
+     * @return the configured name for this ParameterProvider
+     */
+    String getName();
+
+    /**
+     * @return the {@link ControllerServiceLookup} which can be used to obtain
+     * Controller Services
+     */
+    ControllerServiceLookup getControllerServiceLookup();

Review comment:
       I don't think we need the ControllerServiceLookup, do we? I think this exists in the API for processors, etc. because it came before the ability to call getProperty(...).asControllerService(...) but now that should be used.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterProviderResource.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.web.api;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiParam;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
+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.ui.extension.UiExtension;
+import org.apache.nifi.ui.extension.UiExtensionMapping;
+import org.apache.nifi.web.NiFiServiceFacade;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.ResumeFlowException;
+import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.UiExtensionType;
+import org.apache.nifi.web.api.concurrent.AsyncRequestManager;
+import org.apache.nifi.web.api.concurrent.AsynchronousWebRequest;
+import org.apache.nifi.web.api.concurrent.RequestManager;
+import org.apache.nifi.web.api.concurrent.StandardAsynchronousWebRequest;
+import org.apache.nifi.web.api.concurrent.StandardUpdateStep;
+import org.apache.nifi.web.api.concurrent.UpdateStep;
+import org.apache.nifi.web.api.dto.BundleDTO;
+import org.apache.nifi.web.api.dto.ComponentStateDTO;
+import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
+import org.apache.nifi.web.api.dto.DtoFactory;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderApplyParametersRequestDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderApplyParametersUpdateStepDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderDTO;
+import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.VerifyParameterProviderConfigRequestDTO;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+import org.apache.nifi.web.api.entity.ComponentStateEntity;
+import org.apache.nifi.web.api.entity.Entity;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.api.entity.ParameterContextUpdateEntity;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderApplyParametersRequestEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderParameterApplicationEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderParameterFetchEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderReferencingComponentsEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
+import org.apache.nifi.web.api.entity.VerifyParameterProviderConfigRequestEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+import org.apache.nifi.web.api.request.LongParameter;
+import org.apache.nifi.web.util.ComponentLifecycle;
+import org.apache.nifi.web.util.ParameterUpdateManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * RESTful endpoint for managing a Parameter Provider.
+ */
+@Path("/parameter-providers")
+@Api(
+        value = "/parameter-providers",
+        description = "Endpoint for managing a Parameter Provider."
+)
+public class ParameterProviderResource extends AbstractParameterResource {
+    private static final Logger logger = LoggerFactory.getLogger(ParameterProviderResource.class);
+
+    private NiFiServiceFacade serviceFacade;
+    private DtoFactory dtoFactory;
+    private Authorizer authorizer;
+    private ParameterUpdateManager parameterUpdateManager;
+
+    private static final String VERIFICATION_REQUEST_TYPE = "verification-request";
+    private RequestManager<VerifyParameterProviderConfigRequestEntity, List<ConfigVerificationResultDTO>> configVerificationRequestManager =
+            new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Verify Parameter Provider Config Thread");
+
+    private RequestManager<List<ParameterContextEntity>, List<ParameterContextEntity>> updateRequestManager =
+            new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Parameter Provider Apply Thread");
+
+    private ComponentLifecycle clusterComponentLifecycle;
+    private ComponentLifecycle localComponentLifecycle;
+
+    @Context
+    private ServletContext servletContext;
+
+    public void init() {
+        parameterUpdateManager = new ParameterUpdateManager(serviceFacade, dtoFactory, authorizer, this);
+    }
+
+    private void authorizeReadParameterProvider(final String parameterProviderId) {
+        if (parameterProviderId == null) {
+            throw new IllegalArgumentException("Parameter Provider ID must be specified");
+        }
+
+        serviceFacade.authorizeAccess(lookup -> {
+            final ComponentAuthorizable parameterProvider = lookup.getParameterProvider(parameterProviderId);
+            parameterProvider.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+    }
+
+    /**
+     * Populate the uri's for the specified parameter providers.
+     *
+     * @param parameterProviderEntities parameter providers
+     * @return dtos
+     */
+    public Set<ParameterProviderEntity> populateRemainingParameterProviderEntitiesContent(final Set<ParameterProviderEntity> parameterProviderEntities) {
+        for (ParameterProviderEntity parameterProviderEntity : parameterProviderEntities) {
+            populateRemainingParameterProviderEntityContent(parameterProviderEntity);
+        }
+        return parameterProviderEntities;
+    }
+
+    /**
+     * Populate the uri's for the specified parameter provider.
+     *
+     * @param parameterProviderEntity parameter provider
+     * @return dtos
+     */
+    public ParameterProviderEntity populateRemainingParameterProviderEntityContent(final ParameterProviderEntity parameterProviderEntity) {
+        parameterProviderEntity.setUri(generateResourceUri("parameter-providers", parameterProviderEntity.getId()));
+
+        // populate the remaining content
+        if (parameterProviderEntity.getComponent() != null) {
+            populateRemainingParameterProviderContent(parameterProviderEntity.getComponent());
+        }
+        return parameterProviderEntity;
+    }
+
+    /**
+     * Populates the uri for the specified parameter provider.
+     */
+    public ParameterProviderDTO populateRemainingParameterProviderContent(final ParameterProviderDTO parameterProvider) {
+        final BundleDTO bundle = parameterProvider.getBundle();
+        if (bundle == null) {
+            return parameterProvider;
+        }
+
+        // see if this processor has any ui extensions
+        final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions");
+        if (uiExtensionMapping.hasUiExtension(parameterProvider.getType(), bundle.getGroup(), bundle.getArtifact(), bundle.getVersion())) {
+            final List<UiExtension> uiExtensions = uiExtensionMapping.getUiExtension(parameterProvider.getType(), bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
+            for (final UiExtension uiExtension : uiExtensions) {
+                if (UiExtensionType.ParameterProviderConfiguration.equals(uiExtension.getExtensionType())) {
+                    parameterProvider.setCustomUiUrl(uiExtension.getContextPath() + "/configure");
+                }
+            }
+        }
+
+        return parameterProvider;
+    }
+
+    /**
+     * Retrieves the specified parameter provider.
+     *
+     * @param id The id of the parameter provider to retrieve
+     * @return A parameterProviderEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    @ApiOperation(
+            value = "Gets a parameter provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getParameterProvider(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the parameter provider
+        final ParameterProviderEntity parameterProvider = serviceFacade.getParameterProvider(id);
+        populateRemainingParameterProviderEntityContent(parameterProvider);
+
+        return generateOkResponse(parameterProvider).build();
+    }
+
+    /**
+     * Retrieves the references of the specified parameter provider.
+     *
+     * @param id The id of the parameter provider to retrieve
+     * @return A parameterProviderEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/references")
+    @ApiOperation(
+            value = "Gets all references to a parameter provider",
+            response = ParameterProviderReferencingComponentsEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getParameterProviderReferences(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the parameter provider references
+        final ParameterProviderReferencingComponentsEntity entity = serviceFacade.getParameterProviderReferencingComponents(id);
+
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Returns the descriptor for the specified property.
+     *
+     * @param id           The id of the parameter provider.
+     * @param propertyName The property
+     * @return a propertyDescriptorEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/descriptors")
+    @ApiOperation(
+            value = "Gets a parameter provider property descriptor",
+            response = PropertyDescriptorEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getPropertyDescriptor(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id,
+            @ApiParam(
+                    value = "The property name.",
+                    required = true
+            )
+            @QueryParam("propertyName") final String propertyName) {
+
+        // ensure the property name is specified
+        if (propertyName == null) {
+            throw new IllegalArgumentException("The property name must be specified.");
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the property descriptor
+        final PropertyDescriptorDTO descriptor = serviceFacade.getParameterProviderPropertyDescriptor(id, propertyName);
+
+        // generate the response entity
+        final PropertyDescriptorEntity entity = new PropertyDescriptorEntity();
+        entity.setPropertyDescriptor(descriptor);
+
+        // generate the response
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Gets the state for a parameter provider.
+     *
+     * @param id The id of the parameter provider
+     * @return a componentStateEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/state")
+    @ApiOperation(
+            value = "Gets the state for a parameter provider",
+            response = ComponentStateEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getState(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the component state
+        final ComponentStateDTO state = serviceFacade.getParameterProviderState(id);
+
+        // generate the response entity
+        final ComponentStateEntity entity = new ComponentStateEntity();
+        entity.setComponentState(state);
+
+        // generate the response
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Clears the state for a parameter provider.
+     *
+     * @param httpServletRequest servlet request
+     * @param id                 The id of the parameter provider
+     * @return a componentStateEntity
+     */
+    @POST
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/state/clear-requests")
+    @ApiOperation(
+            value = "Clears the state for a parameter provider",
+            response = ComponentStateEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response clearState(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST);
+        }
+
+        final ParameterProviderEntity requestParameterProviderEntity = new ParameterProviderEntity();
+        requestParameterProviderEntity.setId(id);
+
+        return withWriteLock(
+                serviceFacade,
+                requestParameterProviderEntity,
+                lookup -> {
+                    final Authorizable processor = lookup.getParameterProvider(id).getAuthorizable();
+                    processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                },
+                () -> serviceFacade.verifyCanClearParameterProviderState(id),
+                (parameterProviderEntity) -> {
+                    // get the component state
+                    serviceFacade.clearParameterProviderState(parameterProviderEntity.getId());
+
+                    // generate the response entity
+                    final ComponentStateEntity entity = new ComponentStateEntity();
+
+                    // generate the response
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+    /**
+     * Updates the specified a Parameter Provider.
+     *
+     * @param httpServletRequest  request
+     * @param id                  The id of the parameter provider to update.
+     * @param requestParameterProviderEntity A parameterProviderEntity.
+     * @return A parameterProviderEntity.
+     */
+    @PUT
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    @ApiOperation(
+            value = "Updates a parameter provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}"),
+                    @Authorization(value = "Read - any referenced Controller Services if this request changes the reference - /controller-services/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response updateParameterProvider(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id,
+            @ApiParam(
+                    value = "The parameter provider configuration details.",
+                    required = true
+            ) final ParameterProviderEntity requestParameterProviderEntity) {
+
+        if (requestParameterProviderEntity == null || requestParameterProviderEntity.getComponent() == null) {
+            throw new IllegalArgumentException("Parameter provider details must be specified.");
+        }
+
+        if (requestParameterProviderEntity.getRevision() == null) {
+            throw new IllegalArgumentException("Revision must be specified.");
+        }
+
+        // ensure the ids are the same
+        final ParameterProviderDTO requestParameterProviderDTO = requestParameterProviderEntity.getComponent();
+        if (!id.equals(requestParameterProviderDTO.getId())) {
+            throw new IllegalArgumentException(String.format("The parameter provider id (%s) in the request body does not equal the "
+                    + "parameter provider id of the requested resource (%s).", requestParameterProviderDTO.getId(), id));
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.PUT, requestParameterProviderEntity);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(requestParameterProviderEntity.isDisconnectedNodeAcknowledged());
+        }
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = getRevision(requestParameterProviderEntity, id);
+        return withWriteLock(
+                serviceFacade,
+                requestParameterProviderEntity,
+                requestRevision,
+                lookup -> {
+                    // authorize parameter provider
+                    final ComponentAuthorizable authorizable = lookup.getParameterProvider(id);
+                    authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // authorize any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(requestParameterProviderDTO.getProperties(), authorizable, authorizer, lookup);
+                },
+                () -> serviceFacade.verifyUpdateParameterProvider(requestParameterProviderDTO),
+                (revision, parameterProviderEntity) -> {
+                    final ParameterProviderDTO parameterProviderDTO = parameterProviderEntity.getComponent();
+
+                    // update the parameter provider
+                    final ParameterProviderEntity entity = serviceFacade.updateParameterProvider(revision, parameterProviderDTO);
+                    populateRemainingParameterProviderEntityContent(entity);
+
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+    /**
+     * Removes the specified parameter provider.
+     *
+     * @param httpServletRequest request
+     * @param version            The revision is used to verify the client is working with
+     *                           the latest version of the flow.
+     * @param clientId           Optional client id. If the client id is not specified, a
+     *                           new one will be generated. This value (whether specified or generated) is
+     *                           included in the response.
+     * @param id                 The id of the parameter provider to remove.
+     * @return A entity containing the client id and an updated revision.
+     */
+    @DELETE
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    @ApiOperation(
+            value = "Deletes a parameter provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}"),
+                    @Authorization(value = "Write - /controller"),
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response removeParameterProvider(
+            @Context HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The revision is used to verify the client is working with the latest version of the flow.",
+                    required = false
+            )
+            @QueryParam(VERSION) LongParameter version,
+            @ApiParam(
+                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
+                    required = false
+            )
+            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @QueryParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.DELETE);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(disconnectedNodeAcknowledged);
+        }
+
+        final ParameterProviderEntity requestParameterProviderEntity = new ParameterProviderEntity();
+        requestParameterProviderEntity.setId(id);
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
+        return withWriteLock(
+                serviceFacade,
+                requestParameterProviderEntity,
+                requestRevision,
+                lookup -> {
+                    final ComponentAuthorizable parameterProvider = lookup.getParameterProvider(id);
+
+                    // ensure write permission to the parameter provider
+                    parameterProvider.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // ensure write permission to the parent process group
+                    parameterProvider.getAuthorizable().getParentAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // verify any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(parameterProvider, authorizer, lookup, false);
+                },
+                () -> serviceFacade.verifyDeleteParameterProvider(id),
+                (revision, parameterProviderEntity) -> {
+                    // delete the specified parameter provider
+                    final ParameterProviderEntity entity = serviceFacade.deleteParameterProvider(revision, parameterProviderEntity.getId());
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+    /**
+     * Tells the Parameter Provider to fetch its parameters.  This will temporarily cache the fetched parameters,
+     * but the changes will not be applied to the flow until an "apply-parameters-requests" request is created.
+     *
+     * @param httpServletRequest  request
+     * @param parameterProviderId The id of the parameter provider.
+     * @return A parameterProviderEntity.
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/parameters/fetch-requests")
+    @ApiOperation(
+            value = "Fetches and temporarily caches the parameters for a provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid} or  or /operation/parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response fetchParameters(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String parameterProviderId,
+            @ApiParam(
+                    value = "The parameter fetch request.",
+                    required = true
+            ) final ParameterProviderParameterFetchEntity fetchParametersEntity) {
+
+        if (fetchParametersEntity.getId() == null) {
+            throw new IllegalArgumentException("The ID of the Parameter Provider must be specified");
+        }
+        if (!fetchParametersEntity.getId().equals(parameterProviderId)) {
+            throw new IllegalArgumentException("The ID of the Parameter Provider must match the ID specified in the URL's path");
+        }
+
+        if (fetchParametersEntity.getRevision() == null) {
+            throw new IllegalArgumentException("Revision must be specified.");
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST, fetchParametersEntity);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(fetchParametersEntity.isDisconnectedNodeAcknowledged());
+        }
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = getRevision(fetchParametersEntity.getRevision(), parameterProviderId);
+        return withWriteLock(
+                serviceFacade,
+                fetchParametersEntity,
+                requestRevision,
+                lookup -> {
+                    // authorize parameter provider
+                    final ComponentAuthorizable authorizable = lookup.getParameterProvider(parameterProviderId);
+                    authorizable.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+                    authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                },
+                () -> serviceFacade.verifyCanFetchParameters(parameterProviderId),
+                (revision, parameterProviderFetchEntity) -> {
+                    // fetch the parameters
+                    final ParameterProviderEntity entity = serviceFacade.fetchParameters(parameterProviderId);

Review comment:
       Rather than use `parameterProviderId` here, we need to use `parameterProviderFetchEntity.getId()`. Because this is a mutable request it's 2 phases. Need to ensure that the entity of the first request is what we use, and that's what is provided by `parameterProviderFetchEntity`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
##########
@@ -198,7 +198,7 @@ public void testParameterCreatedWithNullValueOnImportWithSensitivePropertyRefere
         getRootGroup().addProcessGroup(innerGroup);
 
         final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(getFlowController().getFlowManager());
-        final ParameterContext parameterContext = new StandardParameterContext("param-context-id", "parameter-context", parameterReferenceManager, null);
+        final ParameterContext parameterContext = new StandardParameterContext("param-context-id", "parameter-context", parameterReferenceManager, null, null, null, null);

Review comment:
       I'm seeing a lot of this same type of thing, where we add 3 `null` values to the constructor. And there are becoming quite a few arguments to the constructor. Time to consider a Builder pattern perhaps?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
##########
@@ -3143,6 +3241,128 @@ public ReportingTaskEntity deleteReportingTask(final Revision revision, final St
         return entityFactory.createReportingTaskEntity(snapshot, null, permissions, operatePermissions, null);
     }
 
+    @Override
+    public ParameterProviderEntity createParameterProvider(final Revision revision, final ParameterProviderDTO parameterProviderDTO) {
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        // request claim for component to be created... revision already verified (version == 0)
+        final RevisionClaim claim = new StandardRevisionClaim(revision);
+
+        // update revision through revision manager
+        final RevisionUpdate<ParameterProviderDTO> snapshot = revisionManager.updateRevision(claim, user, () -> {
+            // create the parameter provider
+            final ParameterProviderNode parameterProvider = parameterProviderDAO.createParameterProvider(parameterProviderDTO);
+
+            // save the update
+            controllerFacade.save();
+            awaitValidationCompletion(parameterProvider);
+
+            final ParameterProviderDTO dto = dtoFactory.createParameterProviderDto(parameterProvider);
+            final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity());
+            return new StandardRevisionUpdate<>(dto, lastMod);
+        });
+
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderDTO.getId());
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterProvider);
+        final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(parameterProvider));
+        final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(parameterProvider.getIdentifier()));
+        final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
+        return entityFactory.createParameterProviderEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities);
+    }
+
+    @Override
+    public ParameterProviderEntity updateParameterProvider(final Revision revision, final ParameterProviderDTO parameterProviderDTO) {
+        // get the component, ensure we have access to it, and perform the update request
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderDTO.getId());
+        final RevisionUpdate<ParameterProviderDTO> snapshot = updateComponent(revision,
+                parameterProvider,
+                () -> parameterProviderDAO.updateParameterProvider(parameterProviderDTO),
+                rt -> {
+                    awaitValidationCompletion(rt);
+                    return dtoFactory.createParameterProviderDto(rt);
+                });
+
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterProvider);
+        final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(parameterProvider));
+        final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(parameterProvider.getIdentifier()));
+        final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
+        return entityFactory.createParameterProviderEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities);
+    }
+
+    @Override
+    public List<ConfigVerificationResultDTO> verifyParameterProviderConfiguration(final String parameterProviderId, final ParameterProviderDTO parameterProvider) {
+        return parameterProviderDAO.verifyConfiguration(parameterProviderId, parameterProvider);
+    }
+
+    @Override
+    public ParameterProviderEntity deleteParameterProvider(final Revision revision, final String parameterProviderId) {
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderId);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterProvider);
+        final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(parameterProvider));
+        final ParameterProviderDTO snapshot = deleteComponent(
+                revision,
+                parameterProvider.getResource(),
+                () -> parameterProviderDAO.deleteParameterProvider(parameterProviderId),
+                true,
+                dtoFactory.createParameterProviderDto(parameterProvider));
+
+        return entityFactory.createParameterProviderEntity(snapshot, null, permissions, operatePermissions, null);
+    }
+
+    @Override
+    public ParameterProviderEntity fetchParameters(final String parameterProviderId) {
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderId);
+
+        parameterProvider.fetchParameters();
+        awaitValidationCompletion(parameterProvider);

Review comment:
       Shouldn't we validate the parameter provider *before* fetching parameters?

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
##########
@@ -296,6 +298,53 @@ public ReportingTaskNode createReportingTask(final String type, final String id,
         return taskNode;
     }
 
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded,

Review comment:
       Within Stateless, we have a ParameterValueProvider, which is capable of fetching parameter values. This notion of a ParameterProvider is a bit different, where users have the ability to go fetch parameters and apply them. I'm not sure we need to support this at all, TBH, in stateless. Can probably just have this `throw new UnsupportedOperationException();` no?

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProvider.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation.lifecycle.OnConfigurationRestored;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Map;
+
+/**
+ * Defines a provider that is responsible for fetching from an external source Parameters with
+ * which a ParameterContext can be populated.
+ *
+ * <p>
+ * <code>ParameterProvider</code>s are discovered using Java's
+ * <code>ServiceLoader</code> mechanism. As a result, all implementations must
+ * follow these rules:
+ *
+ * <ul>
+ * <li>The implementation must implement this interface.</li>
+ * <li>The implementation must have a file named
+ * org.apache.nifi.parameter.ParameterProvider located within the jar's
+ * <code>META-INF/services</code> directory. This file contains a list of
+ * fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
+ * one-per-line.
+ * <li>The implementation must support a default constructor.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * All implementations of this interface must be thread-safe.
+ * </p>
+ *
+ * <p>
+ * Parameter Providers may choose to annotate a method with the
+ * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
+ * will be invoked after all properties have been set for the ParameterProvider and
+ * before its parameters are fetched.
+ * </p>
+ */
+public interface ParameterProvider extends ConfigurableComponent {
+
+    /**
+     * Provides the Parameter Provider with access to objects that may be of use
+     * throughout the life of the service
+     *
+     * @param config of initialization context
+     * @throws org.apache.nifi.reporting.InitializationException if unable to init
+     */
+    void initialize(ParameterProviderInitializationContext config) throws InitializationException;
+
+    /**
+     * Fetches parameters from an external source.
+     * @param context The <code>ConfigurationContext</code>for the provider
+     * @return An unmodifiable map of available parameters, mapped from descriptor to Parameter
+     */
+    Map<ParameterDescriptor, Parameter> fetchParameters(ConfigurationContext context);

Review comment:
       I think this should be able to throw IOException.

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/AbstractParameterProvider.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.AbstractConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+public abstract class AbstractParameterProvider extends AbstractConfigurableComponent implements ParameterProvider {
+    private String identifier;
+    private String name;
+    private ControllerServiceLookup serviceLookup;
+    private ComponentLog logger;
+    private NodeTypeProvider nodeTypeProvider;
+
+    @Override
+    public final void initialize(final ParameterProviderInitializationContext config) throws InitializationException {
+        identifier = config.getIdentifier();
+        logger = config.getLogger();
+        name = config.getName();
+        serviceLookup = config.getControllerServiceLookup();
+        nodeTypeProvider = config.getNodeTypeProvider();
+        verifyInterfaces();
+        init(config);
+    }
+
+    private void verifyInterfaces() {
+        if (!(this instanceof SensitiveParameterProvider) && !(this instanceof NonSensitiveParameterProvider)) {
+            throw new IllegalStateException(String.format("Parameter Provider [%s] must be either a SensitiveParameterProvider or a NonSensitiveParameterProvider, but is neither",
+                    name));
+        }
+        if ((this instanceof SensitiveParameterProvider) && (this instanceof NonSensitiveParameterProvider)) {
+            throw new IllegalStateException(String.format("Parameter Provider [%s] must be either a SensitiveParameterProvider or a NonSensitiveParameterProvider, but is both",
+                    name));
+        }
+    }
+
+    /**
+     * @return the identifier of this Parameter Provider
+     */
+    @Override
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    /**
+     * Calls {@link AbstractParameterProvider#fetchParameterList(ConfigurationContext)} and returns the parameters
+     * as a map.
+     * @param context The configuration context
+     * @return A map from <code>ParameterDescriptor</code> to <code>Parameter</code>
+     */
+    @Override
+    public final Map<ParameterDescriptor, Parameter> fetchParameters(final ConfigurationContext context) {
+        this.name = context.getName();
+
+        final List<Parameter> parameters = Objects.requireNonNull(this.fetchParameterList(context), "Fetched parameter list may not be null");

Review comment:
       Verbiage of this error message feels odd to me. This is what i would expect if a `null` value were passed in as an argument, perhaps, but not when the implementation returns a `null` value. I would also avoid throwing an Exception in this case. We want to be specific in what we provide to methods but lenient in what we expect. If the return value is null, it's easy enough to handle that in the same way we would handle an empty collection.

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProvider.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation.lifecycle.OnConfigurationRestored;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Map;
+
+/**
+ * Defines a provider that is responsible for fetching from an external source Parameters with
+ * which a ParameterContext can be populated.
+ *
+ * <p>
+ * <code>ParameterProvider</code>s are discovered using Java's
+ * <code>ServiceLoader</code> mechanism. As a result, all implementations must
+ * follow these rules:
+ *
+ * <ul>
+ * <li>The implementation must implement this interface.</li>
+ * <li>The implementation must have a file named
+ * org.apache.nifi.parameter.ParameterProvider located within the jar's
+ * <code>META-INF/services</code> directory. This file contains a list of
+ * fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
+ * one-per-line.
+ * <li>The implementation must support a default constructor.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * All implementations of this interface must be thread-safe.
+ * </p>
+ *
+ * <p>
+ * Parameter Providers may choose to annotate a method with the
+ * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
+ * will be invoked after all properties have been set for the ParameterProvider and
+ * before its parameters are fetched.
+ * </p>
+ */
+public interface ParameterProvider extends ConfigurableComponent {
+
+    /**
+     * Provides the Parameter Provider with access to objects that may be of use
+     * throughout the life of the service
+     *
+     * @param config of initialization context
+     * @throws org.apache.nifi.reporting.InitializationException if unable to init
+     */
+    void initialize(ParameterProviderInitializationContext config) throws InitializationException;
+
+    /**
+     * Fetches parameters from an external source.
+     * @param context The <code>ConfigurationContext</code>for the provider
+     * @return An unmodifiable map of available parameters, mapped from descriptor to Parameter

Review comment:
       Why do we document that the method must return an unmodifiable map? If the method wants to return a HashMap, I don't see any reason that would be a problem.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
##########
@@ -405,6 +421,66 @@ public void onReportingTaskAdded(final ReportingTaskNode taskNode) {
         allReportingTasks.put(taskNode.getIdentifier(), taskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProvider(final String id) {
+        if (id == null) {

Review comment:
       Any reason to check for `null` here explicitly? Can just call `get` on the map and it'll return null.

##########
File path: nifi-docs/src/main/asciidoc/developer-guide.adoc
##########
@@ -2023,6 +2029,64 @@ ReportingTasks, allowing reports to be generated
 in many different ways to expose metrics and monitoring capabilities
 needed for any number of operational concerns.
 
+== Parameter Providers
+
+Although Parameter Contexts cannot be extended, the `ParameterProvider`
+interface allows an extension point for providing parameters to
+Parameter Contexts.  Each Parameter Context can be configured
+optionally with a Sensitive and a Non-Sensitive `ParameterProvider`,
+and provided Parameters can be fetched and applied to their
+referencing Parameter Contexts either on-demand or periodically if
+opted in.
+
+This extensibility allows parameters to be both initially provided and
+managed external to NiFi.
+
+
+=== Developing a Parameter Provider
+
+The ParameterProvider interface exposes methods for
+configuration, validation, and initialization. These methods are all
+identical to those of the
+Processor and ControllerService interfaces except that the
+`initialize` method is passed a `ParameterProviderInitializationContext`
+object, as opposed to the initialization objects received by the other
+Components. The ParameterProvider also has
+a `fetchParameters` method that is invoked by the framework to fetch
+the parameters from its source.
+
+Within the `fetchParameters` method, the ParameterProvider is given access to a
+`ConfigurationContext`, from which configuration
+and information about the NiFi instance can be obtained. The
+BulletinRepository allows Bulletins to be queried
+and allows the ParameterProvider to submit its own Bulletins, so that
+information will be rendered to users. The

Review comment:
       The `ConfigurationContext` does not expose a BulletinRepository - nor should it, bulletins should be emitted simply by logging

##########
File path: nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AlertHandler.java
##########
@@ -132,7 +132,7 @@ protected void executeAction(PropertyContext propertyContext, Action action, Map
                 } catch (IllegalArgumentException iae) {
                     severity = Severity.INFO;
                 }
-                BulletinRepository bulletinRepository = context.getBulletinRepository();
+                BulletinRepository bulletinRepository = context. getBulletinRepository();

Review comment:
       Think this was a typo. :)

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/AbstractParameterProviderNode.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterProvider;
+import org.apache.nifi.parameter.SensitiveParameterProvider;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {
+
+    private final AtomicReference<ParameterProviderDetails> parameterProviderRef;
+    private final ControllerServiceLookup serviceLookup;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private final Set<ParameterContext> referencingParameterContexts;
+
+    private final Map<ParameterDescriptor, Parameter> fetchedParameters = new LinkedHashMap<>();
+
+    private volatile String comment;
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent,
+                                         final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
+
+        this(parameterProvider, id, controllerServiceProvider, validationContextFactory,
+                parameterProvider.getComponent().getClass().getSimpleName(), parameterProvider.getComponent().getClass().getCanonicalName(),
+                variableRegistry, reloadComponent, extensionManager, validationTrigger, false);
+    }
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
+                                         final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                         final boolean isExtensionMissing) {
+
+        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
+        this.parameterProviderRef = new AtomicReference<>(new ParameterProviderDetails(parameterProvider));
+        this.serviceLookup = controllerServiceProvider;
+        this.referencingParameterContexts = new HashSet<>();
+    }
+
+    @Override
+    public ConfigurableComponent getComponent() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public BundleCoordinate getBundleCoordinate() {
+        return parameterProviderRef.get().getBundleCoordinate();
+    }
+
+    @Override
+    public TerminationAwareLogger getLogger() {
+        return parameterProviderRef.get().getComponentLog();
+    }
+
+    @Override
+    public ParameterProvider getParameterProvider() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public void setParameterProvider(final LoggableComponent<ParameterProvider> parameterProvider) {
+        this.parameterProviderRef.set(new ParameterProviderDetails(parameterProvider));
+    }
+
+    @Override
+    public void reload(final Set<URL> additionalUrls) throws ParameterProviderInstantiationException {
+        String additionalResourcesFingerprint = ClassLoaderUtils.generateAdditionalUrlsFingerprint(additionalUrls);
+        setAdditionalResourcesFingerprint(additionalResourcesFingerprint);
+        getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+    }
+
+    @Override
+    public boolean isValidationNecessary() {
+        return getValidationStatus() != ValidationStatus.VALID;
+    }
+
+    @Override
+    public ConfigurationContext getConfigurationContext() {
+        return new StandardConfigurationContext(this, serviceLookup, null, getVariableRegistry());
+    }
+
+    @Override
+    public void verifyModifiable() throws IllegalStateException {
+
+    }
+
+    @Override
+    public String getComments() {
+        return comment;
+    }
+
+    @Override
+    public void setComments(final String comment) {
+        this.comment = CharacterFilterUtils.filterInvalidXmlCharacters(comment);
+    }
+
+    @Override
+    public void verifyCanClearState() {
+
+    }
+
+    @Override
+    public boolean isSensitiveParameterProvider() {
+        return parameterProviderRef.get().getParameterProvider() instanceof SensitiveParameterProvider;
+    }
+
+    @Override
+    public String toString() {
+        return "ParameterProvider[id=" + getIdentifier() + "]";
+    }
+
+    @Override
+    public String getProcessGroupIdentifier() {
+        return null;
+    }
+
+    @Override
+    public ParameterLookup getParameterLookup() {
+        return ParameterLookup.EMPTY;
+    }
+    @Override
+    public Set<ParameterContext> getReferences() {
+        readLock.lock();
+        try {
+            return Collections.unmodifiableSet(referencingParameterContexts);
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    @Override
+    public void addReference(final ParameterContext parameterContext) {
+        writeLock.lock();
+        try {
+            referencingParameterContexts.add(parameterContext);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void removeReference(final ParameterContext parameterContext) {
+        writeLock.lock();
+        try {
+            referencingParameterContexts.remove(parameterContext);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void verifyCanFetchParameters() {
+        final ValidationStatus validationStatus = performValidation();
+        if (validationStatus != ValidationStatus.VALID) {
+            throw new IllegalStateException(String.format("Parameter Provider [%s] cannot fetch parameters while validation state is %s",
+                    getIdentifier(), validationStatus));
+        }
+    }
+
+    @Override
+    public void fetchParameters() {
+        final ParameterProvider parameterProvider = parameterProviderRef.get().getParameterProvider();
+        final ConfigurationContext configurationContext = getConfigurationContext();
+        Map<ParameterDescriptor, Parameter> fetchedParameters;
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), parameterProvider.getClass(), parameterProvider.getIdentifier())) {
+            fetchedParameters = parameterProvider.fetchParameters(configurationContext);
+        }
+
+        for(final Map.Entry<ParameterDescriptor, Parameter> entry : fetchedParameters.entrySet()) {
+            final ParameterDescriptor descriptor = entry.getKey();
+            if (descriptor.isSensitive() != isSensitiveParameterProvider()) {
+                throw new IllegalStateException(String.format("Fetched parameter [%s] does not match the sensitivity of Parameter Provider [%s]",
+                        descriptor.getName(), configurationContext.getName()));
+            }
+        }
+
+        writeLock.lock();
+        try {
+            this.fetchedParameters.clear();
+            this.fetchedParameters.putAll(toProvidedParameters(fetchedParameters));
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void verifyCanApplyParameters(final Set<String> parameterNames) {
+        if (fetchedParameters.isEmpty()) {
+            throw new IllegalStateException("No parameters have been fetched from Parameter Provider " + getName());

Review comment:
       Not sure this needs to be an Exceptional case. Seems like a valid case to me.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/AbstractParameterProviderNode.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterProvider;
+import org.apache.nifi.parameter.SensitiveParameterProvider;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {
+
+    private final AtomicReference<ParameterProviderDetails> parameterProviderRef;
+    private final ControllerServiceLookup serviceLookup;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private final Set<ParameterContext> referencingParameterContexts;
+
+    private final Map<ParameterDescriptor, Parameter> fetchedParameters = new LinkedHashMap<>();
+
+    private volatile String comment;
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent,
+                                         final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
+
+        this(parameterProvider, id, controllerServiceProvider, validationContextFactory,
+                parameterProvider.getComponent().getClass().getSimpleName(), parameterProvider.getComponent().getClass().getCanonicalName(),
+                variableRegistry, reloadComponent, extensionManager, validationTrigger, false);
+    }
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
+                                         final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                         final boolean isExtensionMissing) {
+
+        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
+        this.parameterProviderRef = new AtomicReference<>(new ParameterProviderDetails(parameterProvider));
+        this.serviceLookup = controllerServiceProvider;
+        this.referencingParameterContexts = new HashSet<>();
+    }
+
+    @Override
+    public ConfigurableComponent getComponent() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public BundleCoordinate getBundleCoordinate() {
+        return parameterProviderRef.get().getBundleCoordinate();
+    }
+
+    @Override
+    public TerminationAwareLogger getLogger() {
+        return parameterProviderRef.get().getComponentLog();
+    }
+
+    @Override
+    public ParameterProvider getParameterProvider() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public void setParameterProvider(final LoggableComponent<ParameterProvider> parameterProvider) {
+        this.parameterProviderRef.set(new ParameterProviderDetails(parameterProvider));
+    }
+
+    @Override
+    public void reload(final Set<URL> additionalUrls) throws ParameterProviderInstantiationException {
+        String additionalResourcesFingerprint = ClassLoaderUtils.generateAdditionalUrlsFingerprint(additionalUrls);
+        setAdditionalResourcesFingerprint(additionalResourcesFingerprint);
+        getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+    }
+
+    @Override
+    public boolean isValidationNecessary() {

Review comment:
       The intent of this method is to indicate whether or not the component should be validated based on its state - enabled/disabled/started/stopped/etc. As this is written, it's currently saying that once it becomes valid, it can't become invalid. If this component doesn't have a notion of enabled/disabled, it should always return `true`, I think.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/parameter/ParameterProviderProvider.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.nar.ExtensionManager;
+
+import java.util.Set;
+
+/**
+ * A ParameterProviderProvider is responsible for providing management of, and
+ * access to, ParameterProviders.
+ */
+public interface ParameterProviderProvider {

Review comment:
       I don't think we need this interface. I'm guessing this was intended to mimic the ControllerServiceProvider. At this point, however, I would say that all of these methods - create/get/remove parameter provider - belong in `FlowManager`. The only place this interface is accessed, it appears, is the `StandardParameterProviderDAO`, and that has access already to the `FlowController`, which means it has access already to the `FlowManager`. Would opt for just deleting this interface.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterProviderInitializationContext.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.controller.kerberos.KerberosConfig;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.File;
+import java.util.Set;
+
+public class StandardParameterProviderInitializationContext implements ParameterProviderInitializationContext, ControllerServiceLookup {

Review comment:
       Should not implement `ControllerServiceLookup`. I don't believe there is any need for a `ControllerServiceLookup` at all for Parameter Providers. But if there is, the context should just have a `getControllerServiceLookup()` method instead of implementing `ControllerServiceLookup`. Then, it can just simply provide `ControllerServiceProvider`. I.e., prefer composition over inheritance.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
##########
@@ -2023,6 +2034,27 @@ public void removeReportingTask(final ReportingTaskNode reportingTaskNode) {
         flowManager.removeReportingTask(reportingTaskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProviderNode(final String identifier) {
+        return flowManager.getParameterProvider(identifier);
+    }
+
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded)
+            throws ParameterProviderInstantiationException {
+        return flowManager.createParameterProvider(type, id, bundleCoordinate, firstTimeAdded);
+    }
+
+    @Override
+    public Set<ParameterProviderNode> getAllParameterProviders() {
+        return flowManager.getAllParameterProviders();
+    }
+
+    @Override
+    public void removeParameterProvider(final ParameterProviderNode parameterProviderNode) {
+        flowManager.removeParameterProvider(parameterProviderNode);
+    }
+

Review comment:
       I think it makes more sense to return a ParameterProviderProvider here rather than implement the interface. But again, I don't think we need that interface at all - just have those methods added directly to FlowManager.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterProviderInitializationContext.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.controller.kerberos.KerberosConfig;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.File;
+import java.util.Set;
+
+public class StandardParameterProviderInitializationContext implements ParameterProviderInitializationContext, ControllerServiceLookup {
+
+    private final String id;
+    private final String name;
+    private final ControllerServiceProvider serviceProvider;
+    private final ComponentLog logger;
+    private final KerberosConfig kerberosConfig;
+    private final NodeTypeProvider nodeTypeProvider;
+
+    public StandardParameterProviderInitializationContext(final String id, final String name, final ComponentLog logger,
+                                                          final ControllerServiceProvider serviceProvider, final KerberosConfig kerberosConfig,
+                                                  final NodeTypeProvider nodeTypeProvider) {
+        this.id = id;
+        this.name = name;
+        this.serviceProvider = serviceProvider;
+        this.logger = logger;
+        this.kerberosConfig = kerberosConfig;
+        this.nodeTypeProvider = nodeTypeProvider;
+    }
+
+    @Override
+    public String getIdentifier() {
+        return id;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) {
+        return serviceProvider.getControllerServiceIdentifiers(serviceType, null);
+    }
+
+    @Override
+    public ControllerService getControllerService(final String identifier) {
+        return serviceProvider.getControllerService(identifier);
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final ControllerService service) {
+        return serviceProvider.isControllerServiceEnabled(service);
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+        return serviceProvider.isControllerServiceEnabled(serviceIdentifier);
+    }
+
+    @Override
+    public boolean isControllerServiceEnabling(final String serviceIdentifier) {
+        return serviceProvider.isControllerServiceEnabling(serviceIdentifier);
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return this;
+    }
+
+    @Override
+    public String getControllerServiceName(final String serviceIdentifier) {
+        return serviceProvider.getControllerServiceName(serviceIdentifier);
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return logger;
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return kerberosConfig.getPrincipal();

Review comment:
       I think it is valid for `KerberosConfig` to be null, so we should return `null` for the principal/keytable/configfile in that case.

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
##########
@@ -296,6 +298,53 @@ public ReportingTaskNode createReportingTask(final String type, final String id,
         return taskNode;
     }
 
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded,

Review comment:
       Then again, this might be able to take the place of ParamterValueProvider in stateless. So that way we have feature parity between the two. Need to understand better exactly how that would work....

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProvider.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation.lifecycle.OnConfigurationRestored;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Map;
+
+/**
+ * Defines a provider that is responsible for fetching from an external source Parameters with
+ * which a ParameterContext can be populated.
+ *
+ * <p>
+ * <code>ParameterProvider</code>s are discovered using Java's
+ * <code>ServiceLoader</code> mechanism. As a result, all implementations must
+ * follow these rules:
+ *
+ * <ul>
+ * <li>The implementation must implement this interface.</li>
+ * <li>The implementation must have a file named
+ * org.apache.nifi.parameter.ParameterProvider located within the jar's
+ * <code>META-INF/services</code> directory. This file contains a list of
+ * fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
+ * one-per-line.
+ * <li>The implementation must support a default constructor.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * All implementations of this interface must be thread-safe.
+ * </p>
+ *
+ * <p>
+ * Parameter Providers may choose to annotate a method with the
+ * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
+ * will be invoked after all properties have been set for the ParameterProvider and
+ * before its parameters are fetched.
+ * </p>
+ */
+public interface ParameterProvider extends ConfigurableComponent {
+
+    /**
+     * Provides the Parameter Provider with access to objects that may be of use
+     * throughout the life of the service
+     *
+     * @param config of initialization context
+     * @throws org.apache.nifi.reporting.InitializationException if unable to init
+     */
+    void initialize(ParameterProviderInitializationContext config) throws InitializationException;
+
+    /**
+     * Fetches parameters from an external source.
+     * @param context The <code>ConfigurationContext</code>for the provider
+     * @return An unmodifiable map of available parameters, mapped from descriptor to Parameter
+     */
+    Map<ParameterDescriptor, Parameter> fetchParameters(ConfigurationContext context);

Review comment:
       Given that the `Parameter` has a reference to the `ParameterDescriptor` does it make more sense to have this return a `List<Parameter>` or a `Set<Parameter>`?

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/SensitiveParameterProvider.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+/**
+ * A base interface for all sensitive <code>ParameterProvider</code>s
+ */
+public interface SensitiveParameterProvider extends ParameterProvider {

Review comment:
       I'm not a fan of marker interfaces generally. Makes more sense to me, if we need to differentiate the two, to instead use a method like:
   ```
   boolean isParameterSensitive();
   ```
   Or, better yet:
   ```
   ParameterSensitivity getParameterSensitivity();
   ```
   where `ParameterSensitivity` is an enum with two values: `SENSITIVE`, `NON_SENSITIVE` and possibly a value of `BOTH`

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/pom.xml
##########
@@ -0,0 +1,55 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-standard-bundle</artifactId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-standard-parameter-providers</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>

Review comment:
       This isn't a dependency that extension points should have. I don't think it's needed at all here, though. Think it's just an extraneous dependency that can be removed.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AbstractParameterResource.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.web.api;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.util.LifecycleManagementException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.HttpMethod;
+import java.net.URI;
+import java.util.Map;
+
+public abstract class AbstractParameterResource extends ApplicationResource {

Review comment:
       Not sure just this one simple method warrants introducing a new abstract class. Especially considering that it's only called by way of the `ParameterUpdateManager` - perhaps it makes more sense to just implement this logic in the `ParameterUpdateManager`?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterDTO.java
##########
@@ -30,8 +30,10 @@
     private Boolean sensitive;
     private String value;
     private Boolean valueRemoved;
+    private boolean provided;

Review comment:
       Generally we want to use Boolean objects instead of primitives in DTOs as a way to differentiate between the value being unset and being false.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ParameterProviderNode.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.controller;
+
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterProvider;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public interface ParameterProviderNode extends ComponentNode {
+
+    ParameterProvider getParameterProvider();
+
+    void setParameterProvider(LoggableComponent<ParameterProvider> parameterProvider);
+
+    ConfigurationContext getConfigurationContext();
+
+    String getComments();
+
+    void setComments(String comment);
+
+    void verifyCanFetchParameters();
+
+    void fetchParameters();
+
+    void verifyCanApplyParameters(Set<String> parameterNames);
+
+    Set<String> getFetchedParameterNames();
+
+    Map<ParameterContext, Map<String, Parameter>> getFetchedParametersToApply(Set<String> parameterNames);

Review comment:
       Not sure yet, but might be worth considering creating a `ParametersApplication` class or something instead of a `Map<ParameterContext, Map<String, Parameter>>`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
##########
@@ -405,6 +421,66 @@ public void onReportingTaskAdded(final ReportingTaskNode taskNode) {
         allReportingTasks.put(taskNode.getIdentifier(), taskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProvider(final String id) {
+        if (id == null) {
+            return null;
+        }
+        return allParameterProviders.get(id);
+    }
+
+    @Override
+    public void removeParameterProvider(final ParameterProviderNode parameterProvider) {
+        final ParameterProviderNode existing = allParameterProviders.get(parameterProvider.getIdentifier());
+        if (existing == null || existing != parameterProvider) {
+            throw new IllegalStateException("Parameter Provider " + parameterProvider + " does not exist in this Flow");
+        }
+
+        final Class<?> taskClass = parameterProvider.getParameterProvider().getClass();
+        try (final NarCloseable x = NarCloseable.withComponentNarLoader(getExtensionManager(), taskClass, parameterProvider.getParameterProvider().getIdentifier())) {
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, parameterProvider.getParameterProvider(), parameterProvider.getConfigurationContext());
+        }
+
+        for (final Map.Entry<PropertyDescriptor, String> entry : parameterProvider.getEffectivePropertyValues().entrySet()) {
+            final PropertyDescriptor descriptor = entry.getKey();
+            if (descriptor.getControllerServiceDefinition() != null) {
+                final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
+                if (value != null) {
+                    final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value);
+                    if (serviceNode != null) {
+                        serviceNode.removeReference(parameterProvider, descriptor);
+                    }
+                }
+            }
+        }
+        for(final ParameterContext parameterContext : parameterProvider.getReferences()) {
+            parameterContext.getSensitiveParameterProvider().ifPresent(sensitiveProvider -> {
+                if (parameterProvider.getIdentifier().equals(sensitiveProvider.getIdentifier())) {
+                    parameterContext.setSensitiveParameterProvider(null);
+                }
+            });
+            parameterContext.getNonSensitiveParameterProvider().ifPresent(nonSensitiveProvider -> {
+                if (parameterProvider.getIdentifier().equals(nonSensitiveProvider.getIdentifier())) {
+                    parameterContext.setNonSensitiveParameterProvider(null);
+                }
+            });
+        }
+
+        allParameterProviders.remove(parameterProvider.getIdentifier());
+        LogRepositoryFactory.removeRepository(parameterProvider.getIdentifier());
+
+        getExtensionManager().removeInstanceClassLoader(parameterProvider.getIdentifier());
+    }
+
+    @Override
+    public Set<ParameterProviderNode> getAllParameterProviders() {
+        return Collections.unmodifiableSet(new HashSet<>(allParameterProviders.values()));

Review comment:
       Given that this is already making a defensive copy by using `new HashSet<>()`, no need to wrap in an `unmodifiableSet`, is there?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#discussion_r719264527



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
##########
@@ -296,6 +298,53 @@ public ReportingTaskNode createReportingTask(final String type, final String id,
         return taskNode;
     }
 
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded,

Review comment:
       I agree some more careful thought is needed regarding these two concepts.  For this PR, I'll just throw an `UnsupportedOperationException` here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#discussion_r717933999



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/AbstractParameterProviderNode.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterProvider;
+import org.apache.nifi.parameter.SensitiveParameterProvider;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {

Review comment:
       Perhaps it just hasn't clicked yet, but I don't think we need to have an `AbstractParameterProviderNode`, a `StandardParameterProviderNode`, and a `StatelessParameterProviderNode`. Instead, I think we can have just a `StandardParameterProviderNode`. The only thing that I think may have caused an issue with that is that `StandardParameterProviderNode` as it stands takes the `FlowController` in its constructor. But that `FlowController` shouldn't be provided at all, I don't think. The only way it's used is to return it as the parent. But the parent could instead just be defined as an `Authorizable` so any `Authorizable` could be provided in the constructor instead. So in standard nifi it would take in the FlowController (referenced though as an `Authorizable`) and in stateless the parent would be `null`.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/src/main/java/org/apache/nifi/parameter/AbstractEnvironmentVariableParameterProvider.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public abstract class AbstractEnvironmentVariableParameterProvider extends AbstractParameterProvider {

Review comment:
       Again, I think we should really have just an EnvironmentVariableParameterProvider and not an abstract with children. Instead, the interface can have a method, if necessary, to determine whether provided parameters are sensitive or not.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardParameterProviderDAO.java
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.web.dao.impl;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.exception.ValidationException;
+import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
+import org.apache.nifi.controller.parameter.ParameterProviderProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.logging.LogRepository;
+import org.apache.nifi.logging.repository.NopLogRepository;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.processor.SimpleProcessLogger;
+import org.apache.nifi.util.BundleUtils;
+import org.apache.nifi.web.NiFiCoreException;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
+import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderDTO;
+import org.apache.nifi.web.dao.ComponentStateDAO;
+import org.apache.nifi.web.dao.ParameterProviderDAO;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class StandardParameterProviderDAO extends ComponentDAO implements ParameterProviderDAO {
+
+    private ParameterProviderProvider parameterProviderProvider;
+    private ComponentStateDAO componentStateDAO;
+    private ReloadComponent reloadComponent;
+    private FlowController flowController;
+
+    private ParameterProviderNode locateParameterProvider(final String parameterProviderId) {
+        // get the parameter provider
+        final ParameterProviderNode parameterProvider = parameterProviderProvider.getParameterProviderNode(parameterProviderId);
+
+        // ensure the parameter provider exists
+        if (parameterProvider == null) {
+            throw new ResourceNotFoundException(String.format("Unable to locate parameter provider with id '%s'.", parameterProviderId));
+        }
+
+        return parameterProvider;
+    }
+
+    @Override
+    public void verifyCreate(final ParameterProviderDTO parameterProviderDTO) {
+        verifyCreate(parameterProviderProvider.getExtensionManager(), parameterProviderDTO.getType(), parameterProviderDTO.getBundle());
+    }
+
+    @Override
+    public ParameterProviderNode createParameterProvider(final ParameterProviderDTO parameterProviderDTO) {
+        // ensure the type is specified
+        if (parameterProviderDTO.getType() == null) {
+            throw new IllegalArgumentException("The parameter provider type must be specified.");
+        }
+
+        try {
+            // create the parameter provider
+            final ExtensionManager extensionManager = parameterProviderProvider.getExtensionManager();
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, parameterProviderDTO.getType(), parameterProviderDTO.getBundle());
+            final ParameterProviderNode parameterProvider = parameterProviderProvider.createParameterProvider(
+                    parameterProviderDTO.getType(), parameterProviderDTO.getId(), bundleCoordinate, true);
+
+            // ensure we can perform the update
+            verifyUpdate(parameterProvider, parameterProviderDTO);
+
+            // perform the update
+            configureParameterProvider(parameterProvider, parameterProviderDTO);
+
+            return parameterProvider;
+        } catch (final ParameterProviderInstantiationException e) {
+            throw new NiFiCoreException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public ParameterProviderNode getParameterProvider(final String parameterProviderId) {
+        return locateParameterProvider(parameterProviderId);
+    }
+
+    @Override
+    public boolean hasParameterProvider(final String parameterProviderId) {
+        return parameterProviderProvider.getParameterProviderNode(parameterProviderId) != null;
+    }
+
+    @Override
+    public Set<ParameterProviderNode> getParameterProviders() {
+        return parameterProviderProvider.getAllParameterProviders();
+    }
+
+    @Override
+    public ParameterProviderNode updateParameterProvider(final ParameterProviderDTO parameterProviderDTO) {
+        // get the parameter provider
+        final ParameterProviderNode parameterProvider = locateParameterProvider(parameterProviderDTO.getId());
+
+        // ensure we can perform the update
+        verifyUpdate(parameterProvider, parameterProviderDTO);
+
+        // perform the update
+        configureParameterProvider(parameterProvider, parameterProviderDTO);
+
+        // attempt to change the underlying processor if an updated bundle is specified
+        // updating the bundle must happen after configuring so that any additional classpath resources are set first
+        updateBundle(parameterProvider, parameterProviderDTO);
+
+        return parameterProvider;
+    }
+
+    @Override
+    public ParameterProviderNode fetchParameters(final String parameterProviderId) {
+        final ParameterProviderNode parameterProviderNode = locateParameterProvider(parameterProviderId);
+
+        parameterProviderNode.fetchParameters();
+        return parameterProviderNode;
+    }
+
+    @Override
+    public void verifyCanApplyParameters(final String parameterProviderId, final Set<String> parameterNames) {
+        final ParameterProviderNode parameterProviderNode = locateParameterProvider(parameterProviderId);
+        parameterProviderNode.verifyCanApplyParameters(parameterNames);
+    }
+
+    @Override
+    public Map<ParameterContext, Map<String, Parameter>> getFetchedParametersToApply(final String parameterProviderId, final Set<String> parameterNames) {
+        final ParameterProviderNode parameterProviderNode = locateParameterProvider(parameterProviderId);
+
+        return parameterProviderNode.getFetchedParametersToApply(parameterNames);
+    }
+
+    private void updateBundle(final ParameterProviderNode parameterProvider, final ParameterProviderDTO parameterProviderDTO) {
+        final BundleDTO bundleDTO = parameterProviderDTO.getBundle();
+        if (bundleDTO != null) {
+            final ExtensionManager extensionManager = parameterProviderProvider.getExtensionManager();
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(extensionManager, parameterProvider.getCanonicalClassName(), bundleDTO);
+            final BundleCoordinate existingCoordinate = parameterProvider.getBundleCoordinate();
+            if (!existingCoordinate.getCoordinate().equals(incomingCoordinate.getCoordinate())) {
+                try {
+                    // we need to use the property descriptors from the temp component here in case we are changing from a ghost component to a real component
+                    final ConfigurableComponent tempComponent = extensionManager.getTempComponent(parameterProvider.getCanonicalClassName(), incomingCoordinate);
+                    final Set<URL> additionalUrls = parameterProvider.getAdditionalClasspathResources(tempComponent.getPropertyDescriptors());
+                    reloadComponent.reload(parameterProvider, parameterProvider.getCanonicalClassName(), incomingCoordinate, additionalUrls);
+                } catch (final ParameterProviderInstantiationException e) {
+                    throw new NiFiCoreException(String.format("Unable to update parameter provider %s from %s to %s due to: %s",
+                            parameterProviderDTO.getId(), parameterProvider.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
+                }
+            }
+        }
+    }
+
+    // A placeholder in case validation is required in the future
+    private List<String> validateProposedConfiguration(final ParameterProviderNode parameterProvider, final ParameterProviderDTO parameterProviderDTO) {
+        final List<String> validationErrors = new ArrayList<>();

Review comment:
       Should probably just `return Collections.emptyList();`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterContextDTO.java
##########
@@ -88,8 +91,42 @@ public void setInheritedParameterContexts(final List<ParameterContextReferenceEn
         return boundProcessGroups;
     }
 
+    @ApiModelProperty("An optional Parameter Provider for sensitive Parameters")
+    public ComponentReferenceEntity getSensitiveParameterProviderRef() {
+        return sensitiveParameterProviderRef;
+    }
+
+    public void setSensitiveParameterProviderRef(final ComponentReferenceEntity sensitiveParameterProviderRef) {
+        this.sensitiveParameterProviderRef = sensitiveParameterProviderRef;
+    }
+
+    @ApiModelProperty("An optional Parameter Provider for non-sensitive Parameters")
+    public ComponentReferenceEntity getNonSensitiveParameterProviderRef() {
+        return nonSensitiveParameterProviderRef;
+    }
+
+    public void setNonSensitiveParameterProviderRef(final ComponentReferenceEntity nonSensitiveParameterProviderRef) {
+        this.nonSensitiveParameterProviderRef = nonSensitiveParameterProviderRef;
+    }
+
     @Override
     public String toString() {
         return "ParameterContext[id=" + identifier + ", name=" + name + ", parameters=" + parameters + "]";
     }
+
+    /**
+     * A utility method for safely returning the identifier of a ComponentReferenceEntity.
+     * @param referenceEntity A ComponentReferenceEntity
+     * @return The <code>referenceEntity.getComponent().getId()</code> (may be null)
+     */
+    public static String getReferenceId(final ComponentReferenceEntity referenceEntity) {

Review comment:
       I haven't seen where this is used yet but it seems a bit odd to have a static get method like this in a DTO. Especially when it's not even used by the DTO.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java
##########
@@ -99,12 +99,12 @@ public void setType(String type) {
     }
 
     /**
-     * The details of the artifact that bundled this processor type.
+     * The details of the artifact that bundled this reporting task type.

Review comment:
       Ha nice catch, good to know someone reads the documentation :)

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ParameterProviderProviderFactoryBean.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.web.spring;
+
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.parameter.ParameterProviderProvider;
+import org.springframework.beans.BeansException;
+import org.springframework.beans.factory.FactoryBean;
+import org.springframework.context.ApplicationContext;
+import org.springframework.context.ApplicationContextAware;
+
+/**
+ *
+ */
+public class ParameterProviderProviderFactoryBean implements FactoryBean<ParameterProviderProvider>, ApplicationContextAware {

Review comment:
       If we eliminate the ParameterProviderProvider class, we can eliminate this as well.

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProviderInitializationContext.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.kerberos.KerberosContext;
+import org.apache.nifi.logging.ComponentLog;
+
+public interface ParameterProviderInitializationContext extends KerberosContext {
+
+    /**
+     * @return the identifier associated with the {@link ParameterProvider} with
+     * which this context is associated
+     */
+    String getIdentifier();
+
+    /**
+     * @return the configured name for this ParameterProvider
+     */
+    String getName();
+
+    /**
+     * @return the {@link ControllerServiceLookup} which can be used to obtain
+     * Controller Services
+     */
+    ControllerServiceLookup getControllerServiceLookup();

Review comment:
       I don't think we need the ControllerServiceLookup, do we? I think this exists in the API for processors, etc. because it came before the ability to call getProperty(...).asControllerService(...) but now that should be used.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterProviderResource.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.web.api;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiParam;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
+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.ui.extension.UiExtension;
+import org.apache.nifi.ui.extension.UiExtensionMapping;
+import org.apache.nifi.web.NiFiServiceFacade;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.ResumeFlowException;
+import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.UiExtensionType;
+import org.apache.nifi.web.api.concurrent.AsyncRequestManager;
+import org.apache.nifi.web.api.concurrent.AsynchronousWebRequest;
+import org.apache.nifi.web.api.concurrent.RequestManager;
+import org.apache.nifi.web.api.concurrent.StandardAsynchronousWebRequest;
+import org.apache.nifi.web.api.concurrent.StandardUpdateStep;
+import org.apache.nifi.web.api.concurrent.UpdateStep;
+import org.apache.nifi.web.api.dto.BundleDTO;
+import org.apache.nifi.web.api.dto.ComponentStateDTO;
+import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
+import org.apache.nifi.web.api.dto.DtoFactory;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderApplyParametersRequestDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderApplyParametersUpdateStepDTO;
+import org.apache.nifi.web.api.dto.ParameterProviderDTO;
+import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.VerifyParameterProviderConfigRequestDTO;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+import org.apache.nifi.web.api.entity.ComponentStateEntity;
+import org.apache.nifi.web.api.entity.Entity;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.api.entity.ParameterContextUpdateEntity;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderApplyParametersRequestEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderParameterApplicationEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderParameterFetchEntity;
+import org.apache.nifi.web.api.entity.ParameterProviderReferencingComponentsEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
+import org.apache.nifi.web.api.entity.VerifyParameterProviderConfigRequestEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+import org.apache.nifi.web.api.request.LongParameter;
+import org.apache.nifi.web.util.ComponentLifecycle;
+import org.apache.nifi.web.util.ParameterUpdateManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * RESTful endpoint for managing a Parameter Provider.
+ */
+@Path("/parameter-providers")
+@Api(
+        value = "/parameter-providers",
+        description = "Endpoint for managing a Parameter Provider."
+)
+public class ParameterProviderResource extends AbstractParameterResource {
+    private static final Logger logger = LoggerFactory.getLogger(ParameterProviderResource.class);
+
+    private NiFiServiceFacade serviceFacade;
+    private DtoFactory dtoFactory;
+    private Authorizer authorizer;
+    private ParameterUpdateManager parameterUpdateManager;
+
+    private static final String VERIFICATION_REQUEST_TYPE = "verification-request";
+    private RequestManager<VerifyParameterProviderConfigRequestEntity, List<ConfigVerificationResultDTO>> configVerificationRequestManager =
+            new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Verify Parameter Provider Config Thread");
+
+    private RequestManager<List<ParameterContextEntity>, List<ParameterContextEntity>> updateRequestManager =
+            new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Parameter Provider Apply Thread");
+
+    private ComponentLifecycle clusterComponentLifecycle;
+    private ComponentLifecycle localComponentLifecycle;
+
+    @Context
+    private ServletContext servletContext;
+
+    public void init() {
+        parameterUpdateManager = new ParameterUpdateManager(serviceFacade, dtoFactory, authorizer, this);
+    }
+
+    private void authorizeReadParameterProvider(final String parameterProviderId) {
+        if (parameterProviderId == null) {
+            throw new IllegalArgumentException("Parameter Provider ID must be specified");
+        }
+
+        serviceFacade.authorizeAccess(lookup -> {
+            final ComponentAuthorizable parameterProvider = lookup.getParameterProvider(parameterProviderId);
+            parameterProvider.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+    }
+
+    /**
+     * Populate the uri's for the specified parameter providers.
+     *
+     * @param parameterProviderEntities parameter providers
+     * @return dtos
+     */
+    public Set<ParameterProviderEntity> populateRemainingParameterProviderEntitiesContent(final Set<ParameterProviderEntity> parameterProviderEntities) {
+        for (ParameterProviderEntity parameterProviderEntity : parameterProviderEntities) {
+            populateRemainingParameterProviderEntityContent(parameterProviderEntity);
+        }
+        return parameterProviderEntities;
+    }
+
+    /**
+     * Populate the uri's for the specified parameter provider.
+     *
+     * @param parameterProviderEntity parameter provider
+     * @return dtos
+     */
+    public ParameterProviderEntity populateRemainingParameterProviderEntityContent(final ParameterProviderEntity parameterProviderEntity) {
+        parameterProviderEntity.setUri(generateResourceUri("parameter-providers", parameterProviderEntity.getId()));
+
+        // populate the remaining content
+        if (parameterProviderEntity.getComponent() != null) {
+            populateRemainingParameterProviderContent(parameterProviderEntity.getComponent());
+        }
+        return parameterProviderEntity;
+    }
+
+    /**
+     * Populates the uri for the specified parameter provider.
+     */
+    public ParameterProviderDTO populateRemainingParameterProviderContent(final ParameterProviderDTO parameterProvider) {
+        final BundleDTO bundle = parameterProvider.getBundle();
+        if (bundle == null) {
+            return parameterProvider;
+        }
+
+        // see if this processor has any ui extensions
+        final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions");
+        if (uiExtensionMapping.hasUiExtension(parameterProvider.getType(), bundle.getGroup(), bundle.getArtifact(), bundle.getVersion())) {
+            final List<UiExtension> uiExtensions = uiExtensionMapping.getUiExtension(parameterProvider.getType(), bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
+            for (final UiExtension uiExtension : uiExtensions) {
+                if (UiExtensionType.ParameterProviderConfiguration.equals(uiExtension.getExtensionType())) {
+                    parameterProvider.setCustomUiUrl(uiExtension.getContextPath() + "/configure");
+                }
+            }
+        }
+
+        return parameterProvider;
+    }
+
+    /**
+     * Retrieves the specified parameter provider.
+     *
+     * @param id The id of the parameter provider to retrieve
+     * @return A parameterProviderEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    @ApiOperation(
+            value = "Gets a parameter provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getParameterProvider(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the parameter provider
+        final ParameterProviderEntity parameterProvider = serviceFacade.getParameterProvider(id);
+        populateRemainingParameterProviderEntityContent(parameterProvider);
+
+        return generateOkResponse(parameterProvider).build();
+    }
+
+    /**
+     * Retrieves the references of the specified parameter provider.
+     *
+     * @param id The id of the parameter provider to retrieve
+     * @return A parameterProviderEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/references")
+    @ApiOperation(
+            value = "Gets all references to a parameter provider",
+            response = ParameterProviderReferencingComponentsEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getParameterProviderReferences(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the parameter provider references
+        final ParameterProviderReferencingComponentsEntity entity = serviceFacade.getParameterProviderReferencingComponents(id);
+
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Returns the descriptor for the specified property.
+     *
+     * @param id           The id of the parameter provider.
+     * @param propertyName The property
+     * @return a propertyDescriptorEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/descriptors")
+    @ApiOperation(
+            value = "Gets a parameter provider property descriptor",
+            response = PropertyDescriptorEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getPropertyDescriptor(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id,
+            @ApiParam(
+                    value = "The property name.",
+                    required = true
+            )
+            @QueryParam("propertyName") final String propertyName) {
+
+        // ensure the property name is specified
+        if (propertyName == null) {
+            throw new IllegalArgumentException("The property name must be specified.");
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the property descriptor
+        final PropertyDescriptorDTO descriptor = serviceFacade.getParameterProviderPropertyDescriptor(id, propertyName);
+
+        // generate the response entity
+        final PropertyDescriptorEntity entity = new PropertyDescriptorEntity();
+        entity.setPropertyDescriptor(descriptor);
+
+        // generate the response
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Gets the state for a parameter provider.
+     *
+     * @param id The id of the parameter provider
+     * @return a componentStateEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/state")
+    @ApiOperation(
+            value = "Gets the state for a parameter provider",
+            response = ComponentStateEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getState(
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        serviceFacade.authorizeAccess(lookup -> {
+            final Authorizable parameterProvider = lookup.getParameterProvider(id).getAuthorizable();
+            parameterProvider.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+        });
+
+        // get the component state
+        final ComponentStateDTO state = serviceFacade.getParameterProviderState(id);
+
+        // generate the response entity
+        final ComponentStateEntity entity = new ComponentStateEntity();
+        entity.setComponentState(state);
+
+        // generate the response
+        return generateOkResponse(entity).build();
+    }
+
+    /**
+     * Clears the state for a parameter provider.
+     *
+     * @param httpServletRequest servlet request
+     * @param id                 The id of the parameter provider
+     * @return a componentStateEntity
+     */
+    @POST
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/state/clear-requests")
+    @ApiOperation(
+            value = "Clears the state for a parameter provider",
+            response = ComponentStateEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response clearState(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST);
+        }
+
+        final ParameterProviderEntity requestParameterProviderEntity = new ParameterProviderEntity();
+        requestParameterProviderEntity.setId(id);
+
+        return withWriteLock(
+                serviceFacade,
+                requestParameterProviderEntity,
+                lookup -> {
+                    final Authorizable processor = lookup.getParameterProvider(id).getAuthorizable();
+                    processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                },
+                () -> serviceFacade.verifyCanClearParameterProviderState(id),
+                (parameterProviderEntity) -> {
+                    // get the component state
+                    serviceFacade.clearParameterProviderState(parameterProviderEntity.getId());
+
+                    // generate the response entity
+                    final ComponentStateEntity entity = new ComponentStateEntity();
+
+                    // generate the response
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+    /**
+     * Updates the specified a Parameter Provider.
+     *
+     * @param httpServletRequest  request
+     * @param id                  The id of the parameter provider to update.
+     * @param requestParameterProviderEntity A parameterProviderEntity.
+     * @return A parameterProviderEntity.
+     */
+    @PUT
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    @ApiOperation(
+            value = "Updates a parameter provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}"),
+                    @Authorization(value = "Read - any referenced Controller Services if this request changes the reference - /controller-services/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response updateParameterProvider(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String id,
+            @ApiParam(
+                    value = "The parameter provider configuration details.",
+                    required = true
+            ) final ParameterProviderEntity requestParameterProviderEntity) {
+
+        if (requestParameterProviderEntity == null || requestParameterProviderEntity.getComponent() == null) {
+            throw new IllegalArgumentException("Parameter provider details must be specified.");
+        }
+
+        if (requestParameterProviderEntity.getRevision() == null) {
+            throw new IllegalArgumentException("Revision must be specified.");
+        }
+
+        // ensure the ids are the same
+        final ParameterProviderDTO requestParameterProviderDTO = requestParameterProviderEntity.getComponent();
+        if (!id.equals(requestParameterProviderDTO.getId())) {
+            throw new IllegalArgumentException(String.format("The parameter provider id (%s) in the request body does not equal the "
+                    + "parameter provider id of the requested resource (%s).", requestParameterProviderDTO.getId(), id));
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.PUT, requestParameterProviderEntity);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(requestParameterProviderEntity.isDisconnectedNodeAcknowledged());
+        }
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = getRevision(requestParameterProviderEntity, id);
+        return withWriteLock(
+                serviceFacade,
+                requestParameterProviderEntity,
+                requestRevision,
+                lookup -> {
+                    // authorize parameter provider
+                    final ComponentAuthorizable authorizable = lookup.getParameterProvider(id);
+                    authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // authorize any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(requestParameterProviderDTO.getProperties(), authorizable, authorizer, lookup);
+                },
+                () -> serviceFacade.verifyUpdateParameterProvider(requestParameterProviderDTO),
+                (revision, parameterProviderEntity) -> {
+                    final ParameterProviderDTO parameterProviderDTO = parameterProviderEntity.getComponent();
+
+                    // update the parameter provider
+                    final ParameterProviderEntity entity = serviceFacade.updateParameterProvider(revision, parameterProviderDTO);
+                    populateRemainingParameterProviderEntityContent(entity);
+
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+    /**
+     * Removes the specified parameter provider.
+     *
+     * @param httpServletRequest request
+     * @param version            The revision is used to verify the client is working with
+     *                           the latest version of the flow.
+     * @param clientId           Optional client id. If the client id is not specified, a
+     *                           new one will be generated. This value (whether specified or generated) is
+     *                           included in the response.
+     * @param id                 The id of the parameter provider to remove.
+     * @return A entity containing the client id and an updated revision.
+     */
+    @DELETE
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    @ApiOperation(
+            value = "Deletes a parameter provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid}"),
+                    @Authorization(value = "Write - /controller"),
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response removeParameterProvider(
+            @Context HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The revision is used to verify the client is working with the latest version of the flow.",
+                    required = false
+            )
+            @QueryParam(VERSION) LongParameter version,
+            @ApiParam(
+                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
+                    required = false
+            )
+            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @QueryParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") String id) {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.DELETE);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(disconnectedNodeAcknowledged);
+        }
+
+        final ParameterProviderEntity requestParameterProviderEntity = new ParameterProviderEntity();
+        requestParameterProviderEntity.setId(id);
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
+        return withWriteLock(
+                serviceFacade,
+                requestParameterProviderEntity,
+                requestRevision,
+                lookup -> {
+                    final ComponentAuthorizable parameterProvider = lookup.getParameterProvider(id);
+
+                    // ensure write permission to the parameter provider
+                    parameterProvider.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // ensure write permission to the parent process group
+                    parameterProvider.getAuthorizable().getParentAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // verify any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(parameterProvider, authorizer, lookup, false);
+                },
+                () -> serviceFacade.verifyDeleteParameterProvider(id),
+                (revision, parameterProviderEntity) -> {
+                    // delete the specified parameter provider
+                    final ParameterProviderEntity entity = serviceFacade.deleteParameterProvider(revision, parameterProviderEntity.getId());
+                    return generateOkResponse(entity).build();
+                }
+        );
+    }
+
+    /**
+     * Tells the Parameter Provider to fetch its parameters.  This will temporarily cache the fetched parameters,
+     * but the changes will not be applied to the flow until an "apply-parameters-requests" request is created.
+     *
+     * @param httpServletRequest  request
+     * @param parameterProviderId The id of the parameter provider.
+     * @return A parameterProviderEntity.
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/parameters/fetch-requests")
+    @ApiOperation(
+            value = "Fetches and temporarily caches the parameters for a provider",
+            response = ParameterProviderEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /parameter-providers/{uuid} or  or /operation/parameter-providers/{uuid}")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response fetchParameters(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The parameter provider id.",
+                    required = true
+            )
+            @PathParam("id") final String parameterProviderId,
+            @ApiParam(
+                    value = "The parameter fetch request.",
+                    required = true
+            ) final ParameterProviderParameterFetchEntity fetchParametersEntity) {
+
+        if (fetchParametersEntity.getId() == null) {
+            throw new IllegalArgumentException("The ID of the Parameter Provider must be specified");
+        }
+        if (!fetchParametersEntity.getId().equals(parameterProviderId)) {
+            throw new IllegalArgumentException("The ID of the Parameter Provider must match the ID specified in the URL's path");
+        }
+
+        if (fetchParametersEntity.getRevision() == null) {
+            throw new IllegalArgumentException("Revision must be specified.");
+        }
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST, fetchParametersEntity);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(fetchParametersEntity.isDisconnectedNodeAcknowledged());
+        }
+
+        // handle expects request (usually from the cluster manager)
+        final Revision requestRevision = getRevision(fetchParametersEntity.getRevision(), parameterProviderId);
+        return withWriteLock(
+                serviceFacade,
+                fetchParametersEntity,
+                requestRevision,
+                lookup -> {
+                    // authorize parameter provider
+                    final ComponentAuthorizable authorizable = lookup.getParameterProvider(parameterProviderId);
+                    authorizable.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+                    authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                },
+                () -> serviceFacade.verifyCanFetchParameters(parameterProviderId),
+                (revision, parameterProviderFetchEntity) -> {
+                    // fetch the parameters
+                    final ParameterProviderEntity entity = serviceFacade.fetchParameters(parameterProviderId);

Review comment:
       Rather than use `parameterProviderId` here, we need to use `parameterProviderFetchEntity.getId()`. Because this is a mutable request it's 2 phases. Need to ensure that the entity of the first request is what we use, and that's what is provided by `parameterProviderFetchEntity`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
##########
@@ -198,7 +198,7 @@ public void testParameterCreatedWithNullValueOnImportWithSensitivePropertyRefere
         getRootGroup().addProcessGroup(innerGroup);
 
         final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(getFlowController().getFlowManager());
-        final ParameterContext parameterContext = new StandardParameterContext("param-context-id", "parameter-context", parameterReferenceManager, null);
+        final ParameterContext parameterContext = new StandardParameterContext("param-context-id", "parameter-context", parameterReferenceManager, null, null, null, null);

Review comment:
       I'm seeing a lot of this same type of thing, where we add 3 `null` values to the constructor. And there are becoming quite a few arguments to the constructor. Time to consider a Builder pattern perhaps?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
##########
@@ -3143,6 +3241,128 @@ public ReportingTaskEntity deleteReportingTask(final Revision revision, final St
         return entityFactory.createReportingTaskEntity(snapshot, null, permissions, operatePermissions, null);
     }
 
+    @Override
+    public ParameterProviderEntity createParameterProvider(final Revision revision, final ParameterProviderDTO parameterProviderDTO) {
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        // request claim for component to be created... revision already verified (version == 0)
+        final RevisionClaim claim = new StandardRevisionClaim(revision);
+
+        // update revision through revision manager
+        final RevisionUpdate<ParameterProviderDTO> snapshot = revisionManager.updateRevision(claim, user, () -> {
+            // create the parameter provider
+            final ParameterProviderNode parameterProvider = parameterProviderDAO.createParameterProvider(parameterProviderDTO);
+
+            // save the update
+            controllerFacade.save();
+            awaitValidationCompletion(parameterProvider);
+
+            final ParameterProviderDTO dto = dtoFactory.createParameterProviderDto(parameterProvider);
+            final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity());
+            return new StandardRevisionUpdate<>(dto, lastMod);
+        });
+
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderDTO.getId());
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterProvider);
+        final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(parameterProvider));
+        final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(parameterProvider.getIdentifier()));
+        final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
+        return entityFactory.createParameterProviderEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities);
+    }
+
+    @Override
+    public ParameterProviderEntity updateParameterProvider(final Revision revision, final ParameterProviderDTO parameterProviderDTO) {
+        // get the component, ensure we have access to it, and perform the update request
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderDTO.getId());
+        final RevisionUpdate<ParameterProviderDTO> snapshot = updateComponent(revision,
+                parameterProvider,
+                () -> parameterProviderDAO.updateParameterProvider(parameterProviderDTO),
+                rt -> {
+                    awaitValidationCompletion(rt);
+                    return dtoFactory.createParameterProviderDto(rt);
+                });
+
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterProvider);
+        final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(parameterProvider));
+        final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(parameterProvider.getIdentifier()));
+        final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
+        return entityFactory.createParameterProviderEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities);
+    }
+
+    @Override
+    public List<ConfigVerificationResultDTO> verifyParameterProviderConfiguration(final String parameterProviderId, final ParameterProviderDTO parameterProvider) {
+        return parameterProviderDAO.verifyConfiguration(parameterProviderId, parameterProvider);
+    }
+
+    @Override
+    public ParameterProviderEntity deleteParameterProvider(final Revision revision, final String parameterProviderId) {
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderId);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(parameterProvider);
+        final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(parameterProvider));
+        final ParameterProviderDTO snapshot = deleteComponent(
+                revision,
+                parameterProvider.getResource(),
+                () -> parameterProviderDAO.deleteParameterProvider(parameterProviderId),
+                true,
+                dtoFactory.createParameterProviderDto(parameterProvider));
+
+        return entityFactory.createParameterProviderEntity(snapshot, null, permissions, operatePermissions, null);
+    }
+
+    @Override
+    public ParameterProviderEntity fetchParameters(final String parameterProviderId) {
+        final ParameterProviderNode parameterProvider = parameterProviderDAO.getParameterProvider(parameterProviderId);
+
+        parameterProvider.fetchParameters();
+        awaitValidationCompletion(parameterProvider);

Review comment:
       Shouldn't we validate the parameter provider *before* fetching parameters?

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
##########
@@ -296,6 +298,53 @@ public ReportingTaskNode createReportingTask(final String type, final String id,
         return taskNode;
     }
 
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded,

Review comment:
       Within Stateless, we have a ParameterValueProvider, which is capable of fetching parameter values. This notion of a ParameterProvider is a bit different, where users have the ability to go fetch parameters and apply them. I'm not sure we need to support this at all, TBH, in stateless. Can probably just have this `throw new UnsupportedOperationException();` no?

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProvider.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation.lifecycle.OnConfigurationRestored;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Map;
+
+/**
+ * Defines a provider that is responsible for fetching from an external source Parameters with
+ * which a ParameterContext can be populated.
+ *
+ * <p>
+ * <code>ParameterProvider</code>s are discovered using Java's
+ * <code>ServiceLoader</code> mechanism. As a result, all implementations must
+ * follow these rules:
+ *
+ * <ul>
+ * <li>The implementation must implement this interface.</li>
+ * <li>The implementation must have a file named
+ * org.apache.nifi.parameter.ParameterProvider located within the jar's
+ * <code>META-INF/services</code> directory. This file contains a list of
+ * fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
+ * one-per-line.
+ * <li>The implementation must support a default constructor.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * All implementations of this interface must be thread-safe.
+ * </p>
+ *
+ * <p>
+ * Parameter Providers may choose to annotate a method with the
+ * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
+ * will be invoked after all properties have been set for the ParameterProvider and
+ * before its parameters are fetched.
+ * </p>
+ */
+public interface ParameterProvider extends ConfigurableComponent {
+
+    /**
+     * Provides the Parameter Provider with access to objects that may be of use
+     * throughout the life of the service
+     *
+     * @param config of initialization context
+     * @throws org.apache.nifi.reporting.InitializationException if unable to init
+     */
+    void initialize(ParameterProviderInitializationContext config) throws InitializationException;
+
+    /**
+     * Fetches parameters from an external source.
+     * @param context The <code>ConfigurationContext</code>for the provider
+     * @return An unmodifiable map of available parameters, mapped from descriptor to Parameter
+     */
+    Map<ParameterDescriptor, Parameter> fetchParameters(ConfigurationContext context);

Review comment:
       I think this should be able to throw IOException.

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/AbstractParameterProvider.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.AbstractConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+public abstract class AbstractParameterProvider extends AbstractConfigurableComponent implements ParameterProvider {
+    private String identifier;
+    private String name;
+    private ControllerServiceLookup serviceLookup;
+    private ComponentLog logger;
+    private NodeTypeProvider nodeTypeProvider;
+
+    @Override
+    public final void initialize(final ParameterProviderInitializationContext config) throws InitializationException {
+        identifier = config.getIdentifier();
+        logger = config.getLogger();
+        name = config.getName();
+        serviceLookup = config.getControllerServiceLookup();
+        nodeTypeProvider = config.getNodeTypeProvider();
+        verifyInterfaces();
+        init(config);
+    }
+
+    private void verifyInterfaces() {
+        if (!(this instanceof SensitiveParameterProvider) && !(this instanceof NonSensitiveParameterProvider)) {
+            throw new IllegalStateException(String.format("Parameter Provider [%s] must be either a SensitiveParameterProvider or a NonSensitiveParameterProvider, but is neither",
+                    name));
+        }
+        if ((this instanceof SensitiveParameterProvider) && (this instanceof NonSensitiveParameterProvider)) {
+            throw new IllegalStateException(String.format("Parameter Provider [%s] must be either a SensitiveParameterProvider or a NonSensitiveParameterProvider, but is both",
+                    name));
+        }
+    }
+
+    /**
+     * @return the identifier of this Parameter Provider
+     */
+    @Override
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    /**
+     * Calls {@link AbstractParameterProvider#fetchParameterList(ConfigurationContext)} and returns the parameters
+     * as a map.
+     * @param context The configuration context
+     * @return A map from <code>ParameterDescriptor</code> to <code>Parameter</code>
+     */
+    @Override
+    public final Map<ParameterDescriptor, Parameter> fetchParameters(final ConfigurationContext context) {
+        this.name = context.getName();
+
+        final List<Parameter> parameters = Objects.requireNonNull(this.fetchParameterList(context), "Fetched parameter list may not be null");

Review comment:
       Verbiage of this error message feels odd to me. This is what i would expect if a `null` value were passed in as an argument, perhaps, but not when the implementation returns a `null` value. I would also avoid throwing an Exception in this case. We want to be specific in what we provide to methods but lenient in what we expect. If the return value is null, it's easy enough to handle that in the same way we would handle an empty collection.

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProvider.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation.lifecycle.OnConfigurationRestored;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Map;
+
+/**
+ * Defines a provider that is responsible for fetching from an external source Parameters with
+ * which a ParameterContext can be populated.
+ *
+ * <p>
+ * <code>ParameterProvider</code>s are discovered using Java's
+ * <code>ServiceLoader</code> mechanism. As a result, all implementations must
+ * follow these rules:
+ *
+ * <ul>
+ * <li>The implementation must implement this interface.</li>
+ * <li>The implementation must have a file named
+ * org.apache.nifi.parameter.ParameterProvider located within the jar's
+ * <code>META-INF/services</code> directory. This file contains a list of
+ * fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
+ * one-per-line.
+ * <li>The implementation must support a default constructor.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * All implementations of this interface must be thread-safe.
+ * </p>
+ *
+ * <p>
+ * Parameter Providers may choose to annotate a method with the
+ * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
+ * will be invoked after all properties have been set for the ParameterProvider and
+ * before its parameters are fetched.
+ * </p>
+ */
+public interface ParameterProvider extends ConfigurableComponent {
+
+    /**
+     * Provides the Parameter Provider with access to objects that may be of use
+     * throughout the life of the service
+     *
+     * @param config of initialization context
+     * @throws org.apache.nifi.reporting.InitializationException if unable to init
+     */
+    void initialize(ParameterProviderInitializationContext config) throws InitializationException;
+
+    /**
+     * Fetches parameters from an external source.
+     * @param context The <code>ConfigurationContext</code>for the provider
+     * @return An unmodifiable map of available parameters, mapped from descriptor to Parameter

Review comment:
       Why do we document that the method must return an unmodifiable map? If the method wants to return a HashMap, I don't see any reason that would be a problem.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
##########
@@ -405,6 +421,66 @@ public void onReportingTaskAdded(final ReportingTaskNode taskNode) {
         allReportingTasks.put(taskNode.getIdentifier(), taskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProvider(final String id) {
+        if (id == null) {

Review comment:
       Any reason to check for `null` here explicitly? Can just call `get` on the map and it'll return null.

##########
File path: nifi-docs/src/main/asciidoc/developer-guide.adoc
##########
@@ -2023,6 +2029,64 @@ ReportingTasks, allowing reports to be generated
 in many different ways to expose metrics and monitoring capabilities
 needed for any number of operational concerns.
 
+== Parameter Providers
+
+Although Parameter Contexts cannot be extended, the `ParameterProvider`
+interface allows an extension point for providing parameters to
+Parameter Contexts.  Each Parameter Context can be configured
+optionally with a Sensitive and a Non-Sensitive `ParameterProvider`,
+and provided Parameters can be fetched and applied to their
+referencing Parameter Contexts either on-demand or periodically if
+opted in.
+
+This extensibility allows parameters to be both initially provided and
+managed external to NiFi.
+
+
+=== Developing a Parameter Provider
+
+The ParameterProvider interface exposes methods for
+configuration, validation, and initialization. These methods are all
+identical to those of the
+Processor and ControllerService interfaces except that the
+`initialize` method is passed a `ParameterProviderInitializationContext`
+object, as opposed to the initialization objects received by the other
+Components. The ParameterProvider also has
+a `fetchParameters` method that is invoked by the framework to fetch
+the parameters from its source.
+
+Within the `fetchParameters` method, the ParameterProvider is given access to a
+`ConfigurationContext`, from which configuration
+and information about the NiFi instance can be obtained. The
+BulletinRepository allows Bulletins to be queried
+and allows the ParameterProvider to submit its own Bulletins, so that
+information will be rendered to users. The

Review comment:
       The `ConfigurationContext` does not expose a BulletinRepository - nor should it, bulletins should be emitted simply by logging

##########
File path: nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AlertHandler.java
##########
@@ -132,7 +132,7 @@ protected void executeAction(PropertyContext propertyContext, Action action, Map
                 } catch (IllegalArgumentException iae) {
                     severity = Severity.INFO;
                 }
-                BulletinRepository bulletinRepository = context.getBulletinRepository();
+                BulletinRepository bulletinRepository = context. getBulletinRepository();

Review comment:
       Think this was a typo. :)

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/AbstractParameterProviderNode.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterProvider;
+import org.apache.nifi.parameter.SensitiveParameterProvider;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {
+
+    private final AtomicReference<ParameterProviderDetails> parameterProviderRef;
+    private final ControllerServiceLookup serviceLookup;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private final Set<ParameterContext> referencingParameterContexts;
+
+    private final Map<ParameterDescriptor, Parameter> fetchedParameters = new LinkedHashMap<>();
+
+    private volatile String comment;
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent,
+                                         final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
+
+        this(parameterProvider, id, controllerServiceProvider, validationContextFactory,
+                parameterProvider.getComponent().getClass().getSimpleName(), parameterProvider.getComponent().getClass().getCanonicalName(),
+                variableRegistry, reloadComponent, extensionManager, validationTrigger, false);
+    }
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
+                                         final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                         final boolean isExtensionMissing) {
+
+        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
+        this.parameterProviderRef = new AtomicReference<>(new ParameterProviderDetails(parameterProvider));
+        this.serviceLookup = controllerServiceProvider;
+        this.referencingParameterContexts = new HashSet<>();
+    }
+
+    @Override
+    public ConfigurableComponent getComponent() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public BundleCoordinate getBundleCoordinate() {
+        return parameterProviderRef.get().getBundleCoordinate();
+    }
+
+    @Override
+    public TerminationAwareLogger getLogger() {
+        return parameterProviderRef.get().getComponentLog();
+    }
+
+    @Override
+    public ParameterProvider getParameterProvider() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public void setParameterProvider(final LoggableComponent<ParameterProvider> parameterProvider) {
+        this.parameterProviderRef.set(new ParameterProviderDetails(parameterProvider));
+    }
+
+    @Override
+    public void reload(final Set<URL> additionalUrls) throws ParameterProviderInstantiationException {
+        String additionalResourcesFingerprint = ClassLoaderUtils.generateAdditionalUrlsFingerprint(additionalUrls);
+        setAdditionalResourcesFingerprint(additionalResourcesFingerprint);
+        getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+    }
+
+    @Override
+    public boolean isValidationNecessary() {
+        return getValidationStatus() != ValidationStatus.VALID;
+    }
+
+    @Override
+    public ConfigurationContext getConfigurationContext() {
+        return new StandardConfigurationContext(this, serviceLookup, null, getVariableRegistry());
+    }
+
+    @Override
+    public void verifyModifiable() throws IllegalStateException {
+
+    }
+
+    @Override
+    public String getComments() {
+        return comment;
+    }
+
+    @Override
+    public void setComments(final String comment) {
+        this.comment = CharacterFilterUtils.filterInvalidXmlCharacters(comment);
+    }
+
+    @Override
+    public void verifyCanClearState() {
+
+    }
+
+    @Override
+    public boolean isSensitiveParameterProvider() {
+        return parameterProviderRef.get().getParameterProvider() instanceof SensitiveParameterProvider;
+    }
+
+    @Override
+    public String toString() {
+        return "ParameterProvider[id=" + getIdentifier() + "]";
+    }
+
+    @Override
+    public String getProcessGroupIdentifier() {
+        return null;
+    }
+
+    @Override
+    public ParameterLookup getParameterLookup() {
+        return ParameterLookup.EMPTY;
+    }
+    @Override
+    public Set<ParameterContext> getReferences() {
+        readLock.lock();
+        try {
+            return Collections.unmodifiableSet(referencingParameterContexts);
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    @Override
+    public void addReference(final ParameterContext parameterContext) {
+        writeLock.lock();
+        try {
+            referencingParameterContexts.add(parameterContext);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void removeReference(final ParameterContext parameterContext) {
+        writeLock.lock();
+        try {
+            referencingParameterContexts.remove(parameterContext);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void verifyCanFetchParameters() {
+        final ValidationStatus validationStatus = performValidation();
+        if (validationStatus != ValidationStatus.VALID) {
+            throw new IllegalStateException(String.format("Parameter Provider [%s] cannot fetch parameters while validation state is %s",
+                    getIdentifier(), validationStatus));
+        }
+    }
+
+    @Override
+    public void fetchParameters() {
+        final ParameterProvider parameterProvider = parameterProviderRef.get().getParameterProvider();
+        final ConfigurationContext configurationContext = getConfigurationContext();
+        Map<ParameterDescriptor, Parameter> fetchedParameters;
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), parameterProvider.getClass(), parameterProvider.getIdentifier())) {
+            fetchedParameters = parameterProvider.fetchParameters(configurationContext);
+        }
+
+        for(final Map.Entry<ParameterDescriptor, Parameter> entry : fetchedParameters.entrySet()) {
+            final ParameterDescriptor descriptor = entry.getKey();
+            if (descriptor.isSensitive() != isSensitiveParameterProvider()) {
+                throw new IllegalStateException(String.format("Fetched parameter [%s] does not match the sensitivity of Parameter Provider [%s]",
+                        descriptor.getName(), configurationContext.getName()));
+            }
+        }
+
+        writeLock.lock();
+        try {
+            this.fetchedParameters.clear();
+            this.fetchedParameters.putAll(toProvidedParameters(fetchedParameters));
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void verifyCanApplyParameters(final Set<String> parameterNames) {
+        if (fetchedParameters.isEmpty()) {
+            throw new IllegalStateException("No parameters have been fetched from Parameter Provider " + getName());

Review comment:
       Not sure this needs to be an Exceptional case. Seems like a valid case to me.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/AbstractParameterProviderNode.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterProvider;
+import org.apache.nifi.parameter.SensitiveParameterProvider;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public abstract class AbstractParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {
+
+    private final AtomicReference<ParameterProviderDetails> parameterProviderRef;
+    private final ControllerServiceLookup serviceLookup;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private final Set<ParameterContext> referencingParameterContexts;
+
+    private final Map<ParameterDescriptor, Parameter> fetchedParameters = new LinkedHashMap<>();
+
+    private volatile String comment;
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent,
+                                         final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
+
+        this(parameterProvider, id, controllerServiceProvider, validationContextFactory,
+                parameterProvider.getComponent().getClass().getSimpleName(), parameterProvider.getComponent().getClass().getCanonicalName(),
+                variableRegistry, reloadComponent, extensionManager, validationTrigger, false);
+    }
+
+    public AbstractParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id,
+                                         final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
+                                         final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
+                                         final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                         final boolean isExtensionMissing) {
+
+        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
+        this.parameterProviderRef = new AtomicReference<>(new ParameterProviderDetails(parameterProvider));
+        this.serviceLookup = controllerServiceProvider;
+        this.referencingParameterContexts = new HashSet<>();
+    }
+
+    @Override
+    public ConfigurableComponent getComponent() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public BundleCoordinate getBundleCoordinate() {
+        return parameterProviderRef.get().getBundleCoordinate();
+    }
+
+    @Override
+    public TerminationAwareLogger getLogger() {
+        return parameterProviderRef.get().getComponentLog();
+    }
+
+    @Override
+    public ParameterProvider getParameterProvider() {
+        return parameterProviderRef.get().getParameterProvider();
+    }
+
+    @Override
+    public void setParameterProvider(final LoggableComponent<ParameterProvider> parameterProvider) {
+        this.parameterProviderRef.set(new ParameterProviderDetails(parameterProvider));
+    }
+
+    @Override
+    public void reload(final Set<URL> additionalUrls) throws ParameterProviderInstantiationException {
+        String additionalResourcesFingerprint = ClassLoaderUtils.generateAdditionalUrlsFingerprint(additionalUrls);
+        setAdditionalResourcesFingerprint(additionalResourcesFingerprint);
+        getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+    }
+
+    @Override
+    public boolean isValidationNecessary() {

Review comment:
       The intent of this method is to indicate whether or not the component should be validated based on its state - enabled/disabled/started/stopped/etc. As this is written, it's currently saying that once it becomes valid, it can't become invalid. If this component doesn't have a notion of enabled/disabled, it should always return `true`, I think.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/parameter/ParameterProviderProvider.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.controller.parameter;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.controller.ParameterProviderNode;
+import org.apache.nifi.nar.ExtensionManager;
+
+import java.util.Set;
+
+/**
+ * A ParameterProviderProvider is responsible for providing management of, and
+ * access to, ParameterProviders.
+ */
+public interface ParameterProviderProvider {

Review comment:
       I don't think we need this interface. I'm guessing this was intended to mimic the ControllerServiceProvider. At this point, however, I would say that all of these methods - create/get/remove parameter provider - belong in `FlowManager`. The only place this interface is accessed, it appears, is the `StandardParameterProviderDAO`, and that has access already to the `FlowController`, which means it has access already to the `FlowManager`. Would opt for just deleting this interface.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterProviderInitializationContext.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.controller.kerberos.KerberosConfig;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.File;
+import java.util.Set;
+
+public class StandardParameterProviderInitializationContext implements ParameterProviderInitializationContext, ControllerServiceLookup {

Review comment:
       Should not implement `ControllerServiceLookup`. I don't believe there is any need for a `ControllerServiceLookup` at all for Parameter Providers. But if there is, the context should just have a `getControllerServiceLookup()` method instead of implementing `ControllerServiceLookup`. Then, it can just simply provide `ControllerServiceProvider`. I.e., prefer composition over inheritance.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
##########
@@ -2023,6 +2034,27 @@ public void removeReportingTask(final ReportingTaskNode reportingTaskNode) {
         flowManager.removeReportingTask(reportingTaskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProviderNode(final String identifier) {
+        return flowManager.getParameterProvider(identifier);
+    }
+
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded)
+            throws ParameterProviderInstantiationException {
+        return flowManager.createParameterProvider(type, id, bundleCoordinate, firstTimeAdded);
+    }
+
+    @Override
+    public Set<ParameterProviderNode> getAllParameterProviders() {
+        return flowManager.getAllParameterProviders();
+    }
+
+    @Override
+    public void removeParameterProvider(final ParameterProviderNode parameterProviderNode) {
+        flowManager.removeParameterProvider(parameterProviderNode);
+    }
+

Review comment:
       I think it makes more sense to return a ParameterProviderProvider here rather than implement the interface. But again, I don't think we need that interface at all - just have those methods added directly to FlowManager.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterProviderInitializationContext.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.controller.kerberos.KerberosConfig;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.File;
+import java.util.Set;
+
+public class StandardParameterProviderInitializationContext implements ParameterProviderInitializationContext, ControllerServiceLookup {
+
+    private final String id;
+    private final String name;
+    private final ControllerServiceProvider serviceProvider;
+    private final ComponentLog logger;
+    private final KerberosConfig kerberosConfig;
+    private final NodeTypeProvider nodeTypeProvider;
+
+    public StandardParameterProviderInitializationContext(final String id, final String name, final ComponentLog logger,
+                                                          final ControllerServiceProvider serviceProvider, final KerberosConfig kerberosConfig,
+                                                  final NodeTypeProvider nodeTypeProvider) {
+        this.id = id;
+        this.name = name;
+        this.serviceProvider = serviceProvider;
+        this.logger = logger;
+        this.kerberosConfig = kerberosConfig;
+        this.nodeTypeProvider = nodeTypeProvider;
+    }
+
+    @Override
+    public String getIdentifier() {
+        return id;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) {
+        return serviceProvider.getControllerServiceIdentifiers(serviceType, null);
+    }
+
+    @Override
+    public ControllerService getControllerService(final String identifier) {
+        return serviceProvider.getControllerService(identifier);
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final ControllerService service) {
+        return serviceProvider.isControllerServiceEnabled(service);
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+        return serviceProvider.isControllerServiceEnabled(serviceIdentifier);
+    }
+
+    @Override
+    public boolean isControllerServiceEnabling(final String serviceIdentifier) {
+        return serviceProvider.isControllerServiceEnabling(serviceIdentifier);
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return this;
+    }
+
+    @Override
+    public String getControllerServiceName(final String serviceIdentifier) {
+        return serviceProvider.getControllerServiceName(serviceIdentifier);
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return logger;
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return kerberosConfig.getPrincipal();

Review comment:
       I think it is valid for `KerberosConfig` to be null, so we should return `null` for the principal/keytable/configfile in that case.

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
##########
@@ -296,6 +298,53 @@ public ReportingTaskNode createReportingTask(final String type, final String id,
         return taskNode;
     }
 
+    @Override
+    public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded,

Review comment:
       Then again, this might be able to take the place of ParamterValueProvider in stateless. So that way we have feature parity between the two. Need to understand better exactly how that would work....

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/ParameterProvider.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation.lifecycle.OnConfigurationRestored;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.util.Map;
+
+/**
+ * Defines a provider that is responsible for fetching from an external source Parameters with
+ * which a ParameterContext can be populated.
+ *
+ * <p>
+ * <code>ParameterProvider</code>s are discovered using Java's
+ * <code>ServiceLoader</code> mechanism. As a result, all implementations must
+ * follow these rules:
+ *
+ * <ul>
+ * <li>The implementation must implement this interface.</li>
+ * <li>The implementation must have a file named
+ * org.apache.nifi.parameter.ParameterProvider located within the jar's
+ * <code>META-INF/services</code> directory. This file contains a list of
+ * fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
+ * one-per-line.
+ * <li>The implementation must support a default constructor.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * All implementations of this interface must be thread-safe.
+ * </p>
+ *
+ * <p>
+ * Parameter Providers may choose to annotate a method with the
+ * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
+ * will be invoked after all properties have been set for the ParameterProvider and
+ * before its parameters are fetched.
+ * </p>
+ */
+public interface ParameterProvider extends ConfigurableComponent {
+
+    /**
+     * Provides the Parameter Provider with access to objects that may be of use
+     * throughout the life of the service
+     *
+     * @param config of initialization context
+     * @throws org.apache.nifi.reporting.InitializationException if unable to init
+     */
+    void initialize(ParameterProviderInitializationContext config) throws InitializationException;
+
+    /**
+     * Fetches parameters from an external source.
+     * @param context The <code>ConfigurationContext</code>for the provider
+     * @return An unmodifiable map of available parameters, mapped from descriptor to Parameter
+     */
+    Map<ParameterDescriptor, Parameter> fetchParameters(ConfigurationContext context);

Review comment:
       Given that the `Parameter` has a reference to the `ParameterDescriptor` does it make more sense to have this return a `List<Parameter>` or a `Set<Parameter>`?

##########
File path: nifi-api/src/main/java/org/apache/nifi/parameter/SensitiveParameterProvider.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+/**
+ * A base interface for all sensitive <code>ParameterProvider</code>s
+ */
+public interface SensitiveParameterProvider extends ParameterProvider {

Review comment:
       I'm not a fan of marker interfaces generally. Makes more sense to me, if we need to differentiate the two, to instead use a method like:
   ```
   boolean isParameterSensitive();
   ```
   Or, better yet:
   ```
   ParameterSensitivity getParameterSensitivity();
   ```
   where `ParameterSensitivity` is an enum with two values: `SENSITIVE`, `NON_SENSITIVE` and possibly a value of `BOTH`

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/pom.xml
##########
@@ -0,0 +1,55 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-standard-bundle</artifactId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-standard-parameter-providers</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>

Review comment:
       This isn't a dependency that extension points should have. I don't think it's needed at all here, though. Think it's just an extraneous dependency that can be removed.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AbstractParameterResource.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.web.api;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.util.LifecycleManagementException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.HttpMethod;
+import java.net.URI;
+import java.util.Map;
+
+public abstract class AbstractParameterResource extends ApplicationResource {

Review comment:
       Not sure just this one simple method warrants introducing a new abstract class. Especially considering that it's only called by way of the `ParameterUpdateManager` - perhaps it makes more sense to just implement this logic in the `ParameterUpdateManager`?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ParameterDTO.java
##########
@@ -30,8 +30,10 @@
     private Boolean sensitive;
     private String value;
     private Boolean valueRemoved;
+    private boolean provided;

Review comment:
       Generally we want to use Boolean objects instead of primitives in DTOs as a way to differentiate between the value being unset and being false.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ParameterProviderNode.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.controller;
+
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterProvider;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public interface ParameterProviderNode extends ComponentNode {
+
+    ParameterProvider getParameterProvider();
+
+    void setParameterProvider(LoggableComponent<ParameterProvider> parameterProvider);
+
+    ConfigurationContext getConfigurationContext();
+
+    String getComments();
+
+    void setComments(String comment);
+
+    void verifyCanFetchParameters();
+
+    void fetchParameters();
+
+    void verifyCanApplyParameters(Set<String> parameterNames);
+
+    Set<String> getFetchedParameterNames();
+
+    Map<ParameterContext, Map<String, Parameter>> getFetchedParametersToApply(Set<String> parameterNames);

Review comment:
       Not sure yet, but might be worth considering creating a `ParametersApplication` class or something instead of a `Map<ParameterContext, Map<String, Parameter>>`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
##########
@@ -405,6 +421,66 @@ public void onReportingTaskAdded(final ReportingTaskNode taskNode) {
         allReportingTasks.put(taskNode.getIdentifier(), taskNode);
     }
 
+    @Override
+    public ParameterProviderNode getParameterProvider(final String id) {
+        if (id == null) {
+            return null;
+        }
+        return allParameterProviders.get(id);
+    }
+
+    @Override
+    public void removeParameterProvider(final ParameterProviderNode parameterProvider) {
+        final ParameterProviderNode existing = allParameterProviders.get(parameterProvider.getIdentifier());
+        if (existing == null || existing != parameterProvider) {
+            throw new IllegalStateException("Parameter Provider " + parameterProvider + " does not exist in this Flow");
+        }
+
+        final Class<?> taskClass = parameterProvider.getParameterProvider().getClass();
+        try (final NarCloseable x = NarCloseable.withComponentNarLoader(getExtensionManager(), taskClass, parameterProvider.getParameterProvider().getIdentifier())) {
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, parameterProvider.getParameterProvider(), parameterProvider.getConfigurationContext());
+        }
+
+        for (final Map.Entry<PropertyDescriptor, String> entry : parameterProvider.getEffectivePropertyValues().entrySet()) {
+            final PropertyDescriptor descriptor = entry.getKey();
+            if (descriptor.getControllerServiceDefinition() != null) {
+                final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
+                if (value != null) {
+                    final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value);
+                    if (serviceNode != null) {
+                        serviceNode.removeReference(parameterProvider, descriptor);
+                    }
+                }
+            }
+        }
+        for(final ParameterContext parameterContext : parameterProvider.getReferences()) {
+            parameterContext.getSensitiveParameterProvider().ifPresent(sensitiveProvider -> {
+                if (parameterProvider.getIdentifier().equals(sensitiveProvider.getIdentifier())) {
+                    parameterContext.setSensitiveParameterProvider(null);
+                }
+            });
+            parameterContext.getNonSensitiveParameterProvider().ifPresent(nonSensitiveProvider -> {
+                if (parameterProvider.getIdentifier().equals(nonSensitiveProvider.getIdentifier())) {
+                    parameterContext.setNonSensitiveParameterProvider(null);
+                }
+            });
+        }
+
+        allParameterProviders.remove(parameterProvider.getIdentifier());
+        LogRepositoryFactory.removeRepository(parameterProvider.getIdentifier());
+
+        getExtensionManager().removeInstanceClassLoader(parameterProvider.getIdentifier());
+    }
+
+    @Override
+    public Set<ParameterProviderNode> getAllParameterProviders() {
+        return Collections.unmodifiableSet(new HashSet<>(allParameterProviders.values()));

Review comment:
       Given that this is already making a defensive copy by using `new HashSet<>()`, no need to wrap in an `unmodifiableSet`, is there?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on pull request #5369: NIFI-9003: Adding ParameterProviders to NiFi

Posted by GitBox <gi...@apache.org>.
gresockj commented on pull request #5369:
URL: https://github.com/apache/nifi/pull/5369#issuecomment-931306105


   > @gresockj this is great! This is a huge lift, to add a completely new type of extension point! I've not done any testing at all yet. But I've spent a good bit of time reviewing all of the code. Looks good for the most part - and you found _A LOT_ of minor typos in the javadocs - thanks for fixing those! :)
   > 
   > Wanted to go ahead and submit my review, before jumping in to try it out, just because I feel like some of my thoughts may end up warranting some discussions. Especially around the notion of SensitiveParameterProvider vs NonSensitiveParameterProvider and those really becoming a single interface. I think this would yield a cleaner design. And we need to be really careful that we nail this down really well the first time because once we introduce something into `nifi-api` we can't really break backward compatibility until a 2.0 release comes, so let's take the time to make sure we're both in agreement on what makes the most sense there.
   > 
   > Thanks again!
   
   Thanks for a thorough review, @markap14, this is very helpful!  I believe I've addressed everything in the latest commit, except for collapsing Sensitive/NonSensitive ParameterProviders into one interface.  I agree with your assessment, and will work on this in a separate commit since it will be a non-trivial change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org