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 2022/08/17 10:17:23 UTC

[GitHub] [nifi] bejancsaba commented on a diff in pull request #6281: NIFI-10312 - Fix MiNiFi C2 integration

bejancsaba commented on code in PR #6281:
URL: https://github.com/apache/nifi/pull/6281#discussion_r947718839


##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2ClientService.java:
##########
@@ -41,8 +41,12 @@ public C2ClientService(C2Client client, C2HeartbeatFactory c2HeartbeatFactory, C
     }
 
     public void sendHeartbeat(RuntimeInfoWrapper runtimeInfoWrapper) {
-        C2Heartbeat c2Heartbeat = c2HeartbeatFactory.create(runtimeInfoWrapper);
-        client.publishHeartbeat(c2Heartbeat).ifPresent(this::processResponse);
+        try {
+            C2Heartbeat c2Heartbeat = c2HeartbeatFactory.create(runtimeInfoWrapper);
+            client.publishHeartbeat(c2Heartbeat).ifPresent(this::processResponse);
+        } catch (Exception e) {

Review Comment:
   I think the issues should be handled on the client side there is already logic to cover IOException when sending heartbeat with logging for  "Send Heartbeat failed [{}]" if there are cases when other exception is expected that should be covered there in my opinion. What do you think?



##########
c2/c2-client-bundle/c2-client-service/src/test/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandlerTest.java:
##########
@@ -62,12 +61,34 @@ void testUpdateConfigurationOperationHandlerCreateSuccess() {
     }
 
     @Test
-    void testHandleThrowsExceptionForIncorrectArg() {
+    void testHandleIncorrectArg() {
         UpdateConfigurationOperationHandler handler = new UpdateConfigurationOperationHandler(null, null, null);
         C2Operation operation = new C2Operation();
         operation.setArgs(INCORRECT_LOCATION_MAP);
 
-        IllegalStateException exception = assertThrows(IllegalStateException.class, () -> handler.handle(operation));
+        C2OperationAck response = handler.handle(operation);
+
+        assertEquals(C2OperationState.OperationState.NOT_APPLIED, response.getOperationState().getState());
+    }
+
+    @Test
+    void testHandleFlowIdInArg() {
+        Function<byte[], Boolean> successUpdate = x -> true;
+        when(flowIdHolder.getFlowId()).thenReturn(FLOW_ID);
+        when(client.retrieveUpdateContent(any())).thenReturn(Optional.of("content".getBytes()));
+        UpdateConfigurationOperationHandler handler = new UpdateConfigurationOperationHandler(client, flowIdHolder, successUpdate);
+        C2Operation operation = new C2Operation();
+        operation.setIdentifier(OPERATION_ID);
+
+        Map<String, String> args = new HashMap<>();
+        args.putAll(INCORRECT_LOCATION_MAP);
+        args.put("flowId", "argsFlowId");

Review Comment:
   Please use the FLOW_ID constant defined in the class under test. That way we can avoid test fragility in case of later changes.



##########
minifi/minifi-integration-tests/src/test/resources/logback.xml:
##########
@@ -55,7 +55,7 @@
     
     <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
     
-    <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi" level="DEBUG"/>

Review Comment:
   I understand it is useful for investigation but is it actually needed to be changed permanently. My guess is that it makes the log really noisy.



##########
c2/c2-client-bundle/c2-client-service/src/test/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandlerTest.java:
##########
@@ -17,13 +17,12 @@
 package org.apache.nifi.c2.client.service.operation;
 
 import static org.apache.commons.lang3.StringUtils.EMPTY;
-import static org.apache.nifi.c2.client.service.operation.UpdateConfigurationOperationHandler.LOCATION;

Review Comment:
   This was intentionally not private in the class so it can be used in the test without duplication. Can you revert back that change?



##########
c2/c2-protocol/c2-protocol-api/pom.xml:
##########
@@ -33,5 +33,9 @@ limitations under the License.
             <artifactId>c2-protocol-component-api</artifactId>
             <version>1.18.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>

Review Comment:
   I think we should avoid polluting the -api modules. They should be pure only with the models (maybe it will be used with different serialisation not Jackson somewhere). Could you look into a different way of addressing the issue this solves? We can discuss separately if needed.



##########
c2/c2-protocol/c2-protocol-api/src/main/java/org/apache/nifi/c2/protocol/api/OperandType.java:
##########
@@ -28,6 +30,7 @@ public enum OperandType {
     MANIFEST,
     REPOSITORY;
 
+    @JsonCreator

Review Comment:
   Please see my comment above.



##########
minifi/minifi-integration-tests/src/test/java/org/apache/nifi/minifi/integration/c2/C2ProtocolIntegrationTest.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.minifi.integration.c2;
+
+import com.palantir.docker.compose.DockerComposeExtension;
+import com.palantir.docker.compose.connection.waiting.HealthChecks;
+import org.apache.nifi.minifi.integration.util.LogUtil;
+import org.apache.nifi.security.util.KeystoreType;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.StandardTlsConfiguration;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandalone;
+import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandaloneCommandLine;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+@Timeout(60)
+public class C2ProtocolIntegrationTest {
+    private static final String AGENT_1 = "minifi-edge1";
+    private static final String AGENT_2 = "minifi-edge2";
+    private static final String AGENT_CLASS = "raspi3";
+    private static final String SERVICE = "c2-authoritative";
+    private static final String CONFIG_YAML = "config.text.yml.v2";
+    private static Path certificatesDirectory;
+    private static SSLContext trustSslContext;
+    private static SSLSocketFactory healthCheckSocketFactory;
+    public static DockerComposeExtension docker = DockerComposeExtension.builder()
+            .file("target/test-classes/docker-compose-c2-protocol.yml")
+            .waitingForService(AGENT_1, HealthChecks.toRespond2xxOverHttp(8000, dockerPort -> "http://" + dockerPort.getIp() + ":" + dockerPort.getExternalPort()))
+            .waitingForService(AGENT_2, HealthChecks.toRespond2xxOverHttp(8000, dockerPort -> "http://" + dockerPort.getIp() + ":" + dockerPort.getExternalPort()))
+            .build();
+
+    private static Path resourceDirectory;
+    private static Path authoritativeFiles;
+    private static Path minifiEdge1Version2;
+    private static Path minifiEdge2Version2;
+
+    /**
+     * Generates certificates with the tls-toolkit and then starts up the docker compose file
+     */
+    @BeforeAll
+    public static void init() throws Exception {
+        resourceDirectory = Paths.get(C2ProtocolIntegrationTest.class.getClassLoader()
+                .getResource("docker-compose-c2-protocol.yml").getFile()).getParent();
+        certificatesDirectory = resourceDirectory.toAbsolutePath().resolve("certificates-c2-protocol");
+        authoritativeFiles = resourceDirectory.resolve("c2").resolve("protocol").resolve(SERVICE).resolve("files");
+        minifiEdge1Version2 = authoritativeFiles.resolve("edge1").resolve(AGENT_CLASS).resolve(CONFIG_YAML);
+        minifiEdge2Version2 = authoritativeFiles.resolve("edge2").resolve(AGENT_CLASS).resolve(CONFIG_YAML);
+
+        if (Files.exists(minifiEdge1Version2)) {
+            Files.delete(minifiEdge1Version2);
+        }
+        if (Files.exists(minifiEdge2Version2)) {
+            Files.delete(minifiEdge2Version2);
+        }
+
+        List<String> toolkitCommandLine = new ArrayList<>(Arrays.asList("-O", "-o", certificatesDirectory.toFile().getAbsolutePath(), "-S", "badKeystorePass", "-P", "badTrustPass"));
+        for (String serverHostname : Arrays.asList(SERVICE, AGENT_1, AGENT_2)) {
+            toolkitCommandLine.add("-n");
+            toolkitCommandLine.add(serverHostname);
+        }
+        Files.createDirectories(certificatesDirectory);
+        TlsToolkitStandaloneCommandLine tlsToolkitStandaloneCommandLine = new TlsToolkitStandaloneCommandLine();
+        tlsToolkitStandaloneCommandLine.parse(toolkitCommandLine.toArray(new String[toolkitCommandLine.size()]));
+        new TlsToolkitStandalone().createNifiKeystoresAndTrustStores(tlsToolkitStandaloneCommandLine.createConfig());
+
+        TlsConfiguration tlsConfiguration = new StandardTlsConfiguration(
+                null,null,null,
+                certificatesDirectory.resolve(SERVICE).resolve("truststore.jks").toFile().getAbsolutePath(),
+                "badTrustPass",
+                KeystoreType.JKS);
+        trustSslContext = SslContextFactory.createSslContext(tlsConfiguration);
+        healthCheckSocketFactory = trustSslContext.getSocketFactory();
+
+        docker.before();
+    }
+
+    @AfterAll
+    public static void stopDocker() {
+        docker.after();
+    }
+
+    @Test
+    public void testMiNiFiHeartbeat() throws Exception {

Review Comment:
   Based on the expected.json this validates not only heartbeating but flow definition download as well. Could you please rename the test to reflect that?
   Can we tweak the test to have the two agents belonging to different classes? That would be  amore realistic test in my opinion. What do you think?



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