You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "markap14 (via GitHub)" <gi...@apache.org> on 2023/05/16 14:33:24 UTC

[GitHub] [nifi] markap14 opened a new pull request, #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

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

   <!-- 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. -->
   
   # Summary
   
   [NIFI-00000](https://issues.apache.org/jira/browse/NIFI-00000)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
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] exceptionfactory commented on a diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1224572117


##########
nifi-api/src/main/java/org/apache/nifi/components/PortFunction.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.components;
+
+public enum PortFunction {
+    /**
+     * Port functions as a standard port
+     */
+    STANDARD,

Review Comment:
   The name and description do not say much about the behavior. Perhaps that is intentional since it may be a success or retry condition. Perhaps it would be helpful to indicate the `STANDARD` indicates default behavior, which could include success, retry, or some unspecified condition.



##########
nifi-api/src/main/java/org/apache/nifi/flow/ExecutionEngine.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.flow;
+
+public enum ExecutionEngine {
+    /**
+     * Run using the traditional NiFi engine
+     */
+    TRADITIONAL,

Review Comment:
   Although `TRADITIONAL` has historical meaning, perhaps something else would communicate the standard behavior more clearly. `STANDARD` could be an option, as `TRADITIONAL` could imply a sort of legacy nature, which is not the case. Another option could be `STATEFUL` as parallel to `STATELESS`. Perhaps `PROCESSOR_ORIENTED` as another option? If we stay with `TRADITIONAL`, perhaps additional comments could help.



##########
nifi-docs/src/main/asciidoc/user-guide.pdf:
##########


Review Comment:
   It seems like the rendered PDF should not be included, correct?



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsSnapshotDTO.java:
##########
@@ -538,6 +549,93 @@ public StorageUsageDTO clone() {
         }
     }
 
+    @XmlType(name = "resourceClaimDetails")
+    public static class ResourceClaimDetailsDTO implements Cloneable {
+        private String container;
+        private String section;
+        private String identifier;
+        private Boolean inUse;
+        private Boolean awaitingDestruction;
+        private Boolean writable;

Review Comment:
   Can primitive `boolean` values be used? That would default the value to `false` as opposed to allowing the ambiguous `null` for these properties.



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java:
##########
@@ -2674,7 +2683,7 @@ ControllerServiceReferencingComponentsEntity updateControllerServiceReferencingC
     /**
      * @return the system diagnostics
      */
-    SystemDiagnosticsDTO getSystemDiagnostics();
+    SystemDiagnosticsDTO getSystemDiagnostics(boolean verbose);

Review Comment:
   Should we make this argument an enum named `DiagnosticLevel` or something? Perhaps this won't change much, but it might be worth considering.



##########
nifi-api/src/main/java/org/apache/nifi/components/PortFunction.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.components;
+
+public enum PortFunction {
+    /**
+     * Port functions as a standard port
+     */
+    STANDARD,
+
+    /**
+     * Port denotes that the invocation of the stateless flow has failed. If run using the Traditional Engine, will operate as a Standard port.
+     */
+    FAILURE_PORT;

Review Comment:
   Is it necessary to include `PORT` in the name since `Port` is already included in the name `PortFunction`?



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java:
##########
@@ -1240,6 +1247,11 @@ public FlowControllerComponentScheduler(final FlowController flowController, fin
 
         @Override
         public void startNow(final Connectable component) {
+            if (ExecutionEngine.STATELESS == component.getProcessGroup().resolveExecutionEngine()) {
+                logger.info("{} should be running but will not start it because its Process Group is configured to run Stateless", component);

Review Comment:
   Should this be an info message, or does it make more sense as a debug?



##########
nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/registry/RegistryClientIT.java:
##########
@@ -199,59 +199,65 @@ public void testChangeConnectionDestinationRemoveOldAndMoveGroup() throws NiFiCl
 
     @Test
     public void testControllerServiceUpdateWhileRunning() throws NiFiClientException, IOException, InterruptedException {
-        final FlowRegistryClientEntity clientEntity = registerClient();
-        final NiFiClientUtil util = getClientUtil();
-
-        final ProcessGroupEntity group = util.createProcessGroup("Parent", "root");
-        final ControllerServiceEntity service = util.createControllerService("StandardCountService", group.getId());
-
-        final ProcessorEntity generate = util.createProcessor("GenerateFlowFile", group.getId());
-        final ProcessorEntity countProcessor = util.createProcessor("CountFlowFiles", group.getId());
-        util.updateProcessorProperties(countProcessor, Collections.singletonMap("Count Service", service.getComponent().getId()));
-
-        final ProcessorEntity terminate = util.createProcessor("TerminateFlowFile", group.getId());
-        final ConnectionEntity connectionToTerminate = util.createConnection(countProcessor, terminate, "success");
-        util.setFifoPrioritizer(connectionToTerminate);
-        util.createConnection(generate, countProcessor, "success");
-
-        // Save the flow as v1
-        final VersionControlInformationEntity vci = util.startVersionControl(group, clientEntity, "testControllerServiceUpdateWhileRunning", "Parent");
-
-        // Change the value of of the Controller Service's start value to 2000, and change the text of the GenerateFlowFile just to make it run each time the version is changed
-        util.updateControllerServiceProperties(service, Collections.singletonMap("Start Value", "2000"));
-        util.updateProcessorProperties(generate, Collections.singletonMap("Text", "Hello World"));
-
-        // Save the flow as v2
-        util.saveFlowVersion(group, clientEntity, vci);
-
-        // Change back to v1 and start the flow
-        util.changeFlowVersion(group.getId(), 1);
-        util.assertFlowStaleAndUnmodified(group.getId());
-        util.enableControllerService(service);
-
-        util.waitForValidProcessor(generate.getId());
-        util.startProcessor(generate);
-        util.waitForValidProcessor(countProcessor.getId());
-        util.startProcessor(countProcessor);
-
-        // Ensure that we get the expected result
-        waitForQueueCount(connectionToTerminate.getId(), getNumberOfNodes());
-        final Map<String, String> firstFlowFileAttributes = util.getQueueFlowFile(connectionToTerminate.getId(), 0).getFlowFile().getAttributes();
-        assertEquals("1", firstFlowFileAttributes.get("count"));
-
-        // Change to v2 and ensure that the output is correct
-        util.changeFlowVersion(group.getId(), 2);
-        util.assertFlowUpToDate(group.getId());
-        waitForQueueCount(connectionToTerminate.getId(), 2 * getNumberOfNodes());
-        final Map<String, String> secondFlowFileAttributes = util.getQueueFlowFile(connectionToTerminate.getId(), getNumberOfNodes()).getFlowFile().getAttributes();
-        assertEquals("2001", secondFlowFileAttributes.get("count"));
-
-        // Change back to v1 and ensure that the output is correct. It should reset count back to 0.
-        util.changeFlowVersion(group.getId(), 1);
-        util.assertFlowStaleAndUnmodified(group.getId());
-        waitForQueueCount(connectionToTerminate.getId(), 3 * getNumberOfNodes());
-        final Map<String, String> thirdFlowFileAttributes = util.getQueueFlowFile(connectionToTerminate.getId(), getNumberOfNodes() * 2).getFlowFile().getAttributes();
-        assertEquals("1", thirdFlowFileAttributes.get("count"));
+        // TODO: This try/catch/log/rethrow is temporary
+        try {

Review Comment:
   It looks like this try-catch should be removed?



-- 
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 pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1646145973

   Force pushed to rebase against main, due to conflicts.


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648247430

   With provenance in Volatile mode now I see stateless flow running with all 5 threads.  I think my laptop will melt


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1647814647

   Just experimenting with it.  It appears the 'stateless flow timeout' is always shown on the process group settings but presumably only matters when the execution engine is 'stateless'.  Perhaps we can ensure a tooltip shows up on the process group properties so a user could quickly read that property is often irrelevant/only useful if the engine selection is stateless?


-- 
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 pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648122873

   OK pushed some changes @joewitt . Will now automatically show/hide the "Max  Concurrent Tasks" and "Stateless Flow Timeout" settings based on the chosen Execution Engine. Also addressed wording of the error messages.


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648267778

   I see this in the logs once I tell nifi to restart.
   
   `2023-07-24 12:48:52,219 ERROR [Run Dataflow Stateless Group] o.a.n.s.flow.StandardStatelessFlow Failed to execute dataflow
   java.lang.RuntimeException: Dataflow canceled
           at org.apache.nifi.stateless.engine.StandardExecutionProgress.enqueueTriggerResult(StandardExecutionProgress.java:178)
           at org.apache.nifi.stateless.flow.StandardStatelessFlow.executeDataflow(StandardStatelessFlow.java:571)
           at org.apache.nifi.stateless.flow.StandardStatelessFlow.lambda$trigger$5(StandardStatelessFlow.java:511)
           at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
           at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
           at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
           at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
           at java.base/java.lang.Thread.run(Thread.java:833)`


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1647822545

   Went to try and put a stateful group inside a stateless group and it does block it which is cool.  But the error message buries the lede a bit.  
   
   "Cannot change Execution Engine for StandardProcessGroup[identifier=87dcc86d-0189-1000-9176-d4a998c280f0,name=Stateful Group] to STANDARD because parent group StandardProcessGroup[identifier=87d912dd-0189-1000-13e8-b1b11eb1c2e6,name=Stateless Group] is configured to use the Stateless Engine. A Process Group using the Stateless Engine may be embedded within a Process Group using the Traditional Engine, but the reverse is not allowed."
   
   Instead can we reverse the order of that output, be consistent about references to 'standard vs traditional' and simplify the language to avoid referring to the 'reverse' situation.
   
   For instance:
   
   "A process group using a standard engine may not be a child of a process group using the stateless engine.  Cannot set PG[bla-standard] as a child of PG[bla2-stateless]."


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648371000

   nifi appears stuck runnig the integration tests/system tests in nifi-system-tests with mvn clean install -Pintegration-tests
   
   It isn't moving and stack dump shows
   
   `"main" #1 prio=5 os_prio=31 cpu=1795.56ms elapsed=2750.21s tid=0x000000011a008a00 nid=0x2203 in Object.wait()  [0x0000000170083000]
      java.lang.Thread.State: WAITING (on object monitor)
           at java.lang.Object.wait(java.base@17.0.8/Native Method)
           - waiting on <0x0000000700a00058> (a java.lang.Thread)
           at java.lang.Thread.join(java.base@17.0.8/Thread.java:1304)
           - locked <0x0000000700a00058> (a java.lang.Thread)
           at java.lang.Thread.join(java.base@17.0.8/Thread.java:1372)
           at org.apache.nifi.tests.system.AggregateNiFiInstance.start(AggregateNiFiInstance.java:49)
           at org.apache.nifi.tests.system.NiFiInstanceCache$CachedNiFiInstance.start(NiFiInstanceCache.java:189)
           at org.apache.nifi.tests.system.NiFiInstance.start(NiFiInstance.java:32)
           at org.apache.nifi.tests.system.NiFiSystemIT.setup(NiFiSystemIT.java:121)
           at jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(java.base@17.0.8/Native Method)
           at jdk.internal.reflect.NativeMethodAccessorImpl.invoke(java.base@17.0.8/NativeMethodAccessorImpl.java:77)
           at jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(java.base@17.0.8/DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(java.base@17.0.8/Method.java:568)
           at org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:727)
           at org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
           at org.junit.jupiter.engine.execution.InvocationInterceptorChain$ValidatingInvocation.proceed(InvocationInterceptorChain.java:131)
           at org.junit.jupiter.engine.extension.TimeoutExtension.intercept(TimeoutExtension.java:156)`


-- 
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 pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1591861324

   Rebased to main in order to address conflicts and squashed commits in order to make that rebase easier. Thanks for the feedback @exceptionfactory 


-- 
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 pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1647998014

   @joewitt I was able to replicate the issue around the scheduling period. It looks like this bug is actually present on the `main` branch. Was brought into this branch through a recent rebase. The bug is actually in the UI and was introduced as part of NIFI-11813. I have re-opened that Jira. Thanks for catching & calling that out!


-- 
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 diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1229841152


##########
nifi-api/src/main/java/org/apache/nifi/flow/ExecutionEngine.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.flow;
+
+public enum ExecutionEngine {
+    /**
+     * Run using the traditional NiFi engine
+     */
+    TRADITIONAL,

Review Comment:
   That's fair, I think `STANDARD` is a reasonable option. Will update to 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] markap14 commented on a diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1229841868


##########
nifi-docs/src/main/asciidoc/user-guide.pdf:
##########


Review Comment:
   Whoops, good 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.

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

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


[GitHub] [nifi] exceptionfactory closed pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow
URL: https://github.com/apache/nifi/pull/7253


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1647839934

   On root group created a process group and selected stateless engine.
   
   In that stateless group created a GenerateFlowFile -> UpdateAttribute flow with a batch size of 500 and 1KB non unique text objects.  Realized I cannot start/stop individual components and instead must do so at the group level based on error responses.  Ideally we would prevent those buttons from being used at all as otherwise we're doing a user experience based on 'try and fail' which is often not fun.  Ideally we improve that now or in time.
   
   When I started the group I realized I had the wrong scheduling period set as the default is 1min so a single flowfile went through.  I changed to '0 secs' and tried again.  Nothing happened and I noticed that scheduling period was returned to '1 min' suggesting my changes aren't taking effect.  


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648264843

   Ok again it did not restart the stateless group.
   
   So build a stateless flow in nifi.  Have it running.  Restart NiFi.  Then it all comes back up running.  Restart nifi again without doing anything else and the stateless flow will come up not running.  Traditional flows do run though.
   
   Repeated that pattern twice. and it happened twice.  


-- 
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 diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1229844503


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsSnapshotDTO.java:
##########
@@ -538,6 +549,93 @@ public StorageUsageDTO clone() {
         }
     }
 
+    @XmlType(name = "resourceClaimDetails")
+    public static class ResourceClaimDetailsDTO implements Cloneable {
+        private String container;
+        private String section;
+        private String identifier;
+        private Boolean inUse;
+        private Boolean awaitingDestruction;
+        private Boolean writable;

Review Comment:
   For better or worse, we use Object representations of all field values in DTOs. Very early on, it was determined that the DTO should contain no logic - even providing default values for primitives, which is what happens if using a primitive type. Rather, the DTO should simply convey exactly what was set.



-- 
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 diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1229849910


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java:
##########
@@ -1240,6 +1247,11 @@ public FlowControllerComponentScheduler(final FlowController flowController, fin
 
         @Override
         public void startNow(final Connectable component) {
+            if (ExecutionEngine.STATELESS == component.getProcessGroup().resolveExecutionEngine()) {
+                logger.info("{} should be running but will not start it because its Process Group is configured to run Stateless", component);

Review Comment:
   I think it makes sense as an INFO. I don't believe it's a condition that we're really likely to hit, except maybe if an update changes the engine for a Process Group from Standard to Stateless. It that case, I don't know off the top of my head whether this would hit or not. You could argue that maybe WARN makes more sense, if we don't think it's expected. But since there's no real adverse side effect, I think a WARN is stronger than necessary and can cause alarm for no reason.



-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1647844667

   If I then change the flow to be traditional with generate flowfile feeding into stateless group with updateattribute the strange scheduling behavior remains.  Cannot change from 1min to 0 secs as it keeps returning to its old form.  But when I start things it runs a couple times then stops.   The behavior seems a good bit off even for traditional flow.  I did copy/paste from the stateless group to the traditional so maybe that is a factor.


-- 
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] markobean commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markobean (via GitHub)" <gi...@apache.org>.
markobean commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1671595710

   I am just beginning to review this and it will take some time because it is a significant change. A couple comments off the bat.
   
   Can you add "help" dialogs for the new properties for the configuration of a process group - specifically Execution Engine, Max Concurrent Tasks, Stateless Flow Timeout? See the Log File Suffix property for an example.
   
   Can you add some information in the PR description to assist in evaluating this one? It is non-trivial to say the least and not having any description of what was added/modified, why, or what areas to focus on makes reviewing even more challenging. 
   
   Thanks!


-- 
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] exceptionfactory commented on a diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1284843573


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java:
##########
@@ -413,4 +418,42 @@ public String getLogFileSuffix() {
     public void setLogFileSuffix(final String logFileSuffix) {
         this.logFileSuffix = logFileSuffix;
     }
+
+    @ApiModelProperty(value = "The Execution Engine that should be used to run the flow represented by this Process Group.",
+        allowableValues = "STATELESS, TRADITIONAL, INHERITED")

Review Comment:
   ```suggestion
           allowableValues = "STATELESS, STANDARD, INHERITED")
   ```



-- 
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] exceptionfactory commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1671667010

   Thanks for taking a look @markobean. As you noted, there is a lot here, but it builds on the core capabilities of Stateless execution. If you are not familiar with Stateless operation, or the ExecuteStateless Processor, it would be worth reviewing the associated documentation for those features.
   
   It is worth giving some consideration to how approachable this capability is for general users, so improving usability is worth doing. However, some of those items could be considered under a follow-on effort. One of the main concerns with this pull request is to ensure existing functionality works, and the new optional functionality operates within reasonable parameters.


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648204222

   The above info was with run duration set above zero.  With run duration at 0 on all things the behavior seems to possibly flip with standard being slower due to backlogs on connection hitting 10K slowing it down.


-- 
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 pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648565058

   Thanks @joewitt . I haven't been able to replicate an issue with the system test freezing, but based on the thread dump it looks like the nifi instance failed to startup. Perhaps due to a port conflict or something of that nature?
   
   I did track down the issue with the Stateless Group not starting up when NiFi is restarted, though. I pushed a fix for 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.

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 diff in pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7253:
URL: https://github.com/apache/nifi/pull/7253#discussion_r1229856084


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java:
##########
@@ -2674,7 +2683,7 @@ ControllerServiceReferencingComponentsEntity updateControllerServiceReferencingC
     /**
      * @return the system diagnostics
      */
-    SystemDiagnosticsDTO getSystemDiagnostics();
+    SystemDiagnosticsDTO getSystemDiagnostics(boolean verbose);

Review Comment:
   I like it.



-- 
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] exceptionfactory commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1671595742

   The latest version has a clean shutdown with the Controller Service reference scenario described, thanks @markap14!
   
   I pushed a correction for the unused import, otherwise this looks ready to go pending successful builds.


-- 
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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648197571

   Had a scenario I figured would cause problems.  Nope.  It works perfectly
   
   Cannot change Execution Engine for StandardProcessGroup[identifier=888d6dd0-0189-1000-e88e-b0a720bf53e6,name=Stateless Group] while components are running. UpdateAttribute[id=888d916e-0189-1000-749a-0d0497e2b6e2] is currently running.
   
   Running a typical GenerateFF->UpdateAttr flow in both standard and stateless.  Seeing performance being double the speed in standard vs stateless which I think surprises me.  Stateless has 5 concurrent threads whereas standard has 1 task for generate and update.  It *seems* like standard runs both generate and update at once whereas stateless runs one then the other and never more than one.  So that makes sense speed is double in standard case.  
   
   Other note: We should deprecate in 1.x line the ExecuteStateless processor.  This will be superior in every way to 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] joewitt commented on pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1648245469

   I had a flow running in a PG that is traditional and a PG that is stateless.  After restart the stateless PG did not automatically restart.


-- 
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 pull request #7253: NIFI-11556: Added ability to use a Process Group as a Stateless Flow

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on PR #7253:
URL: https://github.com/apache/nifi/pull/7253#issuecomment-1647910567

   Thanks for testing it out @joewitt. I agree that the message on that error could be improved. Will update that.
   I will try to replicate the issue that you pointed out regarding the scheduling.
   
   As for the UI showing things that aren't really applicable: I agree, ideally we hide things that are not applicable. However, my UI skills are very much lacking. So I did what I felt was minimal in terms of UI. I think we could actually go pretty far there - even removing the 'Scheduling' tab from processors in a Stateless Group that have incoming connections. I do feel that could be a follow-on Jira, though, to improve the UI and remove the elements that are not relevant in a Stateless group.


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