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/07 16:00:32 UTC

[GitHub] [nifi] ferencerdei commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

ferencerdei commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891005901


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/configuration/ConfigurationChangeCoordinator.java:
##########
@@ -99,16 +84,43 @@ public Collection<ListenerHandleResult> notifyListeners(ByteBuffer newConfig) {
                 result = new ListenerHandleResult(listener, ex);
             }
             listenerHandleResults.add(result);
-            logger.info("Listener notification result:" + result.toString());
+            LOGGER.info("Listener notification result: {}", result);
         }
         return listenerHandleResults;
     }
 
 
     @Override
-    public void close() throws IOException {
-        for (ChangeIngestor changeIngestor : changeIngestors) {
-            changeIngestor.close();
+    public void close() {
+        try {
+            for (ChangeIngestor changeIngestor : changeIngestors) {
+                changeIngestor.close();
+            }
+            changeIngestors.clear();
+        } catch (IOException e) {
+            LOGGER.warn("Could not successfully stop notifiers", e);
+        }
+    }
+
+    private void initialize() {
+        close();
+        // cleanup previously initialized ingestors
+        String ingestorsCsv = bootstrapProperties.getProperty(NOTIFIER_INGESTORS_KEY);
+
+        if (ingestorsCsv != null && !ingestorsCsv.isEmpty()) {
+            for (String ingestorClassname : ingestorsCsv.split(",")) {
+                ingestorClassname = ingestorClassname.trim();
+                try {
+                    Class<?> ingestorClass = Class.forName(ingestorClassname);
+                    ChangeIngestor changeIngestor = (ChangeIngestor) ingestorClass.newInstance();
+                    changeIngestor.initialize(bootstrapProperties, runMiNiFi, this);
+                    changeIngestors.add(changeIngestor);
+                    LOGGER.info("Initialized ingestor: {}", ingestorClassname);
+                } catch (Exception e) {
+                    LOGGER.error("Issue instantiating {} ingestor", ingestorClassname);
+                    LOGGER.error("Exception", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.RunMiNiFi.DEFAULT_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 java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);

Review Comment:
   The bootstrap(commands) and the MiNiFi can run on different JVM, so with this method we return the MiNiFi's properties and not the bootstrap process. (This code is a port of RunNiFi)



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";
+                File javaFile = new File(javaHome + File.separatorChar + "bin"
+                    + File.separatorChar + "java" + fileExtension);
+                if (javaFile.exists() && javaFile.canExecute()) {
+                    javaCmd = javaFile.getAbsolutePath();
+                }
+            }
+        }
+
+        MiNiFiListener listener = new MiNiFiListener();
+        int listenPort = listener.start(runMiNiFi);
+
+        List<String> cmd = new ArrayList<>();
+
+        cmd.add(javaCmd);
+        cmd.add("-classpath");
+        cmd.add(classPath);
+        cmd.addAll(javaAdditionalArgs);
+        cmd.add("-Dnifi.properties.file.path=" + minifiPropsFilename);
+        cmd.add("-Dnifi.bootstrap.listen.port=" + listenPort);
+        cmd.add("-Dapp=MiNiFi");
+        cmd.add("-Dorg.apache.nifi.minifi.bootstrap.config.log.dir="+minifiLogDir);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/WindowsService.java:
##########
@@ -17,26 +17,20 @@
 package org.apache.nifi.minifi.bootstrap;
 
 import java.io.IOException;
-import java.io.File;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
 
 public class WindowsService {

Review Comment:
   This class is used only in windows when the MiNiFi is installed through the install-service.bat.
   It uses the apache commons daemon which requires a class with a start and a stop method.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodec.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapCodec {
+
+    private static final String TRUE = "true";
+    private static final String FALSE = "false";

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.RunMiNiFi.DEFAULT_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 java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodec.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapCodec {
+
+    private static final String TRUE = "true";
+    private static final String FALSE = "false";
+
+    private final RunMiNiFi runner;
+    private final BufferedReader reader;
+    private final BufferedWriter writer;
+    private final Logger logger = LoggerFactory.getLogger(BootstrapCodec.class);
+
+    public BootstrapCodec(RunMiNiFi runner, InputStream in, OutputStream out) {
+        this.runner = runner;
+        this.reader = new BufferedReader(new InputStreamReader(in));
+        this.writer = new BufferedWriter(new OutputStreamWriter(out));
+    }
+
+    public void communicate() throws IOException {
+        String line = reader.readLine();
+        String[] splits = line.split(" ");
+        if (splits.length == 0) {
+            throw new IOException("Received invalid command from MiNiFi: " + line);
+        }
+
+        String cmd = splits[0];
+        String[] args;
+        if (splits.length == 1) {
+            args = new String[0];
+        } else {
+            args = Arrays.copyOfRange(splits, 1, splits.length);
+        }
+
+        try {
+            processRequest(cmd, args);
+        } catch (InvalidCommandException exception) {
+            throw new IOException("Received invalid command from MiNiFi: " + line + " : " + (exception.getMessage() == null ? "" : "Details: " + exception));

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);

Review Comment:
   Added the secret.key to the secret keys enum so it will be filtered out.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+public class MiNiFiStatus {
+
+    private final Integer port;

Review Comment:
   Can be null until the MiNiFi process startup is in progress.



##########
nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java:
##########
@@ -26,13 +26,13 @@
 import java.util.Set;
 
 /**
- *
+ * The main interface for declaring a NiFi-based server application
  */
 public interface NiFiServer {
 
     void start();
 
-    void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping);
+    void initialize(final NiFiProperties properties, final Bundle systemBundle, final Set<Bundle> bundles, final ExtensionMapping extensionMapping);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }

Review Comment:
   I'm not sure if that would work. This while loop guarantees that the bootstrap doesn't exit before the MiNiFi process. This is the same approach as in NiFi bootstrap.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java:
##########
@@ -200,1289 +150,38 @@ private static void printUsage() {
         System.out.println();
     }
 
-    public static void main(String[] args) throws IOException, InterruptedException {
-        if (args.length < 1 || args.length > 3) {
-            printUsage();
-            return;
-        }
-
-        File dumpFile = null;
-
-        final String cmd = args[0];
-        if (cmd.equals("dump")) {
-            if (args.length > 1) {
-                dumpFile = new File(args[1]);
-            } else {
-                dumpFile = null;
-            }
-        }
-
-        switch (cmd.toLowerCase()) {
-            case "start":
-            case "run":
-            case "stop":
-            case "status":
-            case "dump":
-            case "restart":
-            case "env":
-            case "flowstatus":
-                break;
-            default:
-                printUsage();
-                return;
-        }
-
-        final File configFile = getBootstrapConfFile();
-        final RunMiNiFi runMiNiFi = new RunMiNiFi(configFile);
-
-        Integer exitStatus = null;
-        switch (cmd.toLowerCase()) {
-            case "start":
-                runMiNiFi.start();
-                break;
-            case "run":
-                runMiNiFi.start();
-                break;
-            case "stop":
-                runMiNiFi.stop();
-                break;
-            case "status":
-                exitStatus = runMiNiFi.status();
-                break;
-            case "restart":
-                runMiNiFi.stop();
-                runMiNiFi.start();
-                break;
-            case "dump":
-                runMiNiFi.dump(dumpFile);
-                break;
-            case "env":
-                runMiNiFi.env();
-                break;
-            case "flowstatus":
-                if(args.length == 2) {
-                    System.out.println(runMiNiFi.statusReport(args[1]));
-                } else {
-                    System.out.println("The 'flowStatus' command requires an input query. See the System Admin Guide 'FlowStatus Script Query' section for complete details.");
-                }
-                break;
-        }
-        if (exitStatus != null) {
-            System.exit(exitStatus);
-        }
-    }
-
-    public static File getBootstrapConfFile() {
-        String configFilename = System.getProperty("org.apache.nifi.minifi.bootstrap.config.file");
-
-        if (configFilename == null) {
-            final String nifiHome = System.getenv("MINIFI_HOME");
-            if (nifiHome != null) {
-                final File nifiHomeFile = new File(nifiHome.trim());
-                final File configFile = new File(nifiHomeFile, DEFAULT_CONFIG_FILE);
-                configFilename = configFile.getAbsolutePath();
-            }
-        }
-
-        if (configFilename == null) {
-            configFilename = DEFAULT_CONFIG_FILE;
-        }
-
-        final File configFile = new File(configFilename);
-        return configFile;
-    }
-
-    private File getBootstrapFile(final Logger logger, String directory, String defaultDirectory, String fileName) throws IOException {
-
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File nifiHome = confDir.getParentFile();
-
-        String confFileDir = System.getProperty(directory);
-
-        final File fileDir;
-
-        if (confFileDir != null) {
-            fileDir = new File(confFileDir.trim());
-        } else {
-            fileDir = new File(nifiHome, defaultDirectory);
-        }
-
-        FileUtils.ensureDirectoryExistAndCanAccess(fileDir);
-        final File statusFile = new File(fileDir, fileName);
-        logger.debug("Status File: {}", statusFile);
-        return statusFile;
-    }
-
-    File getPidFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_PID_FILE_NAME);
-    }
-
-    File getStatusFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_STATUS_FILE_NAME);
-    }
-
-    File getLockFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_LOCK_FILE_NAME);
-    }
-
-    File getStatusFile() throws IOException{
-        return getStatusFile(defaultLogger);
-    }
-
-    public File getReloadFile(final Logger logger) {
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File nifiHome = confDir.getParentFile();
-        final File bin = new File(nifiHome, "bin");
-        final File reloadFile = new File(bin, "minifi.reload.lock");
-
-        logger.debug("Reload File: {}", reloadFile);
-        return reloadFile;
-    }
-
-    public File getSwapFile(final Logger logger) {
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File swapFile = new File(confDir, "swap.yml");
-
-        logger.debug("Swap File: {}", swapFile);
-        return swapFile;
-    }
-
-
-    private Properties loadProperties(final Logger logger) throws IOException {
-        final Properties props = new Properties();
-        final File statusFile = getStatusFile(logger);
-        if (statusFile == null || !statusFile.exists()) {
-            logger.debug("No status file to load properties from");
-            return props;
-        }
-
-        try (final FileInputStream fis = new FileInputStream(getStatusFile(logger))) {
-            props.load(fis);
-        }
-
-        final Map<Object, Object> modified = new HashMap<>(props);
-        modified.remove("secret.key");
-        logger.debug("Properties: {}", modified);
-
-        return props;
-    }
-
-    private synchronized void saveProperties(final Properties minifiProps, final Logger logger) throws IOException {
-        final String pid = minifiProps.getProperty(PID_KEY);
-        if (!StringUtils.isBlank(pid)) {
-            writePidFile(pid, logger);
-        }
-
-        final File statusFile = getStatusFile(logger);
-        if (statusFile.exists() && !statusFile.delete()) {
-            logger.warn("Failed to delete {}", statusFile);
-        }
-
-        if (!statusFile.createNewFile()) {
-            throw new IOException("Failed to create file " + statusFile);
-        }
-
-        try {
-            final Set<PosixFilePermission> perms = new HashSet<>();
-            perms.add(PosixFilePermission.OWNER_WRITE);
-            perms.add(PosixFilePermission.OWNER_READ);
-            perms.add(PosixFilePermission.GROUP_READ);
-            perms.add(PosixFilePermission.OTHERS_READ);
-            Files.setPosixFilePermissions(statusFile.toPath(), perms);
-        } catch (final Exception e) {
-            logger.warn("Failed to set permissions so that only the owner can read status file {}; "
-                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
-                + "Permissions should be changed so that only the owner can read this file", statusFile);
-        }
-
-        try (final FileOutputStream fos = new FileOutputStream(statusFile)) {
-            minifiProps.store(fos, null);
-            fos.getFD().sync();
-        }
-
-        logger.debug("Saved Properties {} to {}", new Object[]{minifiProps, statusFile});
-    }
-
-    private synchronized void writePidFile(final String pid, final Logger logger) throws IOException {
-        final File pidFile = getPidFile(logger);
-        if (pidFile.exists() && !pidFile.delete()) {
-           logger.warn("Failed to delete {}", pidFile);
-        }
-
-        if (!pidFile.createNewFile()) {
-            throw new IOException("Failed to create file " + pidFile);
-        }
-
-        try {
-            final Set<PosixFilePermission> perms = new HashSet<>();
-            perms.add(PosixFilePermission.OWNER_READ);
-            perms.add(PosixFilePermission.OWNER_WRITE);
-            Files.setPosixFilePermissions(pidFile.toPath(), perms);
-        } catch (final Exception e) {
-            logger.warn("Failed to set permissions so that only the owner can read pid file {}; "
-                    + "this may allows others to have access to the key needed to communicate with MiNiFi. "
-                    + "Permissions should be changed so that only the owner can read this file", pidFile);
-        }
-
-        try (final FileOutputStream fos = new FileOutputStream(pidFile)) {
-            fos.write(pid.getBytes(StandardCharsets.UTF_8));
-            fos.getFD().sync();
-        }
-
-        logger.debug("Saved Pid {} to {}", new Object[]{pid, pidFile});
-    }
-
-    private boolean isPingSuccessful(final int port, final String secretKey, final Logger logger) {
-        logger.debug("Pinging {}", port);
-
-        try (final Socket socket = new Socket("localhost", port)) {
-            final OutputStream out = socket.getOutputStream();
-            out.write((PING_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
-            out.flush();
-
-            logger.debug("Sent PING command");
-            socket.setSoTimeout(5000);
-            final InputStream in = socket.getInputStream();
-            final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-            final String response = reader.readLine();
-            logger.debug("PING response: {}", response);
-            out.close();
-            reader.close();
-
-            return PING_CMD.equals(response);
-        } catch (final IOException ioe) {
-            return false;
-        }
-    }
-
-    private Integer getCurrentPort(final Logger logger) throws IOException {
-        final Properties props = loadProperties(logger);
-        final String portVal = props.getProperty("port");
-        if (portVal == null) {
-            logger.debug("No Port found in status file");
-            return null;
-        } else {
-            logger.debug("Port defined in status file: {}", portVal);
-        }
-
-        final int port = Integer.parseInt(portVal);
-        final boolean success = isPingSuccessful(port, props.getProperty("secret.key"), logger);
-        if (success) {
-            logger.debug("Successful PING on port {}", port);
-            return port;
-        }
-
-        final String pid = props.getProperty(PID_KEY);
-        logger.debug("PID in status file is {}", pid);
-        if (pid != null) {
-            final boolean procRunning = isProcessRunning(pid, logger);
-            if (procRunning) {
-                return port;
-            } else {
-                return null;
-            }
-        }
-
-        return null;
-    }
-
-    private boolean isProcessRunning(final String pid, final Logger logger) {
-        try {
-            // We use the "ps" command to check if the process is still running.
-            final ProcessBuilder builder = new ProcessBuilder();
-
-            builder.command("ps", "-p", pid);
-            final Process proc = builder.start();
-
-            // Look for the pid in the output of the 'ps' command.
-            boolean running = false;
-            String line;
-            try (final InputStream in = proc.getInputStream();
-                 final Reader streamReader = new InputStreamReader(in);
-                 final BufferedReader reader = new BufferedReader(streamReader)) {
-
-                while ((line = reader.readLine()) != null) {
-                    if (line.trim().startsWith(pid)) {
-                        running = true;
-                    }
-                }
-            }
-
-            // If output of the ps command had our PID, the process is running.
-            if (running) {
-                logger.debug("Process with PID {} is running", pid);
-            } else {
-                logger.debug("Process with PID {} is not running", pid);
-            }
-
-            return running;
-        } catch (final IOException ioe) {
-            System.err.println("Failed to determine if Process " + pid + " is running; assuming that it is not");
-            return false;
-        }
-    }
-
-    private Status getStatus(final Logger logger) {
-        final Properties props;
-        try {
-            props = loadProperties(logger);
-        } catch (final IOException ioe) {
-            return new Status(null, null, false, false);
-        }
-
-        if (props == null) {
-            return new Status(null, null, false, false);
-        }
-
-        final String portValue = props.getProperty("port");
-        final String pid = props.getProperty(PID_KEY);
-        final String secretKey = props.getProperty("secret.key");
-
-        if (portValue == null && pid == null) {
-            return new Status(null, null, false, false);
-        }
-
-        Integer port = null;
-        boolean pingSuccess = false;
-        if (portValue != null) {
-            try {
-                port = Integer.parseInt(portValue);
-                pingSuccess = isPingSuccessful(port, secretKey, logger);
-            } catch (final NumberFormatException nfe) {
-                return new Status(null, null, false, false);
-            }
-        }
-
-        if (pingSuccess) {
-            return new Status(port, pid, true, true);
-        }
-
-        final boolean alive = (pid != null) && isProcessRunning(pid, logger);
-        return new Status(port, pid, pingSuccess, alive);
-    }
-
-    public int status() throws IOException {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        if (status.isRespondingToPing()) {
-            logger.info("Apache MiNiFi is currently running, listening to Bootstrap on port {}, PID={}",
-                new Object[]{status.getPort(), status.getPid() == null ? "unknown" : status.getPid()});
-            return 0;
-        }
-
-        if (status.isProcessRunning()) {
-            logger.info("Apache MiNiFi is running at PID {} but is not responding to ping requests", status.getPid());
-            return 4;
-        }
-
-        if (status.getPort() == null) {
-            logger.info("Apache MiNiFi is not running");
-            return 3;
-        }
-
-        if (status.getPid() == null) {
-            logger.info("Apache MiNiFi is not responding to Ping requests. The process may have died or may be hung");
-        } else {
-            logger.info("Apache MiNiFi is not running");
-        }
-        return 3;
-    }
-
-    public FlowStatusReport statusReport(String statusRequest) throws IOException {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        final Properties props = loadProperties(logger);
-
-        List<String> problemsGeneratingReport = new LinkedList<>();
-        if (!status.isProcessRunning()) {
-            problemsGeneratingReport.add("MiNiFi process is not running");
-        }
-
-        if (!status.isRespondingToPing()) {
-            problemsGeneratingReport.add("MiNiFi process is not responding to pings");
-        }
-
-        if (!problemsGeneratingReport.isEmpty()) {
-            FlowStatusReport flowStatusReport = new FlowStatusReport();
-            flowStatusReport.setErrorsGeneratingReport(problemsGeneratingReport);
-            return flowStatusReport;
-        }
-
-        return getFlowStatusReport(statusRequest, status.getPort(), props.getProperty("secret.key"), logger);
-    }
-
-    public void env() {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        if (status.getPid() == null) {
-            logger.info("Apache MiNiFi is not running");
+    public void setMiNiFiParameters(int port, String secretKey) throws IOException {
+        if (Optional.ofNullable(secretKey).filter(key -> key.equals(miNiFiParameters.getSecretKey())).isPresent() && miNiFiParameters.getMiNiFiPort() == port) {
+            DEFAULT_LOGGER.debug("secretKey and port match with the known one, nothing to update");
             return;
         }
-        final Class<?> virtualMachineClass;
-        try {
-            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
-        } catch (final ClassNotFoundException cnfe) {
-            logger.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
-            return;
-        }
-        final Method attachMethod;
-        final Method detachMethod;
 
-        try {
-            attachMethod = virtualMachineClass.getMethod("attach", String.class);
-            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
-        } catch (final Exception e) {
-            logger.error("Methods required for getting environment not available", e);
-            return;
-        }
+        miNiFiParameters.setMiNiFiPort(port);
+        miNiFiParameters.setSecretKey(secretKey);
 
-        final Object virtualMachine;
-        try {
-            virtualMachine = attachMethod.invoke(null, status.getPid());
-        } catch (final Throwable t) {
-            logger.error("Problem attaching to MiNiFi", t);
-            return;
+        Properties minifiProps = new Properties();
+        long minifiPid = miNiFiParameters.getMinifiPid();
+        if (minifiPid != UNINITIALIZED) {
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(minifiPid));
         }
+        minifiProps.setProperty(STATUS_FILE_PORT_KEY, String.valueOf(port));
+        minifiProps.setProperty(STATUS_FILE_SECRET_KEY, secretKey);
 
+        File statusFile = bootstrapFileProvider.getStatusFile();
         try {
-            final Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
-
-            final Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
-            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
-                logger.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
-            }
-        } catch (Throwable t) {
-            throw new RuntimeException(t);
-        } finally {
-            try {
-                detachMethod.invoke(virtualMachine);
-            } catch (final Exception e) {
-                logger.warn("Caught exception detaching from process", e);
-            }
-        }
-    }
-
-    /**
-     * Writes a MiNiFi thread dump to the given file; if file is null, logs at
-     * INFO level instead.
-     *
-     * @param dumpFile the file to write the dump content to
-     * @throws IOException if any issues occur while writing the dump file
-     */
-    public void dump(final File dumpFile) throws IOException {
-        final Logger logger = defaultLogger;    // dump to bootstrap log file by default
-        final Integer port = getCurrentPort(logger);
-        if (port == null) {
-            logger.info("Apache MiNiFi is not currently running");
-            return;
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        } catch (IOException ioe) {
+            DEFAULT_LOGGER.warn("Apache MiNiFi has started but failed to persist MiNiFi Port information to {} due to {}", statusFile.getAbsolutePath(), ioe);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+public class MiNiFiStatus {
+
+    private final Integer port;
+    private final String pid;

Review Comment:
   There is not, I can change it to Long. (it can be also null / same reason as for the port / , and doesn't fit to Int)
   We use OsUtils from NiFi commons, but according to the java 9+ documentation it's a long value here as well: https://docs.oracle.com/javase/9/docs/api/java/lang/ProcessHandle.html#pid--



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProcessUtils {

Review Comment:
   You are totally right, naively I thought somehow these are "translated" to the right system command within the JVM, so I simply reused the existing code from NiFi bootstrap. I can rename it for now, but we might need to investigate if we need to implement an OS agnostic solution.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {

Review Comment:
   extracted the command building logics to MiNiFiExecCommandProvider



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);

Review Comment:
   Yes, it can contain the keystore / truststore keys for example. I'll introduce an enum with the sensitive keys, so we can filter based on this.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);
+
+        return props;
+    }
+
+    public synchronized void saveStatusProperties(Properties minifiProps) throws IOException {
+        String pid = minifiProps.getProperty(STATUS_FILE_PID_KEY);
+        if (!StringUtils.isBlank(pid)) {
+            writePidFile(pid);
+        }
+
+        File statusFile = getStatusFile();
+        if (statusFile.exists() && !statusFile.delete()) {
+            LOGGER.warn("Failed to delete {}", statusFile);
+        }
+
+        if (!statusFile.createNewFile()) {
+            throw new IOException("Failed to create file " + statusFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.GROUP_READ);
+            perms.add(PosixFilePermission.OTHERS_READ);
+            Files.setPosixFilePermissions(statusFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read status file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", statusFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(statusFile)) {
+            minifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saving MiNiFi properties to {}", statusFile);
+        logProperties("Saved MiNiFi", minifiProps);
+    }
+
+    private void writePidFile(String pid) throws IOException {
+        File pidFile = getPidFile();
+        if (pidFile.exists() && !pidFile.delete()) {
+            LOGGER.warn("Failed to delete {}", pidFile);
+        }
+
+        if (!pidFile.createNewFile()) {
+            throw new IOException("Failed to create file " + pidFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            Files.setPosixFilePermissions(pidFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read pid file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", pidFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(pidFile)) {
+            fos.write(pid.getBytes(StandardCharsets.UTF_8));
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saved Pid {} to {}", pid, pidFile);
+    }
+
+    private File getBootstrapFile(String fileName) throws IOException {
+        File configFileDir = Optional.ofNullable(System.getProperty(MINIFI_PID_DIR_PROP))
+            .map(String::trim)
+            .map(File::new)
+            .orElseGet(() -> {
+                File confDir = bootstrapConfigFile.getParentFile();
+                File nifiHome = confDir.getParentFile();
+                return new File(nifiHome, DEFAULT_PID_DIR);
+            });
+
+        FileUtils.ensureDirectoryExistAndCanAccess(configFileDir);
+        File statusFile = new File(configFileDir, fileName);
+        LOGGER.debug("Run File: {}", statusFile);
+
+        return statusFile;
+    }
+
+    private void logProperties(String type, Properties props) {
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("{} properties: {}", type, props.entrySet()
+                .stream()
+                .filter(e -> {
+                    String key = ((String) e.getKey()).toLowerCase();
+                    return !"secret.key".equals(key) && !key.contains("password") && !key.contains("passwd");

Review Comment:
   I believe logging properties can be useful when debugging so introduced an enum to be able to filter out the sensitive props.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            response = castResponse(cmd, o);
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise). Exception:" + e;
+            LOGGER.error(message);
+            throw new RuntimeException(message);
+        }
+        return response;
+    }
+
+    private <T> T castResponse(String cmd, Object o) {
+        T response;
+        try {
+            response = (T) o;
+        } catch (ClassCastException e) {
+            String message = "Failed to cast " + cmd + " response to the requested type";
+            LOGGER.error(message);
+            throw new RuntimeException(message);

Review Comment:
   Removed the logging as it's done in different level as well.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            response = castResponse(cmd, o);
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise). Exception:" + e;
+            LOGGER.error(message);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);
+
+        return props;
+    }
+
+    public synchronized void saveStatusProperties(Properties minifiProps) throws IOException {
+        String pid = minifiProps.getProperty(STATUS_FILE_PID_KEY);
+        if (!StringUtils.isBlank(pid)) {
+            writePidFile(pid);
+        }
+
+        File statusFile = getStatusFile();
+        if (statusFile.exists() && !statusFile.delete()) {
+            LOGGER.warn("Failed to delete {}", statusFile);
+        }
+
+        if (!statusFile.createNewFile()) {
+            throw new IOException("Failed to create file " + statusFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.GROUP_READ);
+            perms.add(PosixFilePermission.OTHERS_READ);
+            Files.setPosixFilePermissions(statusFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read status file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", statusFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(statusFile)) {
+            minifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saving MiNiFi properties to {}", statusFile);
+        logProperties("Saved MiNiFi", minifiProps);

Review Comment:
   Introduced a filter based on the secret keys enum



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.RunMiNiFi.DEFAULT_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 java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";

Review Comment:
   Do you mean by executing the "where java" command ?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -88,42 +91,68 @@ public final class ConfigTransformer {
     private ConfigTransformer() {
     }
 
+    public static ByteBuffer generateConfigFiles(InputStream configIs, String configDestinationPath, Properties bootstrapProperties) throws ConfigurationChangeException, IOException {

Review Comment:
   No from my side, it's just an existing code movement. If you don't mind I would leave it as it is for now because I'm unsure about the impact of the change. We'll move to the flow json format soon, so this whole config transformer will be removed / refactored.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/CommandRunner.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+public interface CommandRunner {
+
+    /**
+     * Executes a command.
+     * @param args the input arguments
+     * @return status code
+     */
+    int runCommand(String[] args);

Review Comment:
   sure, changed it



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.RunMiNiFi.DEFAULT_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 java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
+            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
+                CMD_LOGGER.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
+            }
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during the systemproperties call");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        } finally {
+            try {
+                detachMethod.invoke(virtualMachine);
+            } catch (final Exception e) {
+                CMD_LOGGER.warn("Caught exception detaching from process", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.RunMiNiFi.DEFAULT_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 java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");

Review Comment:
   Works with java8 and 11 but the "virtualMachine.getClass().getMethod("getSystemProperties");" throws exception on java 17.
   `java.lang.IllegalAccessException: class org.apache.nifi.minifi.bootstrap.command.EnvRunner cannot access class sun.tools.attach.HotSpotVirtualMachine (in module jdk.attach) because module jdk.attach does not export sun.tools.attach to unnamed module @41a962cf
   `
   Do we need to support java 17?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.RunMiNiFi.DEFAULT_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 java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
+            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
+                CMD_LOGGER.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
+            }
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during the systemproperties call");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   Please see my answer from the previous comment.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down

Review Comment:
   done



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