You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2016/04/01 00:03:11 UTC

nifi-minifi git commit: MINIFI-9 initial commit for bootstrapping/init process

Repository: nifi-minifi
Updated Branches:
  refs/heads/master bf22d35e4 -> 302ae7c64


MINIFI-9 initial commit for bootstrapping/init process

This closes #4.


Project: http://git-wip-us.apache.org/repos/asf/nifi-minifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi-minifi/commit/302ae7c6
Tree: http://git-wip-us.apache.org/repos/asf/nifi-minifi/tree/302ae7c6
Diff: http://git-wip-us.apache.org/repos/asf/nifi-minifi/diff/302ae7c6

Branch: refs/heads/master
Commit: 302ae7c6495fc00fe255f8f7d436266d8257cd22
Parents: bf22d35
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Thu Mar 31 17:31:12 2016 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Thu Mar 31 18:02:52 2016 -0400

----------------------------------------------------------------------
 minifi-assembly/LICENSE                         |   36 +-
 minifi-assembly/pom.xml                         |   37 +
 .../src/main/assembly/dependencies.xml          |    2 +
 minifi-bootstrap/pom.xml                        |   18 +
 .../nifi/minifi/bootstrap/BootstrapCodec.java   |  108 ++
 .../nifi/minifi/bootstrap/MiNiFiListener.java   |  141 +++
 .../apache/nifi/minifi/bootstrap/RunMiNiFi.java | 1186 ++++++++++++++++++
 .../nifi/minifi/bootstrap/ShutdownHook.java     |   93 ++
 .../exception/InvalidCommandException.java      |   38 +
 .../bootstrap/util/LimitingInputStream.java     |  107 ++
 .../src/main/resources/conf/logback.xml         |   30 +-
 pom.xml                                         |   42 +
 12 files changed, 1819 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-assembly/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-assembly/LICENSE b/minifi-assembly/LICENSE
index 7a4a3ea..9864230 100644
--- a/minifi-assembly/LICENSE
+++ b/minifi-assembly/LICENSE
@@ -199,4 +199,38 @@
    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.
\ No newline at end of file
+   limitations under the License.
+
+
+APACHE NIFI - MINIFI SUBCOMPONENTS:
+
+The Apache NiFi - MiNiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+
+  The binary distribution of this product bundles 'Slf4j' which is available
+  under a "3-clause BSD" license.  For details see http://www.slf4j.org/
+
+    Copyright (c) 2004-2013 QOS.ch
+     All rights reserved.
+
+     Permission is hereby granted, free  of charge, to any person obtaining
+     a  copy  of this  software  and  associated  documentation files  (the
+     "Software"), to  deal in  the Software without  restriction, including
+     without limitation  the rights to  use, copy, modify,  merge, publish,
+     distribute,  sublicense, and/or sell  copies of  the Software,  and to
+     permit persons to whom the Software  is furnished to do so, subject to
+     the following conditions:
+
+     The  above  copyright  notice  and  this permission  notice  shall  be
+     included in all copies or substantial portions of the Software.
+
+     THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+     EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+     MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+     NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+     LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+     OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+     WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/pom.xml b/minifi-assembly/pom.xml
index 2006c86..de4a58d 100644
--- a/minifi-assembly/pom.xml
+++ b/minifi-assembly/pom.xml
@@ -76,8 +76,41 @@ limitations under the License.
             <type>zip</type>
             <version>0.0.1-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
 
+
+    <properties>
+        <!--Bootstrap Properties -->
+        <minifi.jvm.heap.mb>512</minifi.jvm.heap.mb>
+        <minifi.jvm.permgen.mb>128</minifi.jvm.permgen.mb>
+        <minifi.run.as />
+    </properties>
+
     <profiles>
         <profile>
             <id>rpm</id>
@@ -203,6 +236,10 @@ limitations under the License.
                                             <dependency>
                                                 <includes>
                                                     <include>org.apache.nifi.minifi:minifi-bootstrap</include>
+                                                    <include>org.slf4j:slf4j-api</include>
+                                                    <include>org.yaml:snakeyaml</include>
+                                                    <include>ch.qos.logback:logback-classic</include>
+                                                    <include>ch.qos.logback:logback-core</include>
                                                 </includes>
                                             </dependency>
                                         </mapping>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-assembly/src/main/assembly/dependencies.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/src/main/assembly/dependencies.xml b/minifi-assembly/src/main/assembly/dependencies.xml
index 8cf91b8..4582a55 100644
--- a/minifi-assembly/src/main/assembly/dependencies.xml
+++ b/minifi-assembly/src/main/assembly/dependencies.xml
@@ -48,6 +48,8 @@
             <useTransitiveFiltering>true</useTransitiveFiltering>
             <includes>
             	<include>minifi-bootstrap</include>
+                <include>slf4j-api</include>
+                <include>logback-classic</include>
             </includes>
         </dependencySet>
         

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/pom.xml b/minifi-bootstrap/pom.xml
index 2e3e6ea..f7929c5 100644
--- a/minifi-bootstrap/pom.xml
+++ b/minifi-bootstrap/pom.xml
@@ -27,4 +27,22 @@ limitations under the License.
     <artifactId>minifi-bootstrap</artifactId>
     <packaging>jar</packaging>
 
+    <dependencies>
+        <dependency>
+            <groupId>org.yaml</groupId>
+            <artifactId>snakeyaml</artifactId>
+            <version>1.17</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>0.6.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${org.slf4j.version}</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
new file mode 100644
index 0000000..4138247
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.minifi.bootstrap;
+
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+
+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;
+
+public class BootstrapCodec {
+
+    private final RunMiNiFi runner;
+    private final BufferedReader reader;
+    private final BufferedWriter writer;
+
+    public BootstrapCodec(final RunMiNiFi runner, final InputStream in, final OutputStream out) {
+        this.runner = runner;
+        this.reader = new BufferedReader(new InputStreamReader(in));
+        this.writer = new BufferedWriter(new OutputStreamWriter(out));
+    }
+
+    public void communicate() throws IOException {
+        final String line = reader.readLine();
+        final String[] splits = line.split(" ");
+        if (splits.length < 0) {
+            throw new IOException("Received invalid command from MiNiFi: " + line);
+        }
+
+        final String cmd = splits[0];
+        final String[] args;
+        if (splits.length == 1) {
+            args = new String[0];
+        } else {
+            args = Arrays.copyOfRange(splits, 1, splits.length);
+        }
+
+        try {
+            processRequest(cmd, args);
+        } catch (final InvalidCommandException ice) {
+            throw new IOException("Received invalid command from MiNiFi: " + line + " : " + ice.getMessage() == null ? "" : "Details: " + ice.toString());
+        }
+    }
+
+    private void processRequest(final String cmd, final String[] args) throws InvalidCommandException, IOException {
+        switch (cmd) {
+            case "PORT": {
+                if (args.length != 2) {
+                    throw new InvalidCommandException();
+                }
+
+                final int port;
+                try {
+                    port = Integer.parseInt(args[0]);
+                } catch (final NumberFormatException nfe) {
+                    throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
+                }
+
+                if (port < 1 || port > 65535) {
+                    throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
+                }
+
+                final String secretKey = args[1];
+
+                runner.setNiFiCommandControlPort(port, secretKey);
+                writer.write("OK");
+                writer.newLine();
+                writer.flush();
+            }
+            break;
+            case "STARTED": {
+                if (args.length != 1) {
+                    throw new InvalidCommandException("STARTED command must contain a status argument");
+                }
+
+                if (!"true".equals(args[0]) && !"false".equals(args[0])) {
+                    throw new InvalidCommandException("Invalid status for STARTED command; should be true or false, but was '" + args[0] + "'");
+                }
+
+                final boolean started = Boolean.parseBoolean(args[0]);
+                runner.setNiFiStarted(started);
+                writer.write("OK");
+                writer.newLine();
+                writer.flush();
+            }
+            break;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiListener.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiListener.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiListener.java
new file mode 100644
index 0000000..ea760be
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiListener.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.minifi.bootstrap;
+
+import org.apache.nifi.minifi.bootstrap.util.LimitingInputStream;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+public class MiNiFiListener {
+
+    private ServerSocket serverSocket;
+    private volatile Listener listener;
+
+    int start(final RunMiNiFi runner) throws IOException {
+        serverSocket = new ServerSocket();
+        serverSocket.bind(new InetSocketAddress("localhost", 0));
+
+        final int localPort = serverSocket.getLocalPort();
+        listener = new Listener(serverSocket, runner);
+        final Thread listenThread = new Thread(listener);
+        listenThread.setName("Listen to MiNiFi");
+        listenThread.setDaemon(true);
+        listenThread.start();
+        return localPort;
+    }
+
+    public void stop() throws IOException {
+        final Listener listener = this.listener;
+        if (listener == null) {
+            return;
+        }
+
+        listener.stop();
+    }
+
+    private class Listener implements Runnable {
+
+        private final ServerSocket serverSocket;
+        private final ExecutorService executor;
+        private final RunMiNiFi runner;
+        private volatile boolean stopped = false;
+
+        public Listener(final ServerSocket serverSocket, final RunMiNiFi runner) {
+            this.serverSocket = serverSocket;
+            this.executor = Executors.newFixedThreadPool(2, new ThreadFactory() {
+                @Override
+                public Thread newThread(final Runnable runnable) {
+                    final Thread t = Executors.defaultThreadFactory().newThread(runnable);
+                    t.setDaemon(true);
+                    t.setName("MiNiFi Bootstrap Command Listener");
+                    return t;
+                }
+            });
+
+            this.runner = runner;
+        }
+
+        public void stop() throws IOException {
+            stopped = true;
+
+            executor.shutdown();
+            try {
+                executor.awaitTermination(3, TimeUnit.SECONDS);
+            } catch (final InterruptedException ie) {
+            }
+
+            serverSocket.close();
+        }
+
+        @Override
+        public void run() {
+            while (!serverSocket.isClosed()) {
+                try {
+                    if (stopped) {
+                        return;
+                    }
+
+                    final Socket socket;
+                    try {
+                        socket = serverSocket.accept();
+                    } catch (final IOException ioe) {
+                        if (stopped) {
+                            return;
+                        }
+
+                        throw ioe;
+                    }
+
+                    executor.submit(new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                // we want to ensure that we don't try to read data from an InputStream directly
+                                // by a BufferedReader because any user on the system could open a socket and send
+                                // a multi-gigabyte file without any new lines in order to crash the Bootstrap,
+                                // which in turn may cause the Shutdown Hook to shutdown MiNiFi.
+                                // So we will limit the amount of data to read to 4 KB
+                                final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096);
+                                final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream());
+                                codec.communicate();
+                            } catch (final Throwable t) {
+                                System.out.println("Failed to communicate with MiNiFi due to " + t);
+                                t.printStackTrace();
+                            } finally {
+                                try {
+                                    socket.close();
+                                } catch (final IOException ioe) {
+                                }
+                            }
+                        }
+                    });
+                } catch (final Throwable t) {
+                    System.err.println("Failed to receive information from MiNiFi due to " + t);
+                    t.printStackTrace();
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
new file mode 100644
index 0000000..f37b7f3
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
@@ -0,0 +1,1186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.minifi.bootstrap;
+
+import org.apache.nifi.util.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ *
+ * <p>
+ * The class which bootstraps Apache MiNiFi. This class looks for the
+ * bootstrap.conf file by looking in the following places (in order):</p>
+ * <ol>
+ * <li>Java System Property named
+ * {@code org.apache.nifi.minifi.bootstrap.config.file}</li>
+ * <li>${MINIFI_HOME}/./conf/bootstrap.conf, where ${MINIFI_HOME} references an
+ * environment variable {@code MINIFI_HOME}</li>
+ * <li>./conf/bootstrap.conf, where {@code ./} represents the working
+ * directory.</li>
+ * </ol>
+ *
+ * If the {@code bootstrap.conf} file cannot be found, throws a {@code FileNotFoundException}.
+ */
+public class RunMiNiFi {
+
+    public static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    public static final String DEFAULT_NIFI_PROPS_FILE = "./conf/nifi.properties";
+    public static final String DEFAULT_JAVA_CMD = "java";
+
+    public static final String GRACEFUL_SHUTDOWN_PROP = "graceful.shutdown.seconds";
+    public static final String DEFAULT_GRACEFUL_SHUTDOWN_VALUE = "20";
+
+    public static final String RUN_AS_PROP = "run.as";
+
+    public static final int MAX_RESTART_ATTEMPTS = 5;
+    public static final int STARTUP_WAIT_SECONDS = 60;
+
+    public static final String SHUTDOWN_CMD = "SHUTDOWN";
+    public static final String PING_CMD = "PING";
+    public static final String DUMP_CMD = "DUMP";
+
+    private volatile boolean autoRestartNiFi = true;
+    private volatile int ccPort = -1;
+    private volatile long nifiPid = -1L;
+    private volatile String secretKey;
+    private volatile ShutdownHook shutdownHook;
+    private volatile boolean nifiStarted;
+
+    private final Lock startedLock = new ReentrantLock();
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+
+    private final File bootstrapConfigFile;
+
+    // used for logging initial info; these will be logged to console by default when the app is started
+    private final Logger cmdLogger = LoggerFactory.getLogger("org.apache.nifi.minifi.bootstrap.Command");
+    // used for logging all info. These by default will be written to the log file
+    private final Logger defaultLogger = LoggerFactory.getLogger(RunMiNiFi.class);
+
+
+    private final ExecutorService loggingExecutor;
+    private volatile Set<Future<?>> loggingFutures = new HashSet<>(2);
+    private volatile int gracefulShutdownSeconds;
+
+    private final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
+
+    public RunMiNiFi(final File bootstrapConfigFile, final boolean verbose) throws IOException {
+        this.bootstrapConfigFile = bootstrapConfigFile;
+
+        loggingExecutor = Executors.newFixedThreadPool(2, new ThreadFactory() {
+            @Override
+            public Thread newThread(final Runnable runnable) {
+                final Thread t = Executors.defaultThreadFactory().newThread(runnable);
+                t.setDaemon(true);
+                t.setName("MiNiFi logging handler");
+                return t;
+            }
+        });
+    }
+
+    private static void printUsage() {
+        System.out.println("Usage:");
+        System.out.println();
+        System.out.println("java org.apache.nifi.minifi.bootstrap.RunMiNiFi [<-verbose>] <command> [options]");
+        System.out.println();
+        System.out.println("Valid commands include:");
+        System.out.println("");
+        System.out.println("Start : Start a new instance of Apache MiNiFi");
+        System.out.println("Stop : Stop a running instance of Apache MiNiFi");
+        System.out.println("Restart : Stop Apache MiNiFi, if it is running, and then start a new instance");
+        System.out.println("Status : Determine if there is a running instance of Apache MiNiFi");
+        System.out.println("Dump : Write a Thread Dump to the file specified by [options], or to the log if no file is given");
+        System.out.println("Run : Start a new instance of Apache MiNiFi and monitor the Process, restarting if the instance dies");
+        System.out.println();
+    }
+
+    private static String[] shift(final String[] orig) {
+        return Arrays.copyOfRange(orig, 1, orig.length);
+    }
+
+    public static void main(String[] args) throws IOException, InterruptedException {
+        if (args.length < 1 || args.length > 3) {
+            printUsage();
+            return;
+        }
+
+        File dumpFile = null;
+        boolean verbose = false;
+        if (args[0].equals("-verbose")) {
+            verbose = true;
+            args = shift(args);
+        }
+
+        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":
+                break;
+            default:
+                printUsage();
+                return;
+        }
+
+        final File configFile = getBootstrapConfFile();
+        final RunMiNiFi runMiNiFi = new RunMiNiFi(configFile, verbose);
+
+        switch (cmd.toLowerCase()) {
+            case "start":
+                runMiNiFi.start();
+                break;
+            case "run":
+                runMiNiFi.start();
+                break;
+            case "stop":
+                runMiNiFi.stop();
+                break;
+            case "status":
+                runMiNiFi.status();
+                break;
+            case "restart":
+                runMiNiFi.stop();
+                runMiNiFi.start();
+                break;
+            case "dump":
+                runMiNiFi.dump(dumpFile);
+                break;
+            case "env":
+                runMiNiFi.env();
+                break;
+        }
+    }
+
+    private 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;
+    }
+
+    File getStatusFile() {
+        return getStatusFile(defaultLogger);
+    }
+
+    public File getStatusFile(final Logger logger) {
+        final File confDir = bootstrapConfigFile.getParentFile();
+        final File nifiHome = confDir.getParentFile();
+        final File bin = new File(nifiHome, "bin");
+        final File statusFile = new File(bin, "minifi.pid");
+
+        logger.debug("Status File: {}", statusFile);
+
+        return statusFile;
+    }
+
+    public File getLockFile(final Logger logger) {
+        final File confDir = bootstrapConfigFile.getParentFile();
+        final File nifiHome = confDir.getParentFile();
+        final File bin = new File(nifiHome, "bin");
+        final File lockFile = new File(bin, "minifi.lock");
+
+        logger.debug("Lock File: {}", lockFile);
+        return lockFile;
+    }
+
+    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 nifiProps, final Logger logger) throws IOException {
+        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_READ);
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            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)) {
+            nifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        logger.debug("Saved Properties {} to {}", new Object[]{nifiProps, statusFile});
+    }
+
+    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");
+        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");
+        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) ? false : isProcessRunning(pid, logger);
+        return new Status(port, pid, pingSuccess, alive);
+    }
+
+    public void 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;
+        }
+
+        if (status.isProcessRunning()) {
+            logger.info("Apache MiNiFi is running at PID {} but is not responding to ping requests", status.getPid());
+            return;
+        }
+
+        if (status.getPort() == null) {
+            logger.info("Apache MiNiFi is not running");
+            return;
+        }
+
+        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");
+        }
+    }
+
+    public void env(){
+        final Logger logger = cmdLogger;
+        final Status status = getStatus(logger);
+        if (status.getPid() == null) {
+            logger.info("Apache MiNiFi is not running");
+            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;
+        }
+
+        final Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (final Throwable t) {
+            logger.error("Problem attaching to MiNiFi", t);
+            return;
+        }
+
+        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;
+        }
+
+        final Properties nifiProps = loadProperties(logger);
+        final String secretKey = nifiProps.getProperty("secret.key");
+
+        final StringBuilder sb = new StringBuilder();
+        try (final Socket socket = new Socket()) {
+            logger.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(60000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            logger.debug("Established connection to MiNiFi instance.");
+            socket.setSoTimeout(60000);
+
+            logger.debug("Sending DUMP Command to port {}", port);
+            final OutputStream out = socket.getOutputStream();
+            out.write((DUMP_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+
+            final InputStream in = socket.getInputStream();
+            try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    sb.append(line).append("\n");
+                }
+            }
+        }
+
+        final String dump = sb.toString();
+        if (dumpFile == null) {
+            logger.info(dump);
+        } else {
+            try (final FileOutputStream fos = new FileOutputStream(dumpFile)) {
+                fos.write(dump.getBytes(StandardCharsets.UTF_8));
+            }
+            // we want to log to the console (by default) that we wrote the thread dump to the specified file
+            cmdLogger.info("Successfully wrote thread dump to {}", dumpFile.getAbsolutePath());
+        }
+    }
+
+    public void stop() throws IOException {
+        final Logger logger = cmdLogger;
+        final Integer port = getCurrentPort(logger);
+        if (port == null) {
+            logger.info("Apache MiNiFi is not currently running");
+            return;
+        }
+
+        // indicate that a stop command is in progress
+        final File lockFile = getLockFile(logger);
+        if (!lockFile.exists()) {
+            lockFile.createNewFile();
+        }
+
+        final Properties nifiProps = loadProperties(logger);
+        final String secretKey = nifiProps.getProperty("secret.key");
+        final String pid = nifiProps.getProperty("pid");
+        final File statusFile = getStatusFile(logger);
+
+        try (final 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.");
+            socket.setSoTimeout(10000);
+
+            logger.debug("Sending SHUTDOWN Command to port {}", port);
+            final OutputStream out = socket.getOutputStream();
+            out.write((SHUTDOWN_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            final InputStream in = socket.getInputStream();
+            int lastChar;
+            final StringBuilder sb = new StringBuilder();
+            while ((lastChar = in.read()) > -1) {
+                sb.append((char) lastChar);
+            }
+            final String response = sb.toString().trim();
+
+            logger.debug("Received response to SHUTDOWN command: {}", response);
+
+            if (SHUTDOWN_CMD.equals(response)) {
+                logger.info("Apache MiNiFi has accepted the Shutdown Command and is shutting down now");
+
+                if (pid != null) {
+                    final Properties bootstrapProperties = new Properties();
+                    try (final FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+                        bootstrapProperties.load(fis);
+                    }
+
+                    String gracefulShutdown = bootstrapProperties.getProperty(GRACEFUL_SHUTDOWN_PROP, DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
+                    int gracefulShutdownSeconds;
+                    try {
+                        gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown);
+                    } catch (final NumberFormatException nfe) {
+                        gracefulShutdownSeconds = Integer.parseInt(DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
+                    }
+
+                    final long startWait = System.nanoTime();
+                    while (isProcessRunning(pid, logger)) {
+                        logger.info("Waiting for Apache MiNiFi to finish shutting down...");
+                        final long waitNanos = System.nanoTime() - startWait;
+                        final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
+                        if (waitSeconds >= gracefulShutdownSeconds && gracefulShutdownSeconds > 0) {
+                            if (isProcessRunning(pid, logger)) {
+                                logger.warn("MiNiFi has not finished shutting down after {} seconds. Killing process.", gracefulShutdownSeconds);
+                                try {
+                                    killProcessTree(pid, logger);
+                                } catch (final IOException ioe) {
+                                    logger.error("Failed to kill Process with PID {}", pid);
+                                }
+                            }
+                            break;
+                        } else {
+                            try {
+                                Thread.sleep(2000L);
+                            } catch (final InterruptedException ie) {
+                            }
+                        }
+                    }
+
+                    if (statusFile.exists() && !statusFile.delete()) {
+                        logger.error("Failed to delete status file {}; this file should be cleaned up manually", statusFile);
+                    }
+                    logger.info("MiNiFi has finished shutting down.");
+                }
+            } else {
+                logger.error("When sending SHUTDOWN command to MiNiFi, got unexpected response {}", response);
+            }
+        } catch (final IOException ioe) {
+            if (pid == null) {
+                logger.error("Failed to send shutdown command to port {} due to {}. No PID found for the MiNiFi process, so unable to kill process; "
+                    + "the process should be killed manually.", new Object[] {port, ioe.toString()});
+            } else {
+                logger.error("Failed to send shutdown command to port {} due to {}. Will kill the MiNiFi Process with PID {}.", new Object[] {port, ioe.toString(), pid});
+                killProcessTree(pid, logger);
+                if (statusFile.exists() && !statusFile.delete()) {
+                    logger.error("Failed to delete status file {}; this file should be cleaned up manually", statusFile);
+                }
+            }
+        } finally {
+            if (lockFile.exists() && !lockFile.delete()) {
+                logger.error("Failed to delete lock file {}; this file should be cleaned up manually", lockFile);
+            }
+        }
+    }
+
+    private static List<String> getChildProcesses(final String ppid) throws IOException {
+        final Process proc = Runtime.getRuntime().exec(new String[]{"ps", "-o", "pid", "--no-headers", "--ppid", ppid});
+        final List<String> childPids = new ArrayList<>();
+        try (final InputStream in = proc.getInputStream();
+            final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+
+            String line;
+            while ((line = reader.readLine()) != null) {
+                childPids.add(line.trim());
+            }
+        }
+
+        return childPids;
+    }
+
+    private void killProcessTree(final String pid, final Logger logger) throws IOException {
+        logger.debug("Killing Process Tree for PID {}", pid);
+
+        final List<String> children = getChildProcesses(pid);
+        logger.debug("Children of PID {}: {}", new Object[]{pid, children});
+
+        for (final String childPid : children) {
+            killProcessTree(childPid, logger);
+        }
+
+        Runtime.getRuntime().exec(new String[]{"kill", "-9", pid});
+    }
+
+    public static boolean isAlive(final Process process) {
+        try {
+            process.exitValue();
+            return false;
+        } catch (final IllegalStateException | IllegalThreadStateException itse) {
+            return true;
+        }
+    }
+
+    private String getHostname() {
+        String hostname = "Unknown Host";
+        String ip = "Unknown IP Address";
+        try {
+            final InetAddress localhost = InetAddress.getLocalHost();
+            hostname = localhost.getHostName();
+            ip = localhost.getHostAddress();
+        } catch (final Exception e) {
+            defaultLogger.warn("Failed to obtain hostname for notification due to:", e);
+        }
+
+        return hostname + " (" + ip + ")";
+    }
+
+    private int getGracefulShutdownSeconds(Map<String, String> props, File bootstrapConfigAbsoluteFile){
+        String gracefulShutdown = props.get(GRACEFUL_SHUTDOWN_PROP);
+        if (gracefulShutdown == null) {
+            gracefulShutdown = DEFAULT_GRACEFUL_SHUTDOWN_VALUE;
+        }
+
+        final int gracefulShutdownSeconds;
+        try {
+            gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown);
+        } catch (final NumberFormatException nfe) {
+            throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File "
+                    + bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer");
+        }
+
+        if (gracefulShutdownSeconds < 0) {
+            throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File "
+                    + bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer");
+        }
+        return gracefulShutdownSeconds;
+    }
+
+    private Map<String, String> readProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        final Properties properties = new Properties();
+        try (final FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            properties.load(fis);
+        }
+
+        final Map<String, String> props = new HashMap<>();
+        props.putAll((Map) properties);
+        return props;
+    }
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public Tuple<ProcessBuilder,Process> startMiNiFi() throws IOException, InterruptedException {
+        final Integer port = getCurrentPort(cmdLogger);
+        if (port != null) {
+            cmdLogger.info("Apache MiNiFi is already running, listening to Bootstrap on port " + port);
+            return null;
+        }
+
+        final File prevLockFile = getLockFile(cmdLogger);
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            cmdLogger.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        final ProcessBuilder builder = new ProcessBuilder();
+
+        final Map<String, String> props = readProperties();
+
+        final String specifiedWorkingDir = props.get("working.dir");
+        if (specifiedWorkingDir != null) {
+            builder.directory(new File(specifiedWorkingDir));
+        }
+
+        final File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        final File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+        final File workingDir = binDir.getParentFile();
+
+        if (specifiedWorkingDir == null) {
+            builder.directory(workingDir);
+        }
+
+        final String libFilename = replaceNull(props.get("lib.dir"), "./lib").trim();
+        File libDir = getFile(libFilename, workingDir);
+
+        final String confFilename = replaceNull(props.get("conf.dir"), "./conf").trim();
+        File confDir = getFile(confFilename, workingDir);
+
+        String nifiPropsFilename = props.get("props.file");
+        if (nifiPropsFilename == null) {
+            if (confDir.exists()) {
+                nifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                nifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        nifiPropsFilename = nifiPropsFilename.trim();
+
+        final List<String> javaAdditionalArgs = new ArrayList<>();
+        for (final Entry<String, String> entry : props.entrySet()) {
+            final String key = entry.getKey();
+            final String value = entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        final File[] libFiles = libDir.listFiles(new FilenameFilter() {
+            @Override
+            public boolean accept(final File dir, final String filename) {
+                return filename.toLowerCase().endsWith(".jar");
+            }
+        });
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        final File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        final List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (final File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        final StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            final String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        final String classPath = classPathBuilder.toString();
+        String javaCmd = props.get("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();
+                }
+            }
+        }
+
+        final MiNiFiListener listener = new MiNiFiListener();
+        final int listenPort = listener.start(this);
+
+        final List<String> cmd = new ArrayList<>();
+
+        cmd.add(javaCmd);
+        cmd.add("-classpath");
+        cmd.add(classPath);
+        cmd.addAll(javaAdditionalArgs);
+        cmd.add("-Dnifi.properties.file.path=" + nifiPropsFilename);
+        cmd.add("-Dnifi.bootstrap.listen.port=" + listenPort);
+        cmd.add("-Dapp=MiNiFi");
+        cmd.add("org.apache.nifi.minifi.MiNiFi");
+
+        builder.command(cmd);
+
+        final StringBuilder cmdBuilder = new StringBuilder();
+        for (final String s : cmd) {
+            cmdBuilder.append(s).append(" ");
+        }
+
+        cmdLogger.info("Starting Apache MiNiFi...");
+        cmdLogger.info("Working Directory: {}", workingDir.getAbsolutePath());
+        cmdLogger.info("Command: {}", cmdBuilder.toString());
+
+
+        Process process = builder.start();
+        handleLogging(process);
+        Long pid = getPid(process, cmdLogger);
+        if (pid != null) {
+            nifiPid = pid;
+            final Properties nifiProps = new Properties();
+            nifiProps.setProperty("pid", String.valueOf(nifiPid));
+            saveProperties(nifiProps, cmdLogger);
+        }
+
+        gracefulShutdownSeconds = getGracefulShutdownSeconds(props, bootstrapConfigAbsoluteFile);
+        shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
+        final Runtime runtime = Runtime.getRuntime();
+        runtime.addShutdownHook(shutdownHook);
+
+        final String hostname = getHostname();
+        String now = sdf.format(System.currentTimeMillis());
+        String user = System.getProperty("user.name");
+        if (user == null || user.trim().isEmpty()) {
+            user = "Unknown User";
+        }
+
+        return  new Tuple<ProcessBuilder,Process>(builder,process);
+    }
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void start() throws IOException, InterruptedException {
+
+        Tuple<ProcessBuilder,Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        while (true) {
+            final boolean alive = isAlive(process);
+
+            if (alive) {
+                try {
+                    Thread.sleep(1000L);
+                } catch (final InterruptedException ie) {
+                }
+            } else {
+                final Runtime runtime = Runtime.getRuntime();
+                try {
+                    runtime.removeShutdownHook(shutdownHook);
+                } catch (final IllegalStateException ise) {
+                    // happens when already shutting down
+                }
+
+                String now = sdf.format(System.currentTimeMillis());
+                if (autoRestartNiFi) {
+                    final File statusFile = getStatusFile(defaultLogger);
+                    if (!statusFile.exists()) {
+                        defaultLogger.info("Status File no longer exists. Will not restart MiNiFi");
+                        return;
+                    }
+
+                    final File  lockFile = getLockFile(defaultLogger);
+                    if (lockFile.exists()) {
+                        defaultLogger.info("A shutdown was initiated. Will not restart MiNiFi");
+                        return;
+                    }
+
+                    final boolean previouslyStarted = getNifiStarted();
+                    if (!previouslyStarted) {
+                        defaultLogger.info("MiNiFi never started. Will not restart MiNiFi");
+                        return;
+                    } else {
+                        setNiFiStarted(false);
+                    }
+
+                    defaultLogger.warn("Apache MiNiFi appears to have died. Restarting...");
+                    process = builder.start();
+                    handleLogging(process);
+
+                    Long pid = getPid(process, defaultLogger);
+                    if (pid != null) {
+                        nifiPid = pid;
+                        final Properties nifiProps = new Properties();
+                        nifiProps.setProperty("pid", String.valueOf(nifiPid));
+                        saveProperties(nifiProps, defaultLogger);
+                    }
+
+                    shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
+                    runtime.addShutdownHook(shutdownHook);
+
+                    final boolean started = waitForStart();
+
+                    final String hostname = getHostname();
+                    if (started) {
+                        defaultLogger.info("Successfully started Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+                    } else {
+                        defaultLogger.error("Apache MiNiFi does not appear to have started");
+                    }
+                } else {
+                    return;
+                }
+            }
+        }
+    }
+
+    private void handleLogging(final Process process) {
+        final Set<Future<?>> existingFutures = loggingFutures;
+        if (existingFutures != null) {
+            for (final Future<?> future : existingFutures) {
+                future.cancel(false);
+            }
+        }
+
+        final Future<?> stdOutFuture = loggingExecutor.submit(new Runnable() {
+            @Override
+            public void run() {
+                final Logger stdOutLogger = LoggerFactory.getLogger("org.apache.nifi.minifi.StdOut");
+                final InputStream in = process.getInputStream();
+                try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                    String line;
+                    while ((line = reader.readLine()) != null) {
+                        stdOutLogger.info(line);
+                    }
+                } catch (IOException e) {
+                    defaultLogger.error("Failed to read from MiNiFi's Standard Out stream", e);
+                }
+            }
+        });
+
+        final Future<?> stdErrFuture = loggingExecutor.submit(new Runnable() {
+            @Override
+            public void run() {
+                final Logger stdErrLogger = LoggerFactory.getLogger("org.apache.nifi.minifi.StdErr");
+                final InputStream in = process.getErrorStream();
+                try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                    String line;
+                    while ((line = reader.readLine()) != null) {
+                        stdErrLogger.error(line);
+                    }
+                } catch (IOException e) {
+                    defaultLogger.error("Failed to read from MiNiFi's Standard Error stream", e);
+                }
+            }
+        });
+
+        final Set<Future<?>> futures = new HashSet<>();
+        futures.add(stdOutFuture);
+        futures.add(stdErrFuture);
+        this.loggingFutures = futures;
+    }
+
+    private Long getPid(final Process process, final Logger logger) {
+        try {
+            final Class<?> procClass = process.getClass();
+            final Field pidField = procClass.getDeclaredField("pid");
+            pidField.setAccessible(true);
+            final Object pidObject = pidField.get(process);
+
+            logger.debug("PID Object = {}", pidObject);
+
+            if (pidObject instanceof Number) {
+                return ((Number) pidObject).longValue();
+            }
+            return null;
+        } catch (final IllegalAccessException | NoSuchFieldException nsfe) {
+            logger.debug("Could not find PID for child process due to {}", nsfe);
+            return null;
+        }
+    }
+
+    private boolean isWindows() {
+        final String osName = System.getProperty("os.name");
+        return osName != null && osName.toLowerCase().contains("win");
+    }
+
+    private boolean waitForStart() {
+        lock.lock();
+        try {
+            final long startTime = System.nanoTime();
+
+            while (ccPort < 1) {
+                try {
+                    startupCondition.await(1, TimeUnit.SECONDS);
+                } catch (final InterruptedException ie) {
+                    return false;
+                }
+
+                final long waitNanos = System.nanoTime() - startTime;
+                final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
+                if (waitSeconds > STARTUP_WAIT_SECONDS) {
+                    return false;
+                }
+            }
+        } finally {
+            lock.unlock();
+        }
+        return true;
+    }
+
+    private File getFile(final String filename, final File workingDir) {
+        File file = new File(filename);
+        if (!file.isAbsolute()) {
+            file = new File(workingDir, filename);
+        }
+
+        return file;
+    }
+
+    private String replaceNull(final String value, final String replacement) {
+        return (value == null) ? replacement : value;
+    }
+
+    void setAutoRestartNiFi(final boolean restart) {
+        this.autoRestartNiFi = restart;
+    }
+
+    void setNiFiCommandControlPort(final int port, final String secretKey) {
+        this.ccPort = port;
+        this.secretKey = secretKey;
+
+        if (shutdownHook != null) {
+            shutdownHook.setSecretKey(secretKey);
+        }
+
+        final File statusFile = getStatusFile(defaultLogger);
+
+        final Properties nifiProps = new Properties();
+        if (nifiPid != -1) {
+            nifiProps.setProperty("pid", String.valueOf(nifiPid));
+        }
+        nifiProps.setProperty("port", String.valueOf(ccPort));
+        nifiProps.setProperty("secret.key", secretKey);
+
+        try {
+            saveProperties(nifiProps, defaultLogger);
+        } catch (final IOException ioe) {
+            defaultLogger.warn("Apache MiNiFi has started but failed to persist MiNiFi Port information to {} due to {}", new Object[]{statusFile.getAbsolutePath(), ioe});
+        }
+
+        defaultLogger.info("Apache MiNiFi now running and listening for Bootstrap requests on port {}", port);
+    }
+
+    int getNiFiCommandControlPort() {
+        return this.ccPort;
+    }
+
+    void setNiFiStarted(final boolean nifiStarted) {
+        startedLock.lock();
+        try {
+            this.nifiStarted = nifiStarted;
+        } finally {
+            startedLock.unlock();
+        }
+    }
+
+    boolean getNifiStarted() {
+        startedLock.lock();
+        try {
+            return nifiStarted;
+        } finally {
+            startedLock.unlock();
+        }
+    }
+
+    private static class Status {
+
+        private final Integer port;
+        private final String pid;
+
+        private final Boolean respondingToPing;
+        private final Boolean processRunning;
+
+        public Status(final Integer port, final String pid, final Boolean respondingToPing, final Boolean processRunning) {
+            this.port = port;
+            this.pid = pid;
+            this.respondingToPing = respondingToPing;
+            this.processRunning = processRunning;
+        }
+
+        public String getPid() {
+            return pid;
+        }
+
+        public Integer getPort() {
+            return port;
+        }
+
+        public boolean isRespondingToPing() {
+            return Boolean.TRUE.equals(respondingToPing);
+        }
+
+        public boolean isProcessRunning() {
+            return Boolean.TRUE.equals(processRunning);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
new file mode 100644
index 0000000..3e3be5e
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.minifi.bootstrap;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ShutdownHook extends Thread {
+
+    private final Process nifiProcess;
+    private final RunMiNiFi runner;
+    private final int gracefulShutdownSeconds;
+    private final ExecutorService executor;
+
+    private volatile String secretKey;
+
+    public ShutdownHook(final Process nifiProcess, final RunMiNiFi runner, final String secretKey, final int gracefulShutdownSeconds, final ExecutorService executor) {
+        this.nifiProcess = nifiProcess;
+        this.runner = runner;
+        this.secretKey = secretKey;
+        this.gracefulShutdownSeconds = gracefulShutdownSeconds;
+        this.executor = executor;
+    }
+
+    void setSecretKey(final String secretKey) {
+        this.secretKey = secretKey;
+    }
+
+    @Override
+    public void run() {
+        executor.shutdown();
+        runner.setAutoRestartNiFi(false);
+        final int ccPort = runner.getNiFiCommandControlPort();
+        if (ccPort > 0) {
+            System.out.println("Initiating Shutdown of MiNiFi...");
+
+            try {
+                final Socket socket = new Socket("localhost", ccPort);
+                final OutputStream out = socket.getOutputStream();
+                out.write(("SHUTDOWN " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
+                out.flush();
+
+                socket.close();
+            } catch (final IOException ioe) {
+                System.out.println("Failed to Shutdown MiNiFi due to " + ioe);
+            }
+        }
+
+        System.out.println("Waiting for Apache MiNiFi to finish shutting down...");
+        final long startWait = System.nanoTime();
+        while (RunMiNiFi.isAlive(nifiProcess)) {
+            final long waitNanos = System.nanoTime() - startWait;
+            final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
+            if (waitSeconds >= gracefulShutdownSeconds && gracefulShutdownSeconds > 0) {
+                if (RunMiNiFi.isAlive(nifiProcess)) {
+                    System.out.println("MiNiFi has not finished shutting down after " + gracefulShutdownSeconds + " seconds. Killing process.");
+                    nifiProcess.destroy();
+                }
+                break;
+            } else {
+                try {
+                    Thread.sleep(1000L);
+                } catch (final InterruptedException ie) {
+                }
+            }
+        }
+
+        final File statusFile = runner.getStatusFile();
+        if (!statusFile.delete()) {
+            System.err.println("Failed to delete status file " + statusFile.getAbsolutePath() + "; this file should be cleaned up manually");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/exception/InvalidCommandException.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/exception/InvalidCommandException.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/exception/InvalidCommandException.java
new file mode 100644
index 0000000..e7ff276
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/exception/InvalidCommandException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.exception;
+
+public class InvalidCommandException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    public InvalidCommandException() {
+        super();
+    }
+
+    public InvalidCommandException(final String message) {
+        super(message);
+    }
+
+    public InvalidCommandException(final Throwable t) {
+        super(t);
+    }
+
+    public InvalidCommandException(final String message, final Throwable t) {
+        super(message, t);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/LimitingInputStream.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/LimitingInputStream.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/LimitingInputStream.java
new file mode 100644
index 0000000..9e8ce6d
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/LimitingInputStream.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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LimitingInputStream extends InputStream {
+
+    private final InputStream in;
+    private final long limit;
+    private long bytesRead = 0;
+
+    public LimitingInputStream(final InputStream in, final long limit) {
+        this.in = in;
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int val = in.read();
+        if (val > -1) {
+            bytesRead++;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
+
+        final int val = in.read(b, 0, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(len, limit - bytesRead);
+
+        final int val = in.read(b, off, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+        final long skipped = in.skip(Math.min(n, limit - bytesRead));
+        bytesRead += skipped;
+        return skipped;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return in.available();
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public void mark(int readlimit) {
+        in.mark(readlimit);
+    }
+
+    @Override
+    public boolean markSupported() {
+        return in.markSupported();
+    }
+
+    @Override
+    public void reset() throws IOException {
+        in.reset();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml b/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
index 8ea2809..e071858 100644
--- a/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
+++ b/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
@@ -20,7 +20,7 @@
     </contextListener>
     
     <appender name="APP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/nifi-app.log</file>
+        <file>logs/minifi-app.log</file>
         <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
             <!--
               For daily rollover, use 'app_%d.log'.
@@ -28,7 +28,7 @@
               To GZIP rolled files, replace '.log' with '.log.gz'.
               To ZIP rolled files, replace '.log' with '.log.zip'.
             -->
-            <fileNamePattern>./logs/nifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
+            <fileNamePattern>./logs/minifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
             <timeBasedFileNamingAndTriggeringPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedFNATP">
                 <maxFileSize>100MB</maxFileSize>
             </timeBasedFileNamingAndTriggeringPolicy>
@@ -42,7 +42,7 @@
     </appender>
     
     <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/nifi-user.log</file>
+        <file>logs/minifi-user.log</file>
         <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
             <!--
               For daily rollover, use 'user_%d.log'.
@@ -50,7 +50,7 @@
               To GZIP rolled files, replace '.log' with '.log.gz'.
               To ZIP rolled files, replace '.log' with '.log.zip'.
             -->
-            <fileNamePattern>./logs/nifi-user_%d.log</fileNamePattern>
+            <fileNamePattern>./logs/minifi-user_%d.log</fileNamePattern>
             <!-- keep 30 log files worth of history -->
             <maxHistory>30</maxHistory>
         </rollingPolicy>
@@ -60,7 +60,7 @@
     </appender>
 
     <appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/nifi-bootstrap.log</file>
+        <file>logs/minifi-bootstrap.log</file>
         <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
             <!--
               For daily rollover, use 'user_%d.log'.
@@ -68,7 +68,7 @@
               To GZIP rolled files, replace '.log' with '.log.gz'.
               To ZIP rolled files, replace '.log' with '.log.zip'.
             -->
-            <fileNamePattern>./logs/nifi-bootstrap_%d.log</fileNamePattern>
+            <fileNamePattern>./logs/minifi-bootstrap_%d.log</fileNamePattern>
             <!-- keep 5 log files worth of history -->
             <maxHistory>5</maxHistory>
         </rollingPolicy>
@@ -86,13 +86,10 @@
     <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
     
     <logger name="org.apache.nifi" level="INFO"/>
-    <logger name="org.apache.nifi.minifi.controller.repository.StandardProcessSession" level="WARN" />
-    
-    <!-- Logger for managing logging statements for nifi clusters. -->
-    <logger name="org.apache.nifi.minifi.cluster" level="INFO"/>
+    <logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
 
     <!-- Logger for logging HTTP requests received by the web server. -->
-    <logger name="org.apache.nifi.minifi.server.JettyServer" level="INFO"/>
+    <logger name="org.apache.nifi.server.JettyServer" level="INFO"/>
 
     <!-- Logger for managing logging statements for jetty -->
     <logger name="org.eclipse.jetty" level="INFO"/>
@@ -110,19 +107,16 @@
         log events to the root logger. These messages are only sent to the
         user-log appender.
     -->
-    <logger name="org.apache.nifi.minifi.web.security" level="INFO" additivity="false">
-        <appender-ref ref="USER_FILE"/>
-    </logger>
-    <logger name="org.apache.nifi.minifi.web.api.config" level="INFO" additivity="false">
+    <logger name="org.apache.nifi.web.security" level="INFO" additivity="false">
         <appender-ref ref="USER_FILE"/>
     </logger>
-    <logger name="org.apache.nifi.minifi.authorization" level="INFO" additivity="false">
+    <logger name="org.apache.nifi.web.api.config" level="INFO" additivity="false">
         <appender-ref ref="USER_FILE"/>
     </logger>
-    <logger name="org.apache.nifi.minifi.cluster.authorization" level="INFO" additivity="false">
+    <logger name="org.apache.nifi.authorization" level="INFO" additivity="false">
         <appender-ref ref="USER_FILE"/>
     </logger>
-    <logger name="org.apache.nifi.minifi.web.filter.RequestLogger" level="INFO" additivity="false">
+    <logger name="org.apache.nifi.web.filter.RequestLogger" level="INFO" additivity="false">
         <appender-ref ref="USER_FILE"/>
     </logger>
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/302ae7c6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 432407c..cc38cd6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,6 +92,7 @@ limitations under the License.
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <inceptionYear>2016</inceptionYear>
+        <org.slf4j.version>1.7.12</org.slf4j.version>
     </properties>
 
     <dependencyManagement>
@@ -116,6 +117,47 @@ limitations under the License.
                 <artifactId>minifi-resources</artifactId>
                 <version>0.0.1-SNAPSHOT</version>
             </dependency>
+            <dependency>
+                <groupId>ch.qos.logback</groupId>
+                <artifactId>logback-classic</artifactId>
+                <version>1.1.3</version>
+            </dependency>
+            <dependency>
+                <groupId>ch.qos.logback</groupId>
+                <artifactId>jcl-over-slf4j</artifactId>
+                <version>1.1.3</version>
+                <scope>provided</scope>
+                <exclusions>
+                    <exclusion>
+                        <groupId>org.slf4j</groupId>
+                        <artifactId>slf4j-api</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>jcl-over-slf4j</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>log4j-over-slf4j</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>jul-to-slf4j</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-api</artifactId>
+                <version>${org.slf4j.version}</version>
+                <scope>provided</scope>
+            </dependency>
         </dependencies>
     </dependencyManagement>