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/06/30 17:32:43 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6160: NIFI-10166 improve MiNiFi bootstrap test coverage

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


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -750,6 +756,72 @@ protected static void addTextElement(final Element element, final String name, f
         element.appendChild(toAdd);
     }
 
+    public static ByteBuffer overrideNonFlowSectionsFromOriginalSchema(byte[] newSchema, ByteBuffer currentConfigScheme, Properties bootstrapProperties)
+        throws IOException, InvalidConfigurationException, SchemaLoaderException {
+        try {
+            boolean overrideCoreProperties = ConfigTransformer.overrideCoreProperties(bootstrapProperties);
+            boolean overrideSecurityProperties = ConfigTransformer.overrideSecurityProperties(bootstrapProperties);
+            if (overrideCoreProperties && overrideSecurityProperties) {
+                return ByteBuffer.wrap(newSchema);
+            } else {
+                ConvertableSchema<ConfigSchema> schemaNew = ConfigTransformer
+                    .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteArrayInputStream(newSchema)));
+                ConfigSchema configSchemaNew = ConfigTransformer.throwIfInvalid(schemaNew.convert());
+                ConvertableSchema<ConfigSchema> schemaOld = ConfigTransformer
+                    .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteBufferInputStream(currentConfigScheme)));
+                ConfigSchema configSchemaOld = ConfigTransformer.throwIfInvalid(schemaOld.convert());
+
+                configSchemaNew.setNifiPropertiesOverrides(configSchemaOld.getNifiPropertiesOverrides());
+
+                if (!overrideCoreProperties) {
+                    logger.debug("Preserving previous core properties...");
+                    configSchemaNew.setCoreProperties(configSchemaOld.getCoreProperties());
+                }
+
+                if (!overrideSecurityProperties) {
+                    logger.debug("Preserving previous security properties...");
+                    configSchemaNew.setSecurityProperties(configSchemaOld.getSecurityProperties());
+                }
+
+                StringWriter writer = new StringWriter();
+                SchemaLoader.toYaml(configSchemaNew, writer);
+                return ByteBuffer.wrap(writer.toString().getBytes()).asReadOnlyBuffer();
+            }
+        } catch (Exception e) {
+            logger.error("Loading the old and the new schema for merging was not successful", e);
+            throw e;

Review Comment:
   Is there a reason for logging the error and re-throwing the exception? It might be helpful to wrap the exception and throw it with a message, instead of this approach.



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/command/CompositeCommandRunnerTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+import static org.mockito.MockitoAnnotations.openMocks;
+
+import java.util.Arrays;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class CompositeCommandRunnerTest {
+
+    private CommandRunner startRunner;
+    private CommandRunner stopRunner;
+    private CompositeCommandRunner compositeCommandRunner;
+
+    @BeforeEach
+    void setup() {
+        startRunner = mock(StartRunner.class);
+        stopRunner = mock(StopRunner.class);

Review Comment:
   Is there a reason for using `mock()` instead of the `Mock` annotation?



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodecTest.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.bootstrap.service;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.stream.Stream;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeCoordinator;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+class BootstrapCodecTest {
+
+    private static final int VALID_PORT = 1;
+    private static final String SECRET = "secret";
+    private static final String OK = "OK";
+    private static final String EMPTY_STRING = "";
+    private RunMiNiFi runner;
+
+    @BeforeEach
+    void setup() {
+        runner = mock(RunMiNiFi.class);
+    }
+
+    @Test
+    void testCommunicateShouldThrowIOExceptionIfThereIsNoCommand() {
+        InputStream inputStream = new ByteArrayInputStream(new byte[0]);
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        BootstrapCodec bootstrapCodec = new BootstrapCodec(runner, inputStream, outputStream);
+
+        IOException expectedException = assertThrows(IOException.class, bootstrapCodec::communicate);
+        assertEquals("Received invalid command from MiNiFi: null", expectedException.getMessage());

Review Comment:
   Asserting the value of an exception message should be avoided. If it is important to indicate something specific in the message, it would be better check for the presence of a particular keyword.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -750,6 +756,72 @@ protected static void addTextElement(final Element element, final String name, f
         element.appendChild(toAdd);
     }
 
+    public static ByteBuffer overrideNonFlowSectionsFromOriginalSchema(byte[] newSchema, ByteBuffer currentConfigScheme, Properties bootstrapProperties)
+        throws IOException, InvalidConfigurationException, SchemaLoaderException {
+        try {
+            boolean overrideCoreProperties = ConfigTransformer.overrideCoreProperties(bootstrapProperties);
+            boolean overrideSecurityProperties = ConfigTransformer.overrideSecurityProperties(bootstrapProperties);
+            if (overrideCoreProperties && overrideSecurityProperties) {
+                return ByteBuffer.wrap(newSchema);
+            } else {
+                ConvertableSchema<ConfigSchema> schemaNew = ConfigTransformer
+                    .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteArrayInputStream(newSchema)));
+                ConfigSchema configSchemaNew = ConfigTransformer.throwIfInvalid(schemaNew.convert());
+                ConvertableSchema<ConfigSchema> schemaOld = ConfigTransformer
+                    .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteBufferInputStream(currentConfigScheme)));
+                ConfigSchema configSchemaOld = ConfigTransformer.throwIfInvalid(schemaOld.convert());
+
+                configSchemaNew.setNifiPropertiesOverrides(configSchemaOld.getNifiPropertiesOverrides());
+
+                if (!overrideCoreProperties) {
+                    logger.debug("Preserving previous core properties...");
+                    configSchemaNew.setCoreProperties(configSchemaOld.getCoreProperties());
+                }
+
+                if (!overrideSecurityProperties) {
+                    logger.debug("Preserving previous security properties...");
+                    configSchemaNew.setSecurityProperties(configSchemaOld.getSecurityProperties());
+                }
+
+                StringWriter writer = new StringWriter();
+                SchemaLoader.toYaml(configSchemaNew, writer);
+                return ByteBuffer.wrap(writer.toString().getBytes()).asReadOnlyBuffer();
+            }
+        } catch (Exception e) {
+            logger.error("Loading the old and the new schema for merging was not successful", e);
+            throw e;
+        }
+    }
+
+    private static boolean overrideSecurityProperties(Properties properties) {
+        String overrideSecurityProperties = (String) properties.getOrDefault(OVERRIDE_SECURITY, "false");
+        boolean overrideSecurity;
+        if ("true".equalsIgnoreCase(overrideSecurityProperties) || "false".equalsIgnoreCase(overrideSecurityProperties)) {
+            overrideSecurity = Boolean.parseBoolean(overrideSecurityProperties);
+        } else {
+            throw new IllegalArgumentException(
+                "Property, " + OVERRIDE_SECURITY + ", to specify whether to override security properties must either be a value boolean value (\"true\" or \"false\")" +
+                    " or left to the default value of \"false\". It is set to \"" + overrideSecurityProperties + "\".");
+        }
+
+        return overrideSecurity;
+    }
+
+    private static boolean overrideCoreProperties(Properties properties) {
+        String overrideCorePropertiesKey = PULL_HTTP_BASE_KEY + ".override.core";
+        String overrideCoreProps = (String) properties.getOrDefault(overrideCorePropertiesKey, "false");
+        boolean overrideCoreProperties;
+        if ("true".equalsIgnoreCase(overrideCoreProps) || "false".equalsIgnoreCase(overrideCoreProps)) {
+            overrideCoreProperties = Boolean.parseBoolean(overrideCoreProps);
+        } else {
+            throw new IllegalArgumentException(
+                "Property, " + overrideCorePropertiesKey + ", to specify whether to override core properties must either be a value boolean value (\"true\" or \"false\")" +
+                    " or left to the default value of \"false\". It is set to \"" + overrideCoreProps + "\".");
+        }

Review Comment:
   See note above on Boolean parsing.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/configuration/ingestors/PullHttpChangeIngestor.java:
##########
@@ -234,8 +218,9 @@ public void run() {
                 .build();
 
         final Request.Builder requestBuilder = new Request.Builder()
-                .get()
-                .url(url);
+            .get()
+            .cacheControl(CacheControl.FORCE_NETWORK)

Review Comment:
   Is there a particular reason for adding this option for force network requests?



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/ShutdownHookTest.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.bootstrap;
+
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.STOP;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class ShutdownHookTest {
+
+    @Mock
+    private RunMiNiFi runner;
+    @Mock
+    private MiNiFiStdLogHandler miNiFiStdLogHandler;
+
+    @InjectMocks
+    private ShutdownHook shutdownHook;
+
+    @Test
+    void testRunShouldShutdownSchedulersAndProcesses() {
+        PeriodicStatusReporterManager periodicStatusReporterManager = mock(PeriodicStatusReporterManager.class);

Review Comment:
   Is there a reason for calling `mock()` instead of using a member variable annotated with `Mock`?



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/configuration/ingestors/PullHttpChangeIngestorSSLTest.java:
##########
@@ -76,10 +84,22 @@ public void pullHttpChangeIngestorInit(Properties properties) {
         properties.put(PullHttpChangeIngestor.PORT_KEY, String.valueOf(port));
         properties.put(PullHttpChangeIngestor.HOST_KEY, "localhost");
         properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
+        properties.put(PULL_HTTP_BASE_KEY + ".override.core", "true");
+        ConfigurationFileHolder configurationFileHolder = Mockito.mock(ConfigurationFileHolder.class);
+
+        try {
+            ConfigSchema configSchema =
+                SchemaLoader.loadConfigSchemaFromYaml(PullHttpChangeIngestorSSLTest.class.getClassLoader().getResourceAsStream("config.yml"));
+            StringWriter writer = new StringWriter();
+            SchemaLoader.toYaml(configSchema, writer);
+            when(configurationFileHolder.getConfigFileReference()).thenReturn(new AtomicReference<>(ByteBuffer.wrap(writer.toString().getBytes())));
+        } catch (Exception e) {
+            fail("Failed to read test config file", e);
+        }

Review Comment:
   Instead of catching the exception, the method should be changed to declare an Exception, which would also fail the test if thrown.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.bootstrap.util;
+
+import java.io.IOException;
+
+public interface ProcessUtils {
+    boolean isProcessRunning(Long pid);
+
+    void gracefulShutDownMiNiFiProcess(Long pid, String s, int gracefulShutdownSeconds);
+
+    void killProcessTree(Long pid) throws IOException;
+
+    /**
+     * Checks the status of the given process.
+     *
+     * @param process the process object what we want to check
+     * @return true if the process is Alive
+     */
+     default boolean isAlive(Process process) {
+        try {
+            process.exitValue();
+            return false;
+        } catch (IllegalStateException | IllegalThreadStateException itse) {
+            return true;
+        }

Review Comment:
   Is there a reason for this approach as opposed to calling `Process.isAlive()`?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.bootstrap.util;
+
+import java.io.IOException;
+
+public interface ProcessUtils {
+    boolean isProcessRunning(Long pid);
+
+    void gracefulShutDownMiNiFiProcess(Long pid, String s, int gracefulShutdownSeconds);

Review Comment:
   Since this is more generic, what do you think about renaming to `shutdownProcess()`?



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/TestLogAppender.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.bootstrap;
+
+import ch.qos.logback.classic.spi.ILoggingEvent;
+import ch.qos.logback.core.AppenderBase;
+import java.util.ArrayList;
+
+/**
+ * Helper class which makes possible to subscribe to logging events, and make assertions based on it.
+ * With this approach we can avoid static mocking of loggers, which is impossible in some cases.
+ */
+public class TestLogAppender extends AppenderBase<ILoggingEvent> {

Review Comment:
   Unit testing the presence of logging messages should be avoided in general, as it introduces a greater degree of evaluation than necessary, making it more difficult to refactor.



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/command/CommandRunnerFactoryTest.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.DUMP;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.ENV;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.FLOWSTATUS;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.RESTART;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.RUN;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.START;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.STATUS;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.STOP;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.UNKNOWN;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.verifyNoInteractions;
+
+import java.io.File;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.GracefulShutdownParameterProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class CommandRunnerFactoryTest {
+
+    @Mock
+    private MiNiFiCommandSender miNiFiCommandSender;
+    @Mock
+    private CurrentPortProvider currentPortProvider;
+    @Mock
+    private MiNiFiParameters miNiFiParameters;
+    @Mock
+    private MiNiFiStatusProvider miNiFiStatusProvider;
+    @Mock
+    private PeriodicStatusReporterManager periodicStatusReporterManager;
+    @Mock
+    private BootstrapFileProvider bootstrapFileProvider;
+    @Mock
+    private MiNiFiStdLogHandler miNiFiStdLogHandler;
+    @Mock
+    private File bootstrapConfigFile;
+    @Mock
+    private RunMiNiFi runMiNiFi;
+    @Mock
+    private GracefulShutdownParameterProvider gracefulShutdownParameterProvider;
+    @Mock
+    private MiNiFiExecCommandProvider miNiFiExecCommandProvider;
+
+    @InjectMocks
+    private CommandRunnerFactory commandRunnerFactory;
+
+    @Test
+    void testRunCommandShouldStartCommandReturnStartRunner() {
+        CommandRunner runner = commandRunnerFactory.getRunner(START);
+
+        assertTrue(runner instanceof StartRunner);

Review Comment:
   JUnit 5 now supports `assertInstanceOf`, which can be used here and in other methods.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -750,6 +756,72 @@ protected static void addTextElement(final Element element, final String name, f
         element.appendChild(toAdd);
     }
 
+    public static ByteBuffer overrideNonFlowSectionsFromOriginalSchema(byte[] newSchema, ByteBuffer currentConfigScheme, Properties bootstrapProperties)
+        throws IOException, InvalidConfigurationException, SchemaLoaderException {
+        try {
+            boolean overrideCoreProperties = ConfigTransformer.overrideCoreProperties(bootstrapProperties);
+            boolean overrideSecurityProperties = ConfigTransformer.overrideSecurityProperties(bootstrapProperties);
+            if (overrideCoreProperties && overrideSecurityProperties) {
+                return ByteBuffer.wrap(newSchema);
+            } else {
+                ConvertableSchema<ConfigSchema> schemaNew = ConfigTransformer
+                    .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteArrayInputStream(newSchema)));
+                ConfigSchema configSchemaNew = ConfigTransformer.throwIfInvalid(schemaNew.convert());
+                ConvertableSchema<ConfigSchema> schemaOld = ConfigTransformer
+                    .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteBufferInputStream(currentConfigScheme)));
+                ConfigSchema configSchemaOld = ConfigTransformer.throwIfInvalid(schemaOld.convert());
+
+                configSchemaNew.setNifiPropertiesOverrides(configSchemaOld.getNifiPropertiesOverrides());
+
+                if (!overrideCoreProperties) {
+                    logger.debug("Preserving previous core properties...");
+                    configSchemaNew.setCoreProperties(configSchemaOld.getCoreProperties());
+                }
+
+                if (!overrideSecurityProperties) {
+                    logger.debug("Preserving previous security properties...");
+                    configSchemaNew.setSecurityProperties(configSchemaOld.getSecurityProperties());
+                }
+
+                StringWriter writer = new StringWriter();
+                SchemaLoader.toYaml(configSchemaNew, writer);
+                return ByteBuffer.wrap(writer.toString().getBytes()).asReadOnlyBuffer();
+            }
+        } catch (Exception e) {
+            logger.error("Loading the old and the new schema for merging was not successful", e);
+            throw e;
+        }
+    }
+
+    private static boolean overrideSecurityProperties(Properties properties) {
+        String overrideSecurityProperties = (String) properties.getOrDefault(OVERRIDE_SECURITY, "false");
+        boolean overrideSecurity;
+        if ("true".equalsIgnoreCase(overrideSecurityProperties) || "false".equalsIgnoreCase(overrideSecurityProperties)) {
+            overrideSecurity = Boolean.parseBoolean(overrideSecurityProperties);
+        } else {
+            throw new IllegalArgumentException(
+                "Property, " + OVERRIDE_SECURITY + ", to specify whether to override security properties must either be a value boolean value (\"true\" or \"false\")" +
+                    " or left to the default value of \"false\". It is set to \"" + overrideSecurityProperties + "\".");
+        }

Review Comment:
   Is there a reason for this approach as opposed to just calling `Boolean.parseBoolean()`? If it is necessary to implement this more restrictive Boolean parsing, it would be helpful to introduce a shared method



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/command/CommandRunnerFactoryTest.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.DUMP;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.ENV;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.FLOWSTATUS;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.RESTART;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.RUN;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.START;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.STATUS;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.STOP;
+import static org.apache.nifi.minifi.bootstrap.BootstrapCommand.UNKNOWN;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.verifyNoInteractions;
+
+import java.io.File;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.GracefulShutdownParameterProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class CommandRunnerFactoryTest {
+
+    @Mock
+    private MiNiFiCommandSender miNiFiCommandSender;
+    @Mock
+    private CurrentPortProvider currentPortProvider;
+    @Mock
+    private MiNiFiParameters miNiFiParameters;
+    @Mock
+    private MiNiFiStatusProvider miNiFiStatusProvider;
+    @Mock
+    private PeriodicStatusReporterManager periodicStatusReporterManager;
+    @Mock
+    private BootstrapFileProvider bootstrapFileProvider;
+    @Mock
+    private MiNiFiStdLogHandler miNiFiStdLogHandler;
+    @Mock
+    private File bootstrapConfigFile;
+    @Mock
+    private RunMiNiFi runMiNiFi;
+    @Mock
+    private GracefulShutdownParameterProvider gracefulShutdownParameterProvider;
+    @Mock
+    private MiNiFiExecCommandProvider miNiFiExecCommandProvider;
+
+    @InjectMocks
+    private CommandRunnerFactory commandRunnerFactory;
+
+    @Test
+    void testRunCommandShouldStartCommandReturnStartRunner() {
+        CommandRunner runner = commandRunnerFactory.getRunner(START);
+
+        assertTrue(runner instanceof StartRunner);
+        verifyNoInteractions(miNiFiCommandSender, currentPortProvider, miNiFiParameters, miNiFiStatusProvider, periodicStatusReporterManager, bootstrapFileProvider,
+            miNiFiStdLogHandler, bootstrapConfigFile, runMiNiFi, gracefulShutdownParameterProvider, miNiFiExecCommandProvider);
+    }
+
+    @Test
+    void testRunCommandShouldRunCommandReturnStartRunner() {
+        CommandRunner runner = commandRunnerFactory.getRunner(RUN);
+
+        assertTrue(runner instanceof StartRunner);
+        verifyNoInteractions(miNiFiCommandSender, currentPortProvider, miNiFiParameters, miNiFiStatusProvider, periodicStatusReporterManager, bootstrapFileProvider,

Review Comment:
   Is this verification necessary?



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/command/DumpRunnerTest.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.command.DumpRunner.DUMP_CMD;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Optional;
+import org.apache.nifi.minifi.bootstrap.TestLogAppender;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class DumpRunnerTest {
+
+    private static final int MINIFI_PORT = 1337;
+    private static final String DUMP_CONTENT = "dump_content";
+    private static final TestLogAppender LOG_APPENDER = new TestLogAppender();
+
+    @Mock
+    private MiNiFiCommandSender miNiFiCommandSender;
+    @Mock
+    private CurrentPortProvider currentPortProvider;
+
+    @InjectMocks
+    private DumpRunner dumpRunner;
+
+    @BeforeAll
+    static void setupAll() {
+        ((ch.qos.logback.classic.Logger) CMD_LOGGER).addAppender(LOG_APPENDER);
+        LOG_APPENDER.start();
+    }
+
+    @BeforeEach
+    void setup() {
+        LOG_APPENDER.reset();
+    }
+
+    @Test
+    void testRunCommandShouldDumpToConsoleIfNoFileDefined() throws IOException {
+        when(currentPortProvider.getCurrentPort()).thenReturn(MINIFI_PORT);
+        when(miNiFiCommandSender.sendCommand(DUMP_CMD, MINIFI_PORT)).thenReturn(Optional.of(DUMP_CONTENT));
+
+        int statusCode = dumpRunner.runCommand(new String[0]);
+
+        assertEquals(OK.getStatusCode(), statusCode);
+        assertEquals(DUMP_CONTENT, LOG_APPENDER.getLastLoggedEvent().getMessage());
+        verifyNoMoreInteractions(currentPortProvider, miNiFiCommandSender);
+    }
+
+    @Test
+    void testRunCommandShouldDumpToFileIfItIsDefined() throws IOException {
+        when(currentPortProvider.getCurrentPort()).thenReturn(MINIFI_PORT);
+        when(miNiFiCommandSender.sendCommand(DUMP_CMD, MINIFI_PORT)).thenReturn(Optional.of(DUMP_CONTENT));
+        File file = Files.createTempFile(null, null).toFile();
+        file.deleteOnExit();
+        String tmpFilePath = file.getAbsolutePath();
+
+        int statusCode = dumpRunner.runCommand(new  String[] {DUMP_CMD, tmpFilePath});
+
+        assertEquals(OK.getStatusCode(), statusCode);
+        assertEquals(DUMP_CONTENT, getDumpContent(file));
+        assertEquals("Successfully wrote thread dump to " + tmpFilePath, LOG_APPENDER.getLastLoggedEvent().getFormattedMessage());

Review Comment:
   Asserting the value of log messages should be avoided. Recommend removing this check and other logging checks.



##########
minifi/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/configuration/ingestors/PullHttpChangeIngestorTest.java:
##########
@@ -55,9 +63,21 @@ public void pullHttpChangeIngestorInit(Properties properties) {
         properties.put(PullHttpChangeIngestor.PORT_KEY, String.valueOf(port));
         properties.put(PullHttpChangeIngestor.HOST_KEY, "localhost");
         properties.put(PullHttpChangeIngestor.PULL_HTTP_POLLING_PERIOD_KEY, "30000");
+        properties.put(PULL_HTTP_BASE_KEY + ".override.core", "true");
+        ConfigurationFileHolder configurationFileHolder = Mockito.mock(ConfigurationFileHolder.class);
+
+        try {
+            ConfigSchema configSchema =
+                SchemaLoader.loadConfigSchemaFromYaml(PullHttpChangeIngestorTest.class.getClassLoader().getResourceAsStream("config.yml"));
+            StringWriter writer = new StringWriter();
+            SchemaLoader.toYaml(configSchema, writer);
+            when(configurationFileHolder.getConfigFileReference()).thenReturn(new AtomicReference<>(ByteBuffer.wrap(writer.toString().getBytes())));
+        } catch (Exception e) {
+            fail("Failed to read test config file", e);
+        }

Review Comment:
   As noted, this should be changed to declare an Exception instead of catching it and failing.



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