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/06/01 17:15:37 UTC

[GitHub] [nifi] markap14 opened a new pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

markap14 opened a new pull request #5113:
URL: https://github.com/apache/nifi/pull/5113


   …ored stateless to use this when creating a dataflow so that Parameter Provider implementations can be externalized into NARs. Also updated ExtensionDiscoveringManager such that callers are able to provide a new type of class to be discovered (e.g., ParameterProvider) so that the extensions will be automatically discovered.
   
   <!--
     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.
   -->
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] 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.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] 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:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] 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:
   - [ ] 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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardParameterProviderInitializationContext.java
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.stateless.engine;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.service.StandardPropertyContext;
+import org.apache.nifi.stateless.parameter.ParameterProvider;
+import org.apache.nifi.stateless.parameter.ParameterProviderInitializationContext;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+public class StandardParameterProviderInitializationContext extends StandardPropertyContext implements ParameterProviderInitializationContext {
+    private final String identifier;
+
+    public StandardParameterProviderInitializationContext(final ParameterProvider parameterProvider, final Map<String, String> propertyValues, final String identifier) {
+        super(createPropertyMap(parameterProvider, propertyValues), parameterProvider);
+        this.identifier = identifier;
+    }
+
+    private static Map<PropertyDescriptor, String> createPropertyMap(final ParameterProvider provider, final Map<String, String> propertyValues) {
+        final Map<PropertyDescriptor, String> propertyMap = new LinkedHashMap<>();
+        for (final Map.Entry<String, String> entry : propertyValues.entrySet()) {
+            final PropertyDescriptor descriptor = provider.getPropertyDescriptor(entry.getKey());

Review comment:
       Whoops! It looks like we're not validating Reporting Tasks at the moment in stateless either! Will update the engine to ensure that they are valid. If not, I think we need to throw an IllegalStateException.




-- 
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.

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/parameters/ParameterContextIT.java
##########
@@ -47,6 +50,174 @@
 
 public class ParameterContextIT extends StatelessSystemIT {
 
+    @Test
+    public void testCustomParameterProvider() throws IOException, StatelessConfigurationException, InterruptedException {
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedPort outPort = flowBuilder.createOutputPort("Out");
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile");
+
+        generate.setProperties(Collections.singletonMap("Batch Size", "#{three}"));
+        flowBuilder.createConnection(generate, outPort, "success");
+
+        final VersionedFlowSnapshot flowSnapshot = flowBuilder.getFlowSnapshot();
+
+        // Define the Parameter Context to use
+        final ParameterProviderDefinition numericParameterProvider = new ParameterProviderDefinition();
+        numericParameterProvider.setName("Numeric Parameter Provider");
+        numericParameterProvider.setType("org.apache.nifi.stateless.parameters.NumericParameterProvider");
+        final List<ParameterProviderDefinition> parameterProviders = Collections.singletonList(numericParameterProvider);
+
+        // Create a Parameter Context & set it on the root group.
+        final VersionedParameterContext parameterContext = flowBuilder.createParameterContext("Context 1");
+        parameterContext.getParameters().add(createVersionedParameter("three", "-1"));  // Set value to -1. This should be overridden by the Numeric Parameter Context.
+        flowBuilder.getRootGroup().setParameterContextName("Context 1");
+
+        // Startup the dataflow
+        final StatelessDataflow dataflow = loadDataflow(flowSnapshot, Collections.emptyList(), parameterProviders, Collections.emptySet(), TransactionThresholds.SINGLE_FLOWFILE);
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final TriggerResult result = trigger.getResult();
+        final List<FlowFile> outputFlowFiles = result.getOutputFlowFiles().get("Out");
+        assertEquals(3, outputFlowFiles.size());
+        result.acknowledge();
+    }
+
+
+    @Test
+    public void testInvalidParameterProvider() throws IOException, StatelessConfigurationException {
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedPort outPort = flowBuilder.createOutputPort("Out");
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile");
+
+        generate.setProperties(Collections.singletonMap("Batch Size", "#{three}"));
+        flowBuilder.createConnection(generate, outPort, "success");
+
+        final VersionedFlowSnapshot flowSnapshot = flowBuilder.getFlowSnapshot();
+
+        // Define the Parameter Context to use
+        final ParameterProviderDefinition numericParameterProvider = new ParameterProviderDefinition();
+        numericParameterProvider.setName("Invalid Parameter Provider");
+        numericParameterProvider.setType("org.apache.nifi.stateless.parameters.InvalidParameterProvider");
+        final List<ParameterProviderDefinition> parameterProviders = Collections.singletonList(numericParameterProvider);
+
+        // Create a Parameter Context & set it on the root group.
+        final VersionedParameterContext parameterContext = flowBuilder.createParameterContext("Context 1");
+        parameterContext.getParameters().add(createVersionedParameter("three", "-1"));  // Set value to -1. This should be overridden by the Numeric Parameter Context.
+        flowBuilder.getRootGroup().setParameterContextName("Context 1");
+
+        try {
+            loadDataflow(flowSnapshot, Collections.emptyList(), parameterProviders, Collections.emptySet(), TransactionThresholds.SINGLE_FLOWFILE);
+            Assert.fail("Expected to fail on startup because parameter provider is not valid");
+        } catch (final IllegalStateException expected) {
+        }

Review comment:
       Is `loadDataflow()` expected to throw an IllegalStateException here?  If so, could `assertThrows()` be used instead of the try-catch and `Assert.fail()`?




-- 
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 #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.stateless.parameter.ParameterProvider
##########
@@ -0,0 +1,18 @@
+# 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.
+
+org.apache.nifi.stateless.parameters.InvalidParameterProvider
+org.apache.nifi.stateless.parameters.NumericParameterProvider
+org.apache.nifi.stateless.parameters.ParameterProviderWithProperties

Review comment:
       Yeah, so this is intended to be used only for tests. But if it were put into the src/test, it wouldn't get packaged into the jar. They are for testing but not for testing in this module.




-- 
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.

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



[GitHub] [nifi] gresockj commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-assembly/README.md
##########
@@ -427,33 +431,54 @@ Additionally, there may be sensitive parameters that users prefer not to include
 Environment Variables, for example.
 
 These parameters may be passed when running NiFi via the `bin/nifi.sh` script by passing a `-p` argument.
-When used, the `-p` argument must be followed by an argument in the format `<context name>:<parameter name>:<parameter value>`
+When used, the `-p` argument must be followed by an argument in the format `[<context name>:]<parameter name>=<parameter value>`
 For example:
 
 ```
-bin/nifi.sh stateless -c -p "Kafka Parameter Context:Kafka Topic:Sensor Data" /var/lib/nifi/stateless/config/stateless.properties /var/lib/nifi/stateless/flows/jms-to-kafka.properties
+bin/nifi.sh stateless -c -p "Kafka Parameter Context:Kafka Topic=Sensor Data" /var/lib/nifi/stateless/config/stateless.properties /var/lib/nifi/stateless/flows/jms-to-kafka.properties
 ```
 
 Note that because of the spaces in the Parameter/Context name and the Parameter value, the argument is quoted.
 Multiple Parameters may be passed using this syntax:
 
 ```
-bin/nifi.sh stateless -c -p "Kafka Parameter Context:Kafka Brokers:kafka-01:9092,kafka-02:9092,kafka-03:9092" -p "Kafka Parameter Context:Kafka Topic:Sensor Data" /var/lib/nifi/stateless/config /stateless.properties
+bin/nifi.sh stateless -c -p "Kafka Parameter Context:Kafka Brokers=kafka-01:9092,kafka-02:9092,kafka-03:9092" -p "Kafka Parameter Context:Kafka Topic=Sensor Data" /var/lib/nifi/stateless/config /stateless.properties
 ```
 
-Note also that the Parameter Context Name and the Parameter Name may not include a colon character.
-The Parameter Value can include colon characters, as in the example here.
+If the name of the Parameter Context contains a colon, it must be escaped using a backslash.
+The name of the Parameter Context and the name of the Parameter may not include an equals sign (=).
+The Parameter Value can include colon characters, as well as equals, as in the example here.
 
 Often times, though, the Parameter Context name is not particularly important, and we just want to provide a Parameter name.
 This can be done by simply leaving off the name of the Parameter Context. For example:
 
 ```
-bin/nifi.sh stateless -c -p "Kafka Brokers:kafka-01:9092,kafka-02:9092,kafka-03:9092" -p "Kafka Topic:Sensor Data" /var/lib/nifi/stateless/config /stateless.properties
+bin/nifi.sh stateless -c -p "Kafka Brokers=kafka-01:9092,kafka-02:9092,kafka-03:9092" -p "Kafka Topic=Sensor Data" /var/lib/nifi/stateless/config /stateless.properties
 ```
 
 In this case, any Parameter Context that has a name of "Kafka Brokers" will have the parameter resolved to `kafka-01:9092,kafka-02:9092,kafka-03:9092`, regardless of the name
 of the Parameter Context.
 
 If a given Parameter is referenced and is not defined using the `-p` syntax, an environment variable may also be used to provide the value. However, environment variables typically are
 allowed to contain only letters, numbers, and underscores in their names. As a result, it is important that the Parameters' names also adhere to that same rule, or the environment variable
-will not be addressable. 
\ No newline at end of file
+will not be addressable.
+
+At times, none of the builtin capabilities for resolving Parameters are ideal, though. In these situations, we can use a custom Parameter Provider in order to source Parameter values from elsewhere.

Review comment:
       I'd add a dash to "built-in".




-- 
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.

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



[GitHub] [nifi] gresockj commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/stateless/parameters/ParameterProviderWithProperties.java
##########
@@ -0,0 +1,63 @@
+/*

Review comment:
       Did you mean to put this in src/main instead of src/test?




-- 
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-assembly/README.md
##########
@@ -427,28 +427,29 @@ Additionally, there may be sensitive parameters that users prefer not to include
 Environment Variables, for example.
 

Review comment:
       Whoops! Totally forgot that! Great catch!




-- 
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/config/PropertiesFileFlowDefinitionParser.java
##########
@@ -174,6 +180,80 @@
         return new ArrayList<>(reportingTaskDefinitions.values());
     }
 
+    private List<ParameterProviderDefinition> getParameterProviders(final Map<String, String> properties, final List<ParameterOverride> parameterOverrides) {
+        final Map<String, ParameterProviderDefinition> parameterProviderDefinitions = new LinkedHashMap<>();
+
+        for (final String propertyName : properties.keySet()) {
+            final Matcher matcher = PARAMETER_PROVIDER_PATTERN.matcher(propertyName);
+            if (!matcher.matches()) {
+                continue;
+            }
+
+            // For a property name like:
+            // nifi.stateless.parameter.provider.abc.name=hello
+            // We consider 'abc' the <parameter provider key> and 'name' the <relative property name>
+            final String parameterProviderKey = matcher.group(1);
+            final ParameterProviderDefinition definition = parameterProviderDefinitions.computeIfAbsent(parameterProviderKey, key -> new ParameterProviderDefinition());
+            final String relativePropertyName = matcher.group(2);
+            final String propertyValue = properties.get(propertyName);
+
+            if (relativePropertyName.startsWith("properties.")) {
+                if (relativePropertyName.length() < 12) {
+                    logger.warn("Encountered unexpected property <" + propertyName + "> in flow definition. This property will be ignored.");
+                    continue;
+                }
+
+                final String providerPropertyName = relativePropertyName.substring(11);
+                definition.getPropertyValues().put(providerPropertyName, propertyValue);

Review comment:
       Thanks. Made it really easy to replicate. I also caught a couple of other minor tweaks that should be added to clarify when things aren't as expected. Will update these.




-- 
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.

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



[GitHub] [nifi] gresockj commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.stateless.parameter.ParameterProvider
##########
@@ -0,0 +1,18 @@
+# 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.
+
+org.apache.nifi.stateless.parameters.InvalidParameterProvider
+org.apache.nifi.stateless.parameters.NumericParameterProvider
+org.apache.nifi.stateless.parameters.ParameterProviderWithProperties

Review comment:
       Got it, forgot this was inside the nifi-system-test-extensions module already.




-- 
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/config/PropertiesFileFlowDefinitionParser.java
##########
@@ -174,6 +180,80 @@
         return new ArrayList<>(reportingTaskDefinitions.values());
     }
 
+    private List<ParameterProviderDefinition> getParameterProviders(final Map<String, String> properties, final List<ParameterOverride> parameterOverrides) {
+        final Map<String, ParameterProviderDefinition> parameterProviderDefinitions = new LinkedHashMap<>();
+
+        for (final String propertyName : properties.keySet()) {
+            final Matcher matcher = PARAMETER_PROVIDER_PATTERN.matcher(propertyName);
+            if (!matcher.matches()) {
+                continue;
+            }
+
+            // For a property name like:
+            // nifi.stateless.parameter.provider.abc.name=hello
+            // We consider 'abc' the <parameter provider key> and 'name' the <relative property name>
+            final String parameterProviderKey = matcher.group(1);
+            final ParameterProviderDefinition definition = parameterProviderDefinitions.computeIfAbsent(parameterProviderKey, key -> new ParameterProviderDefinition());
+            final String relativePropertyName = matcher.group(2);
+            final String propertyValue = properties.get(propertyName);
+
+            if (relativePropertyName.startsWith("properties.")) {
+                if (relativePropertyName.length() < 12) {
+                    logger.warn("Encountered unexpected property <" + propertyName + "> in flow definition. This property will be ignored.");

Review comment:
       Well, I don't think we know at this point whether or not it's valid. At this point, we are only parsing the definition. Validity would really depend on the implementation of the Parameter Provider.




-- 
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.

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



[GitHub] [nifi] markap14 commented on pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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


   Thanks for the thorough review @gresockj! I addressed the concerns and a few other minor tweaks that i found when updating the code. I also added a couple of system tests to verify that the behavior is as expected.


-- 
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
##########
@@ -182,6 +197,98 @@ public StatelessDataflow createFlow(final DataflowDefinition<VersionedFlowSnapsh
         return dataflow;
     }
 
+    private ParameterProvider createParameterProvider(final DataflowDefinition<?> dataflowDefinition) {
+        // Create a Provider for each definition
+        final List<ParameterProvider> providers = new ArrayList<>();
+        for (final ParameterProviderDefinition definition : dataflowDefinition.getParameterProviderDefinitions()) {
+            providers.add(createParameterProvider(definition));
+        }
+
+        // Create a Composite Parameter Provider that wraps all of the others.
+        final CompositeParameterProvider provider = new CompositeParameterProvider(providers);
+        final ParameterProviderInitializationContext initializationContext = new StandardParameterProviderInitializationContext(provider, Collections.emptyMap(), UUID.randomUUID().toString());
+        provider.initialize(initializationContext);
+        return provider;
+    }
+
+    private ParameterProvider createParameterProvider(final ParameterProviderDefinition definition) {
+        final BundleCoordinate bundleCoordinate = determineBundleCoordinate(definition, "Parameter Provider");
+        final Bundle bundle = extensionManager.getBundle(bundleCoordinate);
+        if (bundle == null) {
+            throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
+        }
+
+        final String providerType = definition.getType();
+
+        final String providerId = UUID.randomUUID().toString();
+        final InstanceClassLoader classLoader = extensionManager.createInstanceClassLoader(providerType, providerId, bundle, Collections.emptySet());
+
+        try {
+            final Class<?> rawClass = Class.forName(providerType, true, classLoader);
+            Thread.currentThread().setContextClassLoader(classLoader);
+
+            final ParameterProvider parameterProvider = (ParameterProvider) rawClass.newInstance();
+
+            // Initialize the provider
+            final Map<String, String> properties = resolveProperties(definition.getPropertyValues(), parameterProvider, parameterProvider.getPropertyDescriptors());
+            final ParameterProviderInitializationContext initializationContext = new StandardParameterProviderInitializationContext(parameterProvider, properties, providerId);
+            parameterProvider.initialize(initializationContext);
+
+            // Ensure that the Parameter Provider is valid.
+            final List<ValidationResult> validationResults = validate(parameterProvider, properties, providerId);
+            if (!validationResults.isEmpty()) {
+                throw new IllegalStateException("Parameter Provider with name <" + definition.getName() + "> is not valid: " + validationResults);
+            }
+
+            return parameterProvider;
+        } catch (final Exception e) {
+            throw new IllegalStateException("Could not create Parameter Provider " + definition.getName() + " of type " + definition.getType(), e);
+        }
+    }
+
+    private List<ValidationResult> validate(final ConfigurableComponent component, final Map<String, String> properties, final String componentId) {
+        final Map<PropertyDescriptor, PropertyConfiguration> explicitlyConfiguredPropertyMap = new HashMap<>();
+
+        for (final Map.Entry<String, String> property : properties.entrySet()) {
+            final String propertyName = property.getKey();
+            final String propertyValue = property.getValue();
+
+            final PropertyDescriptor descriptor = component.getPropertyDescriptor(propertyName);
+            final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList());

Review comment:
       Ah yes, now that I have fixed the validation, I need that provider to actually be valid :( Will fix.
   




-- 
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.

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



[GitHub] [nifi] markap14 commented on pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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


   Agreed @gresockj the command line override should take precedence. Followed by environment variables, and then anything provided by a ParameterProvider. Updated the PR to use this precedence. Thanks again!


-- 
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 #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/stateless/parameters/ParameterProviderWithProperties.java
##########
@@ -0,0 +1,63 @@
+/*

Review comment:
       Did you mean to put this in src/main instead of src/test?

##########
File path: nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.stateless.parameter.ParameterProvider
##########
@@ -0,0 +1,18 @@
+# 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.
+
+org.apache.nifi.stateless.parameters.InvalidParameterProvider
+org.apache.nifi.stateless.parameters.NumericParameterProvider
+org.apache.nifi.stateless.parameters.ParameterProviderWithProperties

Review comment:
       Unless I'm missing something, isn't this just a test provider?

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
##########
@@ -182,6 +197,98 @@ public StatelessDataflow createFlow(final DataflowDefinition<VersionedFlowSnapsh
         return dataflow;
     }
 
+    private ParameterProvider createParameterProvider(final DataflowDefinition<?> dataflowDefinition) {
+        // Create a Provider for each definition
+        final List<ParameterProvider> providers = new ArrayList<>();
+        for (final ParameterProviderDefinition definition : dataflowDefinition.getParameterProviderDefinitions()) {
+            providers.add(createParameterProvider(definition));
+        }
+
+        // Create a Composite Parameter Provider that wraps all of the others.
+        final CompositeParameterProvider provider = new CompositeParameterProvider(providers);
+        final ParameterProviderInitializationContext initializationContext = new StandardParameterProviderInitializationContext(provider, Collections.emptyMap(), UUID.randomUUID().toString());
+        provider.initialize(initializationContext);
+        return provider;
+    }
+
+    private ParameterProvider createParameterProvider(final ParameterProviderDefinition definition) {
+        final BundleCoordinate bundleCoordinate = determineBundleCoordinate(definition, "Parameter Provider");
+        final Bundle bundle = extensionManager.getBundle(bundleCoordinate);
+        if (bundle == null) {
+            throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
+        }
+
+        final String providerType = definition.getType();
+
+        final String providerId = UUID.randomUUID().toString();
+        final InstanceClassLoader classLoader = extensionManager.createInstanceClassLoader(providerType, providerId, bundle, Collections.emptySet());
+
+        try {
+            final Class<?> rawClass = Class.forName(providerType, true, classLoader);
+            Thread.currentThread().setContextClassLoader(classLoader);
+
+            final ParameterProvider parameterProvider = (ParameterProvider) rawClass.newInstance();
+
+            // Initialize the provider
+            final Map<String, String> properties = resolveProperties(definition.getPropertyValues(), parameterProvider, parameterProvider.getPropertyDescriptors());
+            final ParameterProviderInitializationContext initializationContext = new StandardParameterProviderInitializationContext(parameterProvider, properties, providerId);
+            parameterProvider.initialize(initializationContext);
+
+            // Ensure that the Parameter Provider is valid.
+            final List<ValidationResult> validationResults = validate(parameterProvider, properties, providerId);
+            if (!validationResults.isEmpty()) {
+                throw new IllegalStateException("Parameter Provider with name <" + definition.getName() + "> is not valid: " + validationResults);
+            }
+
+            return parameterProvider;
+        } catch (final Exception e) {
+            throw new IllegalStateException("Could not create Parameter Provider " + definition.getName() + " of type " + definition.getType(), e);
+        }
+    }
+
+    private List<ValidationResult> validate(final ConfigurableComponent component, final Map<String, String> properties, final String componentId) {
+        final Map<PropertyDescriptor, PropertyConfiguration> explicitlyConfiguredPropertyMap = new HashMap<>();
+
+        for (final Map.Entry<String, String> property : properties.entrySet()) {
+            final String propertyName = property.getKey();
+            final String propertyValue = property.getValue();
+
+            final PropertyDescriptor descriptor = component.getPropertyDescriptor(propertyName);
+            final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList());

Review comment:
       Nice, that appears to have fixed it.  However.. one more problem when I try to override the parameter from the command line:
   
   ```
   Exception in thread "main" java.lang.IllegalStateException: Could not create Parameter Provider Parameter Override Provider of type org.apache.nifi.stateless.parameter.ParameterOverrideProvider
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createParameterProvider(StandardStatelessEngine.java:245)
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createParameterProvider(StandardStatelessEngine.java:204)
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createFlow(StandardStatelessEngine.java:175)
   	at org.apache.nifi.stateless.flow.StandardStatelessDataflowFactory.createDataflow(StandardStatelessDataflowFactory.java:223)
   	at org.apache.nifi.stateless.bootstrap.StatelessBootstrap.createDataflow(StatelessBootstrap.java:66)
   	at org.apache.nifi.stateless.bootstrap.RunStatelessFlow.createDataflow(RunStatelessFlow.java:94)
   	at org.apache.nifi.stateless.bootstrap.RunStatelessFlow.main(RunStatelessFlow.java:56)
   Caused by: java.lang.IllegalStateException: Parameter Provider with name <Parameter Override Provider> is not valid: ['Context:param' validated against 'joe' is invalid because 'Context:param' is not a supported property or has no Validator associated with it]
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createParameterProvider(StandardStatelessEngine.java:240)
   	... 6 more
   ```
   




-- 
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/parameters/ParameterContextIT.java
##########
@@ -47,6 +50,174 @@
 
 public class ParameterContextIT extends StatelessSystemIT {
 
+    @Test
+    public void testCustomParameterProvider() throws IOException, StatelessConfigurationException, InterruptedException {
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedPort outPort = flowBuilder.createOutputPort("Out");
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile");
+
+        generate.setProperties(Collections.singletonMap("Batch Size", "#{three}"));
+        flowBuilder.createConnection(generate, outPort, "success");
+
+        final VersionedFlowSnapshot flowSnapshot = flowBuilder.getFlowSnapshot();
+
+        // Define the Parameter Context to use
+        final ParameterProviderDefinition numericParameterProvider = new ParameterProviderDefinition();
+        numericParameterProvider.setName("Numeric Parameter Provider");
+        numericParameterProvider.setType("org.apache.nifi.stateless.parameters.NumericParameterProvider");
+        final List<ParameterProviderDefinition> parameterProviders = Collections.singletonList(numericParameterProvider);
+
+        // Create a Parameter Context & set it on the root group.
+        final VersionedParameterContext parameterContext = flowBuilder.createParameterContext("Context 1");
+        parameterContext.getParameters().add(createVersionedParameter("three", "-1"));  // Set value to -1. This should be overridden by the Numeric Parameter Context.
+        flowBuilder.getRootGroup().setParameterContextName("Context 1");
+
+        // Startup the dataflow
+        final StatelessDataflow dataflow = loadDataflow(flowSnapshot, Collections.emptyList(), parameterProviders, Collections.emptySet(), TransactionThresholds.SINGLE_FLOWFILE);
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final TriggerResult result = trigger.getResult();
+        final List<FlowFile> outputFlowFiles = result.getOutputFlowFiles().get("Out");
+        assertEquals(3, outputFlowFiles.size());
+        result.acknowledge();
+    }
+
+
+    @Test
+    public void testInvalidParameterProvider() throws IOException, StatelessConfigurationException {
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedPort outPort = flowBuilder.createOutputPort("Out");
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile");
+
+        generate.setProperties(Collections.singletonMap("Batch Size", "#{three}"));
+        flowBuilder.createConnection(generate, outPort, "success");
+
+        final VersionedFlowSnapshot flowSnapshot = flowBuilder.getFlowSnapshot();
+
+        // Define the Parameter Context to use
+        final ParameterProviderDefinition numericParameterProvider = new ParameterProviderDefinition();
+        numericParameterProvider.setName("Invalid Parameter Provider");
+        numericParameterProvider.setType("org.apache.nifi.stateless.parameters.InvalidParameterProvider");
+        final List<ParameterProviderDefinition> parameterProviders = Collections.singletonList(numericParameterProvider);
+
+        // Create a Parameter Context & set it on the root group.
+        final VersionedParameterContext parameterContext = flowBuilder.createParameterContext("Context 1");
+        parameterContext.getParameters().add(createVersionedParameter("three", "-1"));  // Set value to -1. This should be overridden by the Numeric Parameter Context.
+        flowBuilder.getRootGroup().setParameterContextName("Context 1");
+
+        try {
+            loadDataflow(flowSnapshot, Collections.emptyList(), parameterProviders, Collections.emptySet(), TransactionThresholds.SINGLE_FLOWFILE);
+            Assert.fail("Expected to fail on startup because parameter provider is not valid");
+        } catch (final IllegalStateException expected) {
+        }

Review comment:
       Didn't realize that `assertThrows` had been added to JUnit 4. Will update that.




-- 
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] asfgit closed pull request #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5113:
URL: https://github.com/apache/nifi/pull/5113


   


-- 
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 #5113: NIFI-8644: Introduced a notion of ParameterProviderDefinition. Refact…

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



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
##########
@@ -182,6 +197,78 @@ public StatelessDataflow createFlow(final DataflowDefinition<VersionedFlowSnapsh
         return dataflow;
     }
 
+    private ParameterProvider createParameterProvider(final DataflowDefinition<?> dataflowDefinition) {
+        // Create a Provider for each definition
+        final List<ParameterProvider> providers = new ArrayList<>();
+        for (final ParameterProviderDefinition definition : dataflowDefinition.getParameterProviderDefinitions()) {
+            providers.add(createParameterProvider(definition));
+        }
+
+        // Create a Composite Parameter Provider that wraps all of the others.
+        final CompositeParameterProvider provider = new CompositeParameterProvider(providers);
+        final ParameterProviderInitializationContext initializationContext = new StandardParameterProviderInitializationContext(provider, Collections.emptyMap(), UUID.randomUUID().toString());
+        provider.initialize(initializationContext);
+        return provider;
+    }
+
+    private ParameterProvider createParameterProvider(final ParameterProviderDefinition definition) {
+        final BundleCoordinate bundleCoordinate = determineBundleCoordinate(definition, "Parameter Provider");
+        final Bundle bundle = extensionManager.getBundle(bundleCoordinate);
+        if (bundle == null) {
+            throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
+        }
+
+        final String providerType = definition.getType();
+
+        final String providerId = UUID.randomUUID().toString();
+        final InstanceClassLoader classLoader = extensionManager.createInstanceClassLoader(providerType, providerId, bundle, Collections.emptySet());
+
+        try {
+            final Class<?> rawClass = Class.forName(providerType, true, classLoader);
+            Thread.currentThread().setContextClassLoader(classLoader);
+
+            final ParameterProvider parameterProvider = (ParameterProvider) rawClass.newInstance();
+
+            // Initialize the provider
+            final Map<String, String> properties = resolveProperties(definition.getPropertyValues(), parameterProvider, parameterProvider.getPropertyDescriptors());
+            final ParameterProviderInitializationContext initializationContext = new StandardParameterProviderInitializationContext(parameterProvider, properties, providerId);
+            parameterProvider.initialize(initializationContext);
+
+            // Ensure that the Parameter Provider is valid.
+            final List<ValidationResult> validationResults = validate(parameterProvider, properties, providerId);
+            if (!validationResults.isEmpty()) {
+                throw new IllegalStateException("Parameter Provider with name <" + definition.getName() + "> is not valid: " + validationResults);
+            }
+
+            return parameterProvider;
+        } catch (final Exception e) {
+            throw new IllegalStateException("Could not create Parameter Provider " + definition.getName() + " of type " + definition.getType(), e);
+        }
+    }
+
+    private List<ValidationResult> validate(final ConfigurableComponent component, final Map<String, String> properties, final String componentId) {
+        final Map<PropertyDescriptor, PropertyConfiguration> propertyMap = new HashMap<>();
+
+        for (final Map.Entry<String, String> property : properties.entrySet()) {
+            final String propertyName = property.getKey();
+            final String propertyValue = property.getValue();
+
+            final PropertyDescriptor descriptor = component.getPropertyDescriptor(propertyName);
+            final ParameterTokenList tokenList = new StandardParameterTokenList(null, Collections.emptyList());
+            final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, tokenList, Collections.emptyList());
+
+            propertyMap.put(descriptor, propertyConfiguration);
+        }
+
+        final ValidationContext validationContext = new StandardValidationContext(controllerServiceProvider, propertyMap,

Review comment:
       Something fishy is going on with the validation here.  I added a required property:
   ```
       public static final PropertyDescriptor MESSAGE = new PropertyDescriptor.Builder()
               .displayName("Message")
               .name("message")
               .required(true)
               .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
   ```
   And I configured it via:
   ```
   nifi.stateless.parameter.provider.Custom Property Provider.name=My Custom Provider
   nifi.stateless.parameter.provider.Custom Property Provider.type=org.apache.nifi.stateless.parameter.gresock.MyCustomProvider
   nifi.stateless.parameter.provider.Custom Property Provider.bundle=org.apache.nifi:nifi-gresock-nar:1.14.0-SNAPSHOT
   nifi.stateless.parameter.provider.Custom Property Provider.properties.allowed-context-name=Context
   nifi.stateless.parameter.provider.Custom Property Provider.properties.message=A custom parameter:
   ```
   And I get this error:
   ```
   Exception in thread "main" java.lang.IllegalStateException: Could not create Parameter Provider Custom Property Provider of type org.apache.nifi.stateless.parameter.gresock.MyCustomProvider
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createParameterProvider(StandardStatelessEngine.java:245)
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createParameterProvider(StandardStatelessEngine.java:204)
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createFlow(StandardStatelessEngine.java:175)
   	at org.apache.nifi.stateless.flow.StandardStatelessDataflowFactory.createDataflow(StandardStatelessDataflowFactory.java:223)
   	at org.apache.nifi.stateless.bootstrap.StatelessBootstrap.createDataflow(StatelessBootstrap.java:66)
   	at org.apache.nifi.stateless.bootstrap.RunStatelessFlow.createDataflow(RunStatelessFlow.java:94)
   	at org.apache.nifi.stateless.bootstrap.RunStatelessFlow.main(RunStatelessFlow.java:56)
   Caused by: java.lang.IllegalStateException: Parameter Provider with name <Custom Property Provider> is not valid: ['Message' is invalid because Message is required]
   	at org.apache.nifi.stateless.engine.StandardStatelessEngine.createParameterProvider(StandardStatelessEngine.java:240)
   	... 6 more
   ```
   Digging into the validation code, I notice that in `PropertyConfiguration.getEffectiveValue(...)`, the `rawValue` is correctly set, but the following are also true:
   
   - parameterTokenList is not null, but it is empty
   - parameterLookup passed in is null, because the supplied parameterContext here is null
   - Therefore, the effectiveValue appears to always be set to null
   - The effectiveValue then overrides the rawValue, resulting in the validation context to always think the value is null for configured properties




-- 
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.

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