You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "exceptionfactory (via GitHub)" <gi...@apache.org> on 2023/06/09 18:52:49 UTC

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

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