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/03/24 21:14:57 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7085: NIFI-11333: Do not allow components to be removed while a node is dis…

exceptionfactory commented on code in PR #7085:
URL: https://github.com/apache/nifi/pull/7085#discussion_r1148059705


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java:
##########
@@ -638,21 +668,37 @@ private boolean isMutableRequest(final String method) {
         }
     }
 
-    private boolean isDeleteConnection(final String method, final String uriPath) {
+    private boolean isDeleteComponent(final String method, final String uriPath) {
         if (!HttpMethod.DELETE.equalsIgnoreCase(method)) {
             return false;
         }
 
-        final boolean isConnectionUri = ConnectionEndpointMerger.CONNECTION_URI_PATTERN.matcher(uriPath).matches();
-        return isConnectionUri;
+        // Check if the URI indicates that a component should be deleted.
+        // We cannot simply make our decision based on the fact that the request is a DELETE request.
+        // This is because we do need to allow deletion of asynchronous requests, such as updating parameters, querying provenance, etc.
+        // which create a request, poll until the request completes, and then deletes it. Additionally, we want to allow terminating
+        // Processors, which is done by issuing a request to DELETE /processors/<id>/threads

Review Comment:
   This approach is understandable, but it functions as a list of blocked request patterns. Would it make more sense to reverse the logic to check against a list of "non-component" patterns so that it functions as a list of allowed request patterns? With that approach, unless the operation is explicitly listed, it will be cause the `IllegalClusterStateException`, which seems a bit safer in the event of new component types added to the framework. Not sure of the scope without further evaluation, but wanted to raise the question.



##########
nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java:
##########
@@ -424,30 +429,42 @@ public void testCanJoinClusterIfAllNodesMissingNar() throws NiFiClientException,
         assertTrue(getNifiClient().getProcessorClient().getProcessor(generate.getId()).getComponent().getExtensionMissing());
     }
 
+
     @Test
-    public void testCannotJoinIfMissingConnectionHasData() throws NiFiClientException, IOException, InterruptedException {
+    public void testCannotRemoveComponentsWhileNodeDisconnected() throws NiFiClientException, IOException, InterruptedException {
         final ProcessorEntity generate = getClientUtil().createProcessor("GenerateFlowFile");
         final ProcessorEntity terminate = getClientUtil().createProcessor("TerminateFlowFile");
         final ConnectionEntity connection = getClientUtil().createConnection(generate, terminate, "success");
 
-        getClientUtil().updateProcessorSchedulingPeriod(generate, "60 sec");
-
         // Shut down node 2
         disconnectNode(2);
+        waitForNodeState(2, NodeConnectionState.DISCONNECTED);
 
-        switchClientToNode(2);
-        getClientUtil().startProcessor(generate);
-        waitForQueueCount(connection.getId(), 1);
+        // Attempt to delete connection. It should throw an Exception.
+        try {
+            getNifiClient().getConnectionClient().deleteConnection(connection);
+            Assertions.fail("Was able to remove connection while node disconnected");
+        } catch (final Exception expected) {
+        }

Review Comment:
   The `assertThrows` method is preferable to the try-catch approach that was necessary in earlier versions of JUnit.
   ```suggestion
           assertThrows(Exception.class, () -> getNifiClient().getConnectionClient().deleteConnection(connection));
   ```



##########
nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java:
##########
@@ -424,30 +429,42 @@ public void testCanJoinClusterIfAllNodesMissingNar() throws NiFiClientException,
         assertTrue(getNifiClient().getProcessorClient().getProcessor(generate.getId()).getComponent().getExtensionMissing());
     }
 
+
     @Test
-    public void testCannotJoinIfMissingConnectionHasData() throws NiFiClientException, IOException, InterruptedException {
+    public void testCannotRemoveComponentsWhileNodeDisconnected() throws NiFiClientException, IOException, InterruptedException {
         final ProcessorEntity generate = getClientUtil().createProcessor("GenerateFlowFile");
         final ProcessorEntity terminate = getClientUtil().createProcessor("TerminateFlowFile");
         final ConnectionEntity connection = getClientUtil().createConnection(generate, terminate, "success");
 
-        getClientUtil().updateProcessorSchedulingPeriod(generate, "60 sec");
-
         // Shut down node 2
         disconnectNode(2);
+        waitForNodeState(2, NodeConnectionState.DISCONNECTED);
 
-        switchClientToNode(2);
-        getClientUtil().startProcessor(generate);
-        waitForQueueCount(connection.getId(), 1);
+        // Attempt to delete connection. It should throw an Exception.
+        try {
+            getNifiClient().getConnectionClient().deleteConnection(connection);
+            Assertions.fail("Was able to remove connection while node disconnected");
+        } catch (final Exception expected) {
+        }
 
-        switchClientToNode(1);
-        getNifiClient().getConnectionClient().deleteConnection(connection);
+        // Attempt to delete processor. It should throw an Exception.
+        try {
+            getNifiClient().getProcessorClient().deleteProcessor(generate);
+            Assertions.fail("Was able to remove connection while node disconnected");
+        } catch (final Exception expected) {
+        }

Review Comment:
   ```suggestion
           assertThrows(Exception.class, () -> getNifiClient().getProcessorClient().deleteProcessor(generate));
   ```



##########
nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java:
##########
@@ -399,6 +400,10 @@ public void testCannotJoinClusterIfMissingNar() throws NiFiClientException, IOEx
 
         // Wait for node to show as disconnected because it doesn't have the necessary nar
         waitForNodeState(2, NodeConnectionState.DISCONNECTED);
+
+        // Reconnect so that flow teardown can happen
+        reconnectNode(2);
+        waitForAllNodesConnected();

Review Comment:
   This results in test failures on both automated and local builds, which may indicate some other problem with this test method.



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