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/06 20:08:29 UTC

[1/3] nifi-minifi git commit: MINIFI-5: Creating a base MiNiFi project to serve as a basis for further extension and design reusing NiFi libraries

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


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/BootstrapListener.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/BootstrapListener.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/BootstrapListener.java
new file mode 100644
index 0000000..420cb49
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/BootstrapListener.java
@@ -0,0 +1,395 @@
+/*
+ * 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;
+
+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.lang.management.LockInfo;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MonitorInfo;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.nifi.util.LimitingInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapListener {
+
+    private static final Logger logger = LoggerFactory.getLogger(org.apache.nifi.BootstrapListener.class);
+
+    private final MiNiFi nifi;
+    private final int bootstrapPort;
+    private final String secretKey;
+
+    private volatile Listener listener;
+    private volatile ServerSocket serverSocket;
+
+    public BootstrapListener(final MiNiFi nifi, final int bootstrapPort) {
+        this.nifi = nifi;
+        this.bootstrapPort = bootstrapPort;
+        secretKey = UUID.randomUUID().toString();
+    }
+
+    public void start() throws IOException {
+        logger.debug("Starting Bootstrap Listener to communicate with Bootstrap Port {}", bootstrapPort);
+
+        serverSocket = new ServerSocket();
+        serverSocket.bind(new InetSocketAddress("localhost", 0));
+        serverSocket.setSoTimeout(2000);
+
+        final int localPort = serverSocket.getLocalPort();
+        logger.info("Started Bootstrap Listener, Listening for incoming requests on port {}", localPort);
+
+        listener = new Listener(serverSocket);
+        final Thread listenThread = new Thread(listener);
+        listenThread.setDaemon(true);
+        listenThread.setName("Listen to Bootstrap");
+        listenThread.start();
+
+        logger.debug("Notifying Bootstrap that local port is {}", localPort);
+        sendCommand("PORT", new String[] { String.valueOf(localPort), secretKey});
+    }
+
+    public void stop() {
+        if (listener != null) {
+            listener.stop();
+        }
+    }
+
+    public void sendStartedStatus(boolean status) throws IOException {
+        logger.debug("Notifying Bootstrap that the status of starting MiNiFi is {}", status);
+        sendCommand("STARTED", new String[]{ String.valueOf(status) });
+    }
+
+    private void sendCommand(final String command, final String[] args) throws IOException {
+        try (final Socket socket = new Socket()) {
+            socket.setSoTimeout(60000);
+            socket.connect(new InetSocketAddress("localhost", bootstrapPort));
+            socket.setSoTimeout(60000);
+
+            final StringBuilder commandBuilder = new StringBuilder(command);
+            for (final String arg : args) {
+                commandBuilder.append(" ").append(arg);
+            }
+            commandBuilder.append("\n");
+
+            final String commandWithArgs = commandBuilder.toString();
+            logger.debug("Sending command to Bootstrap: " + commandWithArgs);
+
+            final OutputStream out = socket.getOutputStream();
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            out.flush();
+
+            logger.debug("Awaiting response from Bootstrap...");
+            final BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));
+            final String response = reader.readLine();
+            if ("OK".equals(response)) {
+                logger.info("Successfully initiated communication with Bootstrap");
+            } else {
+                logger.error("Failed to communicate with Bootstrap. Bootstrap may be unable to issue or receive commands from MiNiFi");
+            }
+        }
+    }
+
+    private class Listener implements Runnable {
+
+        private final ServerSocket serverSocket;
+        private final ExecutorService executor;
+        private volatile boolean stopped = false;
+
+        public Listener(final ServerSocket serverSocket) {
+            this.serverSocket = serverSocket;
+            this.executor = Executors.newFixedThreadPool(2);
+        }
+
+        public void stop() {
+            stopped = true;
+
+            executor.shutdownNow();
+
+            try {
+                serverSocket.close();
+            } catch (final IOException ioe) {
+                // nothing to really do here. we could log this, but it would just become
+                // confusing in the logs, as we're shutting down and there's no real benefit
+            }
+        }
+
+        @Override
+        public void run() {
+            while (!stopped) {
+                try {
+                    final Socket socket;
+                    try {
+                        logger.debug("Listening for Bootstrap Requests");
+                        socket = serverSocket.accept();
+                    } catch (final SocketTimeoutException ste) {
+                        if (stopped) {
+                            return;
+                        }
+
+                        continue;
+                    } catch (final IOException ioe) {
+                        if (stopped) {
+                            return;
+                        }
+
+                        throw ioe;
+                    }
+
+                    logger.debug("Received connection from Bootstrap");
+                    socket.setSoTimeout(5000);
+
+                    executor.submit(new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                final BootstrapRequest request = readRequest(socket.getInputStream());
+                                final BootstrapRequest.RequestType requestType = request.getRequestType();
+
+                                switch (requestType) {
+                                    case PING:
+                                        logger.debug("Received PING request from Bootstrap; responding");
+                                        echoPing(socket.getOutputStream());
+                                        logger.debug("Responded to PING request from Bootstrap");
+                                        break;
+                                    case SHUTDOWN:
+                                        logger.info("Received SHUTDOWN request from Bootstrap");
+                                        echoShutdown(socket.getOutputStream());
+                                        nifi.shutdownHook();
+                                        return;
+                                    case DUMP:
+                                        logger.info("Received DUMP request from Bootstrap");
+                                        writeDump(socket.getOutputStream());
+                                        break;
+                                }
+                            } catch (final Throwable t) {
+                                logger.error("Failed to process request from Bootstrap due to " + t.toString(), t);
+                            } finally {
+                                try {
+                                    socket.close();
+                                } catch (final IOException ioe) {
+                                    logger.warn("Failed to close socket to Bootstrap due to {}", ioe.toString());
+                                }
+                            }
+                        }
+                    });
+                } catch (final Throwable t) {
+                    logger.error("Failed to process request from Bootstrap due to " + t.toString(), t);
+                }
+            }
+        }
+    }
+
+    private static void writeDump(final OutputStream out) throws IOException {
+        final ThreadMXBean mbean = ManagementFactory.getThreadMXBean();
+        final BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+
+        final ThreadInfo[] infos = mbean.dumpAllThreads(true, true);
+        final long[] deadlockedThreadIds = mbean.findDeadlockedThreads();
+        final long[] monitorDeadlockThreadIds = mbean.findMonitorDeadlockedThreads();
+
+        final List<ThreadInfo> sortedInfos = new ArrayList<>(infos.length);
+        for (final ThreadInfo info : infos) {
+            sortedInfos.add(info);
+        }
+        Collections.sort(sortedInfos, new Comparator<ThreadInfo>() {
+            @Override
+            public int compare(ThreadInfo o1, ThreadInfo o2) {
+                return o1.getThreadName().toLowerCase().compareTo(o2.getThreadName().toLowerCase());
+            }
+        });
+
+        final StringBuilder sb = new StringBuilder();
+        for (final ThreadInfo info : sortedInfos) {
+            sb.append("\n");
+            sb.append("\"").append(info.getThreadName()).append("\" Id=");
+            sb.append(info.getThreadId()).append(" ");
+            sb.append(info.getThreadState().toString()).append(" ");
+
+            switch (info.getThreadState()) {
+                case BLOCKED:
+                case TIMED_WAITING:
+                case WAITING:
+                    sb.append(" on ");
+                    sb.append(info.getLockInfo());
+                    break;
+                default:
+                    break;
+            }
+
+            if (info.isSuspended()) {
+                sb.append(" (suspended)");
+            }
+            if (info.isInNative()) {
+                sb.append(" (in native code)");
+            }
+
+            if (deadlockedThreadIds != null && deadlockedThreadIds.length > 0) {
+                for (final long id : deadlockedThreadIds) {
+                    if (id == info.getThreadId()) {
+                        sb.append(" ** DEADLOCKED THREAD **");
+                    }
+                }
+            }
+
+            if (monitorDeadlockThreadIds != null && monitorDeadlockThreadIds.length > 0) {
+                for (final long id : monitorDeadlockThreadIds) {
+                    if (id == info.getThreadId()) {
+                        sb.append(" ** MONITOR-DEADLOCKED THREAD **");
+                    }
+                }
+            }
+
+            final StackTraceElement[] stackTraces = info.getStackTrace();
+            for (final StackTraceElement element : stackTraces) {
+                sb.append("\n\tat ").append(element);
+
+                final MonitorInfo[] monitors = info.getLockedMonitors();
+                for (final MonitorInfo monitor : monitors) {
+                    if (monitor.getLockedStackFrame().equals(element)) {
+                        sb.append("\n\t- waiting on ").append(monitor);
+                    }
+                }
+            }
+
+            final LockInfo[] lockInfos = info.getLockedSynchronizers();
+            if (lockInfos.length > 0) {
+                sb.append("\n\t");
+                sb.append("Number of Locked Synchronizers: ").append(lockInfos.length);
+                for (final LockInfo lockInfo : lockInfos) {
+                    sb.append("\n\t- ").append(lockInfo.toString());
+                }
+            }
+
+            sb.append("\n");
+        }
+
+        if (deadlockedThreadIds != null && deadlockedThreadIds.length > 0) {
+            sb.append("\n\nDEADLOCK DETECTED!");
+            sb.append("\nThe following thread IDs are deadlocked:");
+            for (final long id : deadlockedThreadIds) {
+                sb.append("\n").append(id);
+            }
+        }
+
+        if (monitorDeadlockThreadIds != null && monitorDeadlockThreadIds.length > 0) {
+            sb.append("\n\nMONITOR DEADLOCK DETECTED!");
+            sb.append("\nThe following thread IDs are deadlocked:");
+            for (final long id : monitorDeadlockThreadIds) {
+                sb.append("\n").append(id);
+            }
+        }
+
+        writer.write(sb.toString());
+        writer.flush();
+    }
+
+    private void echoPing(final OutputStream out) throws IOException {
+        out.write("PING\n".getBytes(StandardCharsets.UTF_8));
+        out.flush();
+    }
+
+    private void echoShutdown(final OutputStream out) throws IOException {
+        out.write("SHUTDOWN\n".getBytes(StandardCharsets.UTF_8));
+        out.flush();
+    }
+
+    @SuppressWarnings("resource")  // we don't want to close the stream, as the caller will do that
+    private BootstrapRequest readRequest(final InputStream in) throws IOException {
+        // 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 NiFi instance
+        // (or at least cause OutOfMemoryErrors, which can wreak havoc on the running instance).
+        // So we will limit the Input Stream to only 4 KB, which should be plenty for any request.
+        final LimitingInputStream limitingIn = new LimitingInputStream(in, 4096);
+        final BufferedReader reader = new BufferedReader(new InputStreamReader(limitingIn));
+
+        final String line = reader.readLine();
+        final String[] splits = line.split(" ");
+        if (splits.length < 1) {
+            throw new IOException("Received invalid request from Bootstrap: " + line);
+        }
+
+        final String requestType = splits[0];
+        final String[] args;
+        if (splits.length == 1) {
+            throw new IOException("Received invalid request from Bootstrap; request did not have a secret key; request type = " + requestType);
+        } else if (splits.length == 2) {
+            args = new String[0];
+        } else {
+            args = Arrays.copyOfRange(splits, 2, splits.length);
+        }
+
+        final String requestKey = splits[1];
+        if (!secretKey.equals(requestKey)) {
+            throw new IOException("Received invalid Secret Key for request type " + requestType);
+        }
+
+        try {
+            return new BootstrapRequest(requestType, args);
+        } catch (final Exception e) {
+            throw new IOException("Received invalid request from Bootstrap; request type = " + requestType);
+        }
+    }
+
+    private static class BootstrapRequest {
+
+        public static enum RequestType {
+
+            SHUTDOWN,
+            DUMP,
+            PING;
+        }
+
+        private final RequestType requestType;
+        private final String[] args;
+
+        public BootstrapRequest(final String request, final String[] args) {
+            this.requestType = RequestType.valueOf(request);
+            this.args = args;
+        }
+
+        public RequestType getRequestType() {
+            return requestType;
+        }
+
+        @SuppressWarnings("unused")
+        public String[] getArgs() {
+            return args;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
new file mode 100644
index 0000000..1a1c0fe
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.NiFiServer;
+import org.apache.nifi.documentation.DocGenerator;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionMapping;
+import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.util.FileUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+public class MiNiFi {
+
+    private static final Logger logger = LoggerFactory.getLogger(MiNiFi.class);
+    private final NiFiServer nifiServer;
+    private final BootstrapListener bootstrapListener;
+
+    public static final String BOOTSTRAP_PORT_PROPERTY = "nifi.bootstrap.listen.port";
+    private volatile boolean shutdown = false;
+
+    public MiNiFi(final NiFiProperties properties)
+        throws ClassNotFoundException, IOException, NoSuchMethodException, InstantiationException, IllegalAccessException, IllegalArgumentException, InvocationTargetException {
+        Thread.setDefaultUncaughtExceptionHandler(new UncaughtExceptionHandler() {
+            @Override
+            public void uncaughtException(final Thread t, final Throwable e) {
+                logger.error("An Unknown Error Occurred in Thread {}: {}", t, e.toString());
+                logger.error("", e);
+            }
+        });
+
+        // register the shutdown hook
+        Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
+            @Override
+            public void run() {
+                // shutdown the jetty server
+                shutdownHook();
+            }
+        }));
+
+        final String bootstrapPort = System.getProperty(BOOTSTRAP_PORT_PROPERTY);
+        if (bootstrapPort != null) {
+            try {
+                final int port = Integer.parseInt(bootstrapPort);
+
+                if (port < 1 || port > 65535) {
+                    throw new RuntimeException("Failed to start MiNiFi because system property '" + BOOTSTRAP_PORT_PROPERTY + "' is not a valid integer in the range 1 - 65535");
+                }
+
+                bootstrapListener = new BootstrapListener(this, port);
+                bootstrapListener.start();
+            } catch (final NumberFormatException nfe) {
+                throw new RuntimeException("Failed to start MiNiFi because system property '" + BOOTSTRAP_PORT_PROPERTY + "' is not a valid integer in the range 1 - 65535");
+            }
+        } else {
+            logger.info("MiNiFi started without Bootstrap Port information provided; will not listen for requests from Bootstrap");
+            bootstrapListener = null;
+        }
+
+        // delete the web working dir - if the application does not start successfully
+        // the web app directories might be in an invalid state. when this happens
+        // jetty will not attempt to re-extract the war into the directory. by removing
+        // the working directory, we can be assured that it will attempt to extract the
+        // war every time the application starts.
+        File webWorkingDir = properties.getWebWorkingDirectory();
+        FileUtils.deleteFilesInDirectory(webWorkingDir, null, logger, true, true);
+        FileUtils.deleteFile(webWorkingDir, logger, 3);
+
+        detectTimingIssues();
+
+        // redirect JUL log events
+        SLF4JBridgeHandler.removeHandlersForRootLogger();
+        SLF4JBridgeHandler.install();
+
+        // expand the nars
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
+
+        // load the extensions classloaders
+        NarClassLoaders.load(properties);
+
+        // load the framework classloader
+        final ClassLoader frameworkClassLoader = NarClassLoaders.getFrameworkClassLoader();
+        if (frameworkClassLoader == null) {
+            throw new IllegalStateException("Unable to find the framework NAR ClassLoader.");
+        }
+
+        // discover the extensions
+        ExtensionManager.discoverExtensions();
+        ExtensionManager.logClassLoaderMapping();
+
+        DocGenerator.generate(properties);
+
+        // load the server from the framework classloader
+        Thread.currentThread().setContextClassLoader(frameworkClassLoader);
+        Class<?> jettyServer = Class.forName("org.apache.nifi.web.server.JettyServer", true, frameworkClassLoader);
+        Constructor<?> jettyConstructor = jettyServer.getConstructor(NiFiProperties.class);
+
+        final long startTime = System.nanoTime();
+        nifiServer = (NiFiServer) jettyConstructor.newInstance(properties);
+        nifiServer.setExtensionMapping(extensionMapping);
+
+        if (shutdown) {
+            logger.info("MiNiFi has been shutdown via MiNiFi Bootstrap. Will not start Controller");
+        } else {
+            nifiServer.start();
+
+            if (bootstrapListener != null) {
+                bootstrapListener.sendStartedStatus(true);
+            }
+
+            final long endTime = System.nanoTime();
+            logger.info("Controller initialization took " + (endTime - startTime) + " nanoseconds.");
+        }
+    }
+
+    protected void shutdownHook() {
+        try {
+            this.shutdown = true;
+
+            logger.info("Initiating shutdown of Jetty web server...");
+            if (nifiServer != null) {
+                nifiServer.stop();
+            }
+            if (bootstrapListener != null) {
+                bootstrapListener.stop();
+            }
+            logger.info("Jetty web server shutdown completed (nicely or otherwise).");
+        } catch (final Throwable t) {
+            logger.warn("Problem occured ensuring Jetty web server was properly terminated due to " + t);
+        }
+    }
+
+    /**
+     * Determine if the machine we're running on has timing issues.
+     */
+    private void detectTimingIssues() {
+        final int minRequiredOccurrences = 25;
+        final int maxOccurrencesOutOfRange = 15;
+        final AtomicLong lastTriggerMillis = new AtomicLong(System.currentTimeMillis());
+
+        final ScheduledExecutorService service = Executors.newScheduledThreadPool(1, new ThreadFactory() {
+            private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+
+            @Override
+            public Thread newThread(final Runnable r) {
+                final Thread t = defaultFactory.newThread(r);
+                t.setDaemon(true);
+                t.setName("Detect Timing Issues");
+                return t;
+            }
+        });
+
+        final AtomicInteger occurrencesOutOfRange = new AtomicInteger(0);
+        final AtomicInteger occurences = new AtomicInteger(0);
+        final Runnable command = new Runnable() {
+            @Override
+            public void run() {
+                final long curMillis = System.currentTimeMillis();
+                final long difference = curMillis - lastTriggerMillis.get();
+                final long millisOff = Math.abs(difference - 2000L);
+                occurences.incrementAndGet();
+                if (millisOff > 500L) {
+                    occurrencesOutOfRange.incrementAndGet();
+                }
+                lastTriggerMillis.set(curMillis);
+            }
+        };
+
+        final ScheduledFuture<?> future = service.scheduleWithFixedDelay(command, 2000L, 2000L, TimeUnit.MILLISECONDS);
+
+        final TimerTask timerTask = new TimerTask() {
+            @Override
+            public void run() {
+                future.cancel(true);
+                service.shutdownNow();
+
+                if (occurences.get() < minRequiredOccurrences || occurrencesOutOfRange.get() > maxOccurrencesOutOfRange) {
+                    logger.warn("MiNiFi has detected that this box is not responding within the expected timing interval, which may cause "
+                        + "Processors to be scheduled erratically. Please see the MiNiFi documentation for more information.");
+                }
+            }
+        };
+        final Timer timer = new Timer(true);
+        timer.schedule(timerTask, 60000L);
+    }
+
+    /**
+     * Main entry point of the application.
+     *
+     * @param args things which are ignored
+     */
+    public static void main(String[] args) {
+        logger.info("Launching MiNiFi...");
+        try {
+            NiFiProperties niFiProperties = NiFiProperties.getInstance();
+            new MiNiFi(niFiProperties);
+        } catch (final Throwable t) {
+            logger.error("Failure to launch MiNiFi due to " + t, t);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/pom.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/pom.xml
new file mode 100644
index 0000000..5de51c6
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/pom.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>minifi-framework-bundle</artifactId>
+        <groupId>org.apache.nifi.minifi</groupId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <artifactId>minifi-framework</artifactId>
+    <packaging>pom</packaging>
+    <modules>
+        <module>minifi-runtime</module>
+        <module>minifi-resources</module>
+    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/pom.xml b/minifi-nar-bundles/minifi-framework-bundle/pom.xml
new file mode 100644
index 0000000..7fbd6b7
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/pom.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>minifi-nar-bundles</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.nifi.minifi</groupId>
+    <artifactId>minifi-framework-bundle</artifactId>
+    <packaging>pom</packaging>
+    <version>0.0.1-SNAPSHOT</version>
+
+    <modules>
+        <module>minifi-framework-nar</module>
+        <module>minifi-framework</module>
+    </modules>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/pom.xml b/minifi-nar-bundles/pom.xml
new file mode 100644
index 0000000..aeaad02
--- /dev/null
+++ b/minifi-nar-bundles/pom.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+--><project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi.minifi</groupId>
+        <artifactId>minifi</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>minifi-nar-bundles</artifactId>
+    <packaging>pom</packaging>
+    <modules>
+        <module>minifi-framework-bundle</module>
+    </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cc38cd6..e4967f9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -35,7 +35,7 @@ limitations under the License.
         <module>minifi-commons</module>
         <module>minifi-api</module>
         <module>minifi-bootstrap</module>
-        <module>minifi-framework</module>
+        <module>minifi-nar-bundles</module>
         <module>minifi-assembly</module>
     </modules>
 
@@ -93,6 +93,7 @@ limitations under the License.
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <inceptionYear>2016</inceptionYear>
         <org.slf4j.version>1.7.12</org.slf4j.version>
+        <org.apache.nifi.version>0.6.0</org.apache.nifi.version>
     </properties>
 
     <dependencyManagement>
@@ -158,12 +159,177 @@ limitations under the License.
                 <version>${org.slf4j.version}</version>
                 <scope>provided</scope>
             </dependency>
+
+            <!-- NiFi Modules -->
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-api</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-site-to-site-client</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-web-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-expression-language</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-custom-ui-utilities</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-ui-extension</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-flowfile-packager</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-socket-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-data-provenance-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-runtime</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-resources</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <classifier>resources</classifier>
+                <scope>runtime</scope>
+                <type>zip</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-docs</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <classifier>resources</classifier>
+                <scope>runtime</scope>
+                <type>zip</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-framework-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-provenance-repository-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-standard-services-api-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-ssl-context-service-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-standard-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-jetty-bundle</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-update-attribute-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-ldap-iaa-providers-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-kerberos-iaa-providers-nar</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <type>nar</type>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-properties</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-security-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-logging-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-nar-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-processor-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-hadoop-utils</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-mock</artifactId>
+                <version>${org.apache.nifi.version}</version>
+                <scope>test</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-write-ahead-log</artifactId>
+                <version>${org.apache.nifi.version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 
-    <dependencies>
-    </dependencies>
-
     <build>
         <pluginManagement>
             <plugins>


[2/3] nifi-minifi git commit: MINIFI-5: Creating a base MiNiFi project to serve as a basis for further extension and design reusing NiFi libraries

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/LICENSE b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/NOTICE b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/NOTICE
new file mode 100644
index 0000000..4885e58
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/NOTICE
@@ -0,0 +1,5 @@
+minifi-resources
+Copyright 2014-2016 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/pom.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/pom.xml
new file mode 100644
index 0000000..44958a7
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/pom.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi.minifi</groupId>
+        <artifactId>minifi-framework</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <artifactId>minifi-resources</artifactId>
+    <packaging>pom</packaging>
+    <description>holds common resources used to build installers</description>
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <attach>true</attach>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make shared resource</id>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <phase>package</phase>
+                        <configuration>
+                            <descriptors>
+                                <descriptor>src/main/assembly/dependencies.xml</descriptor>
+                            </descriptors>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml
new file mode 100644
index 0000000..7c4188d
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0"?>
+<!--
+  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.
+-->
+<assembly>
+    <id>resources</id>
+    <formats>
+        <format>zip</format>
+    </formats>
+    <includeBaseDirectory>false</includeBaseDirectory>
+    <fileSets>
+        <fileSet>
+            <directory>src/main/resources</directory>
+            <outputDirectory>/</outputDirectory>
+        </fileSet>
+        <fileSet>
+            <directory>src/main/resources/bin</directory>
+            <outputDirectory>/bin/</outputDirectory>
+            <includes>
+                <include>minifi.sh</include>
+            </includes>
+            <fileMode>0750</fileMode>
+        </fileSet>
+    </fileSets>
+    <files>
+        <file>
+            <source>./LICENSE</source>
+            <outputDirectory>./</outputDirectory>
+            <destName>LICENSE</destName>
+            <fileMode>0644</fileMode>
+            <filtered>true</filtered>
+        </file>       
+        <file>
+            <source>./NOTICE</source>
+            <outputDirectory>./</outputDirectory>
+            <destName>NOTICE</destName>
+            <fileMode>0644</fileMode>
+            <filtered>true</filtered>
+        </file>
+    </files>
+</assembly>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat
new file mode 100644
index 0000000..d55fcfd
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat
@@ -0,0 +1,47 @@
+@echo off
+rem
+rem    Licensed to the Apache Software Foundation (ASF) under one or more
+rem    contributor license agreements.  See the NOTICE file distributed with
+rem    this work for additional information regarding copyright ownership.
+rem    The ASF licenses this file to You under the Apache License, Version 2.0
+rem    (the "License"); you may not use this file except in compliance with
+rem    the License.  You may obtain a copy of the License at
+rem
+rem       http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem    Unless required by applicable law or agreed to in writing, software
+rem    distributed under the License is distributed on an "AS IS" BASIS,
+rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem    See the License for the specific language governing permissions and
+rem    limitations under the License.
+rem
+
+rem Use JAVA_HOME if it's set; otherwise, just use java
+
+if "%JAVA_HOME%" == "" goto noJavaHome
+if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
+set JAVA_EXE=%JAVA_HOME%\bin\java.exe
+goto startMiNifi
+
+:noJavaHome
+echo The JAVA_HOME environment variable is not defined correctly.
+echo Instead the PATH will be used to find the java executable.
+echo.
+set JAVA_EXE=java
+goto startMiNifi
+
+:startMiNifi
+set MINIFI_ROOT=%~dp0..\
+pushd "%MINIFI_ROOT%"
+set LIB_DIR=lib\bootstrap
+set CONF_DIR=conf
+
+set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
+set JAVA_ARGS=-Dorg.apache.nifi.minifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
+
+SET JAVA_PARAMS=-cp %CONF_DIR%;%LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.minifi.bootstrap.RunMiNiFi
+set BOOTSTRAP_ACTION=dump
+
+cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
+
+popd

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh
new file mode 100755
index 0000000..51bf67e
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh
@@ -0,0 +1,247 @@
+#!/bin/sh
+#
+#    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.
+#
+# chkconfig: 2345 20 80
+# description: Apache NiFi - MiNiFi
+#
+
+# Script structure inspired from Apache Karaf and other Apache projects with similar startup approaches
+
+SCRIPT_DIR=$(dirname "$0")
+SCRIPT_NAME=$(basename "$0")
+MINIFI_HOME=$(cd "${SCRIPT_DIR}" && cd .. && pwd)
+PROGNAME=$(basename "$0")
+
+
+warn() {
+    echo "${PROGNAME}: $*"
+}
+
+die() {
+    warn "$*"
+    exit 1
+}
+
+detectOS() {
+    # OS specific support (must be 'true' or 'false').
+    cygwin=false;
+    aix=false;
+    os400=false;
+    darwin=false;
+    case "$(uname)" in
+        CYGWIN*)
+            cygwin=true
+            ;;
+        AIX*)
+            aix=true
+            ;;
+        OS400*)
+            os400=true
+            ;;
+        Darwin)
+            darwin=true
+            ;;
+    esac
+    # For AIX, set an environment variable
+    if ${aix}; then
+         export LDR_CNTRL=MAXDATA=0xB0000000@DSA
+         echo ${LDR_CNTRL}
+    fi
+}
+
+unlimitFD() {
+    # Use the maximum available, or set MAX_FD != -1 to use that
+    if [ "x${MAX_FD}" = "x" ]; then
+        MAX_FD="maximum"
+    fi
+
+    # Increase the maximum file descriptors if we can
+    if [ "${os400}" = "false" ] && [ "${cygwin}" = "false" ]; then
+        MAX_FD_LIMIT=$(ulimit -H -n)
+        if [ "${MAX_FD_LIMIT}" != 'unlimited' ]; then
+            if [ $? -eq 0 ]; then
+                if [ "${MAX_FD}" = "maximum" -o "${MAX_FD}" = "max" ]; then
+                    # use the system max
+                    MAX_FD="${MAX_FD_LIMIT}"
+                fi
+
+                ulimit -n ${MAX_FD} > /dev/null
+                # echo "ulimit -n" `ulimit -n`
+                if [ $? -ne 0 ]; then
+                    warn "Could not set maximum file descriptor limit: ${MAX_FD}"
+                fi
+            else
+                warn "Could not query system maximum file descriptor limit: ${MAX_FD_LIMIT}"
+            fi
+        fi
+    fi
+}
+
+
+
+locateJava() {
+    # Setup the Java Virtual Machine
+    if $cygwin ; then
+        [ -n "${JAVA}" ] && JAVA=$(cygpath --unix "${JAVA}")
+        [ -n "${JAVA_HOME}" ] && JAVA_HOME=$(cygpath --unix "${JAVA_HOME}")
+    fi
+
+    if [ "x${JAVA}" = "x" ] && [ -r /etc/gentoo-release ] ; then
+        JAVA_HOME=$(java-config --jre-home)
+    fi
+    if [ "x${JAVA}" = "x" ]; then
+        if [ "x${JAVA_HOME}" != "x" ]; then
+            if [ ! -d "${JAVA_HOME}" ]; then
+                die "JAVA_HOME is not valid: ${JAVA_HOME}"
+            fi
+            JAVA="${JAVA_HOME}/bin/java"
+        else
+            warn "JAVA_HOME not set; results may vary"
+            JAVA=$(type java)
+            JAVA=$(expr "${JAVA}" : '.* \(/.*\)$')
+            if [ "x${JAVA}" = "x" ]; then
+                die "java command not found"
+            fi
+        fi
+    fi
+    # if command is env, attempt to add more to the classpath
+    if [ "$1" = "env" ]; then
+        [ "x${TOOLS_JAR}" =  "x" ] && [ -n "${JAVA_HOME}" ] && TOOLS_JAR=$(find -H "${JAVA_HOME}" -name "tools.jar")
+        [ "x${TOOLS_JAR}" =  "x" ] && [ -n "${JAVA_HOME}" ] && TOOLS_JAR=$(find -H "${JAVA_HOME}" -name "classes.jar")
+        if [ "x${TOOLS_JAR}" =  "x" ]; then
+             warn "Could not locate tools.jar or classes.jar. Please set manually to avail all command features."
+        fi
+    fi
+
+}
+
+init() {
+    # Determine if there is special OS handling we must perform
+    detectOS
+
+    # Unlimit the number of file descriptors if possible
+    unlimitFD
+
+    # Locate the Java VM to execute
+    locateJava "$1"
+}
+
+
+install() {
+        SVC_NAME=minifi
+        if [ "x$2" != "x" ] ; then
+                SVC_NAME=$2
+        fi
+
+        SVC_FILE="/etc/init.d/${SVC_NAME}"
+        cp "$0" "${SVC_FILE}"
+        sed -i s:MINIFI_HOME=.*:MINIFI_HOME="${MINIFI_HOME}": "${SVC_FILE}"
+        sed -i s:PROGNAME=.*:PROGNAME="${SCRIPT_NAME}": "${SVC_FILE}"
+        rm -f "/etc/rc2.d/S65${SVC_NAME}"
+        ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/S65${SVC_NAME}"
+        rm -f "/etc/rc2.d/K65${SVC_NAME}"
+        ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/K65${SVC_NAME}"
+        echo "Service ${SVC_NAME} installed"
+}
+
+
+run() {
+    BOOTSTRAP_CONF_DIR="${MINIFI_HOME}/conf"
+    BOOTSTRAP_CONF="${BOOTSTRAP_CONF_DIR}/bootstrap.conf";
+    BOOTSTRAP_LIBS="${MINIFI_HOME}/lib/bootstrap/*"
+
+    run_as=$(grep run.as "${BOOTSTRAP_CONF}" | cut -d'=' -f2)
+    # If the run as user is the same as that starting the process, ignore this configuration
+    if [ "$run_as" = "$(whoami)" ]; then
+        unset run_as
+    fi
+
+    sudo_cmd_prefix=""
+    if $cygwin; then
+        if [ -n "${run_as}" ]; then
+            echo "The run.as option is not supported in a Cygwin environment. Exiting."
+            exit 1
+        fi;
+
+        MINIFI_HOME=$(cygpath --path --windows "${MINIFI_HOME}")
+        BOOTSTRAP_CONF=$(cygpath --path --windows "${BOOTSTRAP_CONF}")
+        BOOTSTRAP_CONF_DIR=$(cygpath --path --windows "${BOOTSTRAP_CONF_DIR}")
+        BOOTSTRAP_LIBS=$(cygpath --path --windows "${BOOTSTRAP_LIBS}")
+        BOOTSTRAP_CLASSPATH="${BOOTSTRAP_CONF_DIR};${BOOTSTRAP_LIBS}"
+        if [ -n "${TOOLS_JAR}" ]; then
+            TOOLS_JAR=$(cygpath --path --windows "${TOOLS_JAR}")
+            BOOTSTRAP_CLASSPATH="${TOOLS_JAR};${BOOTSTRAP_CLASSPATH}"
+        fi
+    else
+        if [ -n "${run_as}" ]; then
+            if id -u "${run_as}" >/dev/null 2>&1; then
+                sudo_cmd_prefix="sudo -u ${run_as}"
+            else
+                echo "The specified run.as user ${run_as} does not exist. Exiting."
+                exit 1
+            fi
+        fi;
+        BOOTSTRAP_CLASSPATH="${BOOTSTRAP_CONF_DIR}:${BOOTSTRAP_LIBS}"
+        if [ -n "${TOOLS_JAR}" ]; then
+            BOOTSTRAP_CLASSPATH="${TOOLS_JAR}:${BOOTSTRAP_CLASSPATH}"
+        fi
+    fi
+
+    echo
+    echo "Java home: ${JAVA_HOME}"
+    echo "MiNiFi home: ${MINIFI_HOME}"
+    echo
+    echo "Bootstrap Config File: ${BOOTSTRAP_CONF}"
+    echo
+
+    # run 'start' in the background because the process will continue to run, monitoring MiNiFi.
+    # all other commands will terminate quickly so want to just wait for them
+    if [ "$1" = "start" ]; then
+        (cd "${MINIFI_HOME}" && ${sudo_cmd_prefix} "${JAVA}" -cp "${BOOTSTRAP_CLASSPATH}" -Xms12m -Xmx24m -Dorg.apache.nifi.minifi.bootstrap.config.file="${BOOTSTRAP_CONF}" org.apache.nifi.minifi.bootstrap.RunMiNiFi $@ &)
+    else
+        (cd "${MINIFI_HOME}" && ${sudo_cmd_prefix} "${JAVA}" -cp "${BOOTSTRAP_CLASSPATH}" -Xms12m -Xmx24m -Dorg.apache.nifi.minifi.bootstrap.config.file="${BOOTSTRAP_CONF}" org.apache.nifi.minifi.bootstrap.RunMiNiFi $@)
+    fi
+
+    # Wait just a bit (3 secs) to wait for the logging to finish and then echo a new-line.
+    # We do this to avoid having logs spewed on the console after running the command and then not giving
+    # control back to the user
+    sleep 3
+    echo
+}
+
+main() {
+    init "$1"
+    run "$@"
+}
+
+
+case "$1" in
+    install)
+        install "$@"
+        ;;
+    start|stop|run|status|dump|env)
+        main "$@"
+        ;;
+    restart)
+        init
+    run "stop"
+    run "start"
+    ;;
+    *)
+        echo "Usage minifi {start|stop|run|restart|status|dump|install}"
+        ;;
+esac

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat
new file mode 100644
index 0000000..8750af6
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat
@@ -0,0 +1,47 @@
+@echo off
+rem
+rem    Licensed to the Apache Software Foundation (ASF) under one or more
+rem    contributor license agreements.  See the NOTICE file distributed with
+rem    this work for additional information regarding copyright ownership.
+rem    The ASF licenses this file to You under the Apache License, Version 2.0
+rem    (the "License"); you may not use this file except in compliance with
+rem    the License.  You may obtain a copy of the License at
+rem
+rem       http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem    Unless required by applicable law or agreed to in writing, software
+rem    distributed under the License is distributed on an "AS IS" BASIS,
+rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem    See the License for the specific language governing permissions and
+rem    limitations under the License.
+rem
+
+rem Use JAVA_HOME if it's set; otherwise, just use java
+
+if "%JAVA_HOME%" == "" goto noJavaHome
+if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
+set JAVA_EXE=%JAVA_HOME%\bin\java.exe
+goto startMiNifi
+
+:noJavaHome
+echo The JAVA_HOME environment variable is not defined correctly.
+echo Instead the PATH will be used to find the java executable.
+echo.
+set JAVA_EXE=java
+goto startMiNifi
+
+:startMiNifi
+set MiNIFI_ROOT=%~dp0..\
+pushd "%MiNIFI_ROOT%"
+set LIB_DIR=lib\bootstrap
+set CONF_DIR=conf
+
+set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
+set JAVA_ARGS=-Dorg.apache.nifi.minifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
+
+SET JAVA_PARAMS=-cp %CONF_DIR%;%LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.minifi.bootstrap.RunMiNiFi
+set BOOTSTRAP_ACTION=run
+
+cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
+
+popd

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat
new file mode 100644
index 0000000..8551baa
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat
@@ -0,0 +1,47 @@
+@echo off
+rem
+rem    Licensed to the Apache Software Foundation (ASF) under one or more
+rem    contributor license agreements.  See the NOTICE file distributed with
+rem    this work for additional information regarding copyright ownership.
+rem    The ASF licenses this file to You under the Apache License, Version 2.0
+rem    (the "License"); you may not use this file except in compliance with
+rem    the License.  You may obtain a copy of the License at
+rem
+rem       http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem    Unless required by applicable law or agreed to in writing, software
+rem    distributed under the License is distributed on an "AS IS" BASIS,
+rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem    See the License for the specific language governing permissions and
+rem    limitations under the License.
+rem
+
+rem Use JAVA_HOME if it's set; otherwise, just use java
+
+if "%JAVA_HOME%" == "" goto noJavaHome
+if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
+set JAVA_EXE=%JAVA_HOME%\bin\java.exe
+goto startMiNifi
+
+:noJavaHome
+echo The JAVA_HOME environment variable is not defined correctly.
+echo Instead the PATH will be used to find the java executable.
+echo.
+set JAVA_EXE=java
+goto startMiNifi
+
+:startMiNifi
+set MINIFI_ROOT=%~dp0..\
+pushd "%MINIFI_ROOT%"
+set LIB_DIR=lib\bootstrap
+set CONF_DIR=conf
+
+set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
+set JAVA_ARGS=-Dorg.apache.nifi.minifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
+
+set JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.minifi.bootstrap.RunNiFiMi
+set BOOTSTRAP_ACTION=status
+
+cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
+
+popd
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authority-providers.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authority-providers.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authority-providers.xml
new file mode 100644
index 0000000..cb68e15
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authority-providers.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<!--
+    This file lists the authority providers to use when running securely. In order
+    to use a specific provider it must be configured here and it's identifier
+    must be specified in the nifi.properties file.
+-->
+<authorityProviders>
+    <provider>
+        <identifier>file-provider</identifier>
+        <class>org.apache.nifi.authorization.FileAuthorizationProvider</class>
+        <property name="Authorized Users File">./conf/authorized-users.xml</property>
+        <property name="Default User Roles"></property>
+    </provider>
+    
+    <!--<provider>
+        <identifier>cluster-ncm-provider</identifier>
+        <class>org.apache.nifi.cluster.authorization.ClusterManagerAuthorizationProvider</class>
+        <property name="Authority Provider Port"></property>
+        <property name="Authority Provider Threads">10</property>
+        <property name="Authorized Users File">./conf/authorized-users.xml</property>
+        <property name="Default User Roles"></property>
+    </provider>-->
+    
+    <!--<provider>
+        <identifier>cluster-node-provider</identifier>
+        <class>org.apache.nifi.cluster.authorization.NodeAuthorizationProvider</class>
+        <property name="Cluster Manager Authority Provider Port"></property>
+    </provider>-->
+</authorityProviders>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authorized-users.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authorized-users.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authorized-users.xml
new file mode 100644
index 0000000..6b07165
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/authorized-users.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<!--
+    This file lists all authorized users for this NiFi instance when using 
+    the FileAuthorizationProvider or ClusterManagerAuthorizationProvider. If one of
+    these providers is not in use then this file is not used. Refer to the properties 
+    file and authority-providers.xml for configuration details.
+    
+    Available roles:
+        ROLE_MONITOR        - for users - read only access to flow
+        ROLE_DFM            - for users - can build and configure data flows
+        ROLE_PROVENANCE     - for users - can access data flow provenance
+        ROLE_ADMIN          - for users - read only access to flow; modify user access; can purge flow configuration history
+        ROLE_PROXY          - for systems - can proxy requests on behalf of users
+        ROLE_NIFI           - for systems - can perform site to site
+-->
+<users>
+    <!--
+    <user dn="[user dn - read only]">
+        <role name="ROLE_MONITOR"/>
+    </user>
+    <user dn="[user dn - data flow manager]">
+        <role name="ROLE_DFM"/>
+    </user>
+    <user dn="[user dn - read only and admin]">
+        <role name="ROLE_ADMIN"/>
+    </user>
+    <user dn="[user dn - data flow manager and admin]">
+        <role name="ROLE_DFM"/>
+        <role name="ROLE_ADMIN"/>
+    </user>
+    <user dn="[user dn - read only and provenance details]">
+        <role name="ROLE_MONITOR"/>
+        <role name="ROLE_PROVENANCE"/>
+    </user>
+    <user dn="[user dn - data flow manager and provenance details]">
+        <role name="ROLE_DFM"/>
+        <role name="ROLE_PROVENANCE"/>
+    </user>
+    <user dn="[system dn - remote NiFi performing site to site]">
+        <role name="ROLE_NIFI"/>
+    </user>
+    -->
+</users>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap-notification-services.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
new file mode 100644
index 0000000..260eaf7
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0"?>
+<!--
+  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.
+-->
+<services>
+    <!-- This file is used to define how interested parties are notified when events in NiFi's lifecycle occur. -->
+    <!-- The format of this file is:
+        <services>
+            <service>
+                <id>service-identifier</id>
+                <class>org.apache.nifi.notifications.DesiredNotificationService</class>
+                <property name="property name">property value</property>
+                <property name="another property">another property value</property>
+            </service>
+        </services>
+        
+        This file can contain 0 to many different service definitions.
+        The id can then be referenced from the bootstrap.conf file in order to configure the notification service
+        to be used when particular lifecycle events occur.
+    -->
+    
+<!--
+     <service>
+        <id>email-notification</id>
+        <class>org.apache.nifi.bootstrap.notification.email.EmailNotificationService</class>
+        <property name="SMTP Hostname"></property>
+        <property name="SMTP Port"></property>
+        <property name="SMTP Username"></property>
+        <property name="SMTP Password"></property>
+        <property name="SMTP TLS"></property>
+        <property name="From"></property>
+        <property name="To"></property>
+     </service>
+-->
+</services>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
new file mode 100644
index 0000000..5e676ff
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
@@ -0,0 +1,60 @@
+#
+# 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.
+#
+
+# Java command to use when running MiNiFi
+java=java
+
+# Username to use when running MiNiFi. This value will be ignored on Windows.
+run.as=${minifi.run.as}
+
+# Configure where MiNiFi's lib and conf directories live
+lib.dir=./lib
+conf.dir=./conf
+
+# How long to wait after telling MiNiFi to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# Disable JSR 199 so that we can use JSP's without running a JDK
+java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
+
+# JVM memory settings
+java.arg.2=-Xms${minifi.jvm.heap.mb}m
+java.arg.3=-Xmx${minifi.jvm.heap.mb}m
+
+# Enable Remote Debugging
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+java.arg.4=-Djava.net.preferIPv4Stack=true
+
+# allowRestrictedHeaders is required for Cluster/Node communications to work properly
+java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
+java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
+
+# Java 7 and below have issues with Code Cache. The following lines allow us to run well even with
+# many classes loaded in the JVM.
+#java.arg.7=-XX:ReservedCodeCacheSize=256m
+#java.arg.8=-XX:CodeCacheFlushingMinimumFreeSpace=10m
+#java.arg.9=-XX:+UseCodeCacheFlushing
+#java.arg.11=-XX:PermSize=${minifi.jvm.permgen.mb}M
+#java.arg.12=-XX:MaxPermSize=${minifi.jvm.permgen.mb}M
+
+# The G1GC is still considered experimental but has proven to be very advantageous in providing great
+# performance without significant "stop-the-world" delays.
+#java.arg.13=-XX:+UseG1GC
+
+#Set headless mode by default
+java.arg.14=-Djava.awt.headless=true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
new file mode 100644
index 0000000..e071858
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
@@ -0,0 +1,150 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<configuration scan="true" scanPeriod="30 seconds">
+    <contextListener class="ch.qos.logback.classic.jul.LevelChangePropagator">
+        <resetJUL>true</resetJUL>
+    </contextListener>
+    
+    <appender name="APP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>logs/minifi-app.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'app_%d.log'.
+              For hourly rollover, use 'app_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>./logs/minifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
+            <timeBasedFileNamingAndTriggeringPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedFNATP">
+                <maxFileSize>100MB</maxFileSize>
+            </timeBasedFileNamingAndTriggeringPolicy>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+            <immediateFlush>true</immediateFlush>
+        </encoder>
+    </appender>
+    
+    <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>logs/minifi-user.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>./logs/minifi-user_%d.log</fileNamePattern>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>logs/minifi-bootstrap.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>./logs/minifi-bootstrap_%d.log</fileNamePattern>
+            <!-- keep 5 log files worth of history -->
+            <maxHistory>5</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+	
+	<appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
+		<encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+	</appender>
+    
+    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
+    
+    <logger name="org.apache.nifi" 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.server.JettyServer" level="INFO"/>
+
+    <!-- Logger for managing logging statements for jetty -->
+    <logger name="org.eclipse.jetty" level="INFO"/>
+
+    <!-- Suppress non-error messages due to excessive logging by class or library -->
+    <logger name="com.sun.jersey.spi.container.servlet.WebComponent" level="ERROR"/>
+    <logger name="com.sun.jersey.spi.spring" level="ERROR"/>
+    <logger name="org.springframework" level="ERROR"/>
+    
+    <!-- Suppress non-error messages due to known warning about redundant path annotation (NIFI-574) -->
+    <logger name="com.sun.jersey.spi.inject.Errors" level="ERROR"/>
+
+    <!--
+        Logger for capturing user events. We do not want to propagate these
+        log events to the root logger. These messages are only sent to the
+        user-log appender.
+    -->
+    <logger name="org.apache.nifi.web.security" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.config" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.filter.RequestLogger" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+
+
+    <!--
+        Logger for capturing Bootstrap logs and MiNiFi's standard error and standard out.
+    -->
+    <logger name="org.apache.nifi.minifi.bootstrap" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+    <logger name="org.apache.nifi.minifi.bootstrap.Command" level="INFO" additivity="false">
+        <appender-ref ref="CONSOLE" />
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to MiNiFi's Standard Out will be logged with the logger org.apache.nifi.minifi.StdOut at INFO level -->
+    <logger name="org.apache.nifi.minifi.StdOut" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+    
+    <!-- Everything written to MiNiFi's Standard Error will be logged with the logger org.apache.nifi.minifi.StdErr at ERROR level -->
+	<logger name="org.apache.nifi.minifi.StdErr" level="ERROR" additivity="false">
+    	<appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+
+    <root level="INFO">
+        <appender-ref ref="APP_FILE"/>
+    </root>
+    
+</configuration>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/login-identity-providers.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/login-identity-providers.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/login-identity-providers.xml
new file mode 100644
index 0000000..3a57e35
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/login-identity-providers.xml
@@ -0,0 +1,109 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<!--
+    This file lists the login identity providers to use when running securely. In order
+    to use a specific provider it must be configured here and it's identifier
+    must be specified in the nifi.properties file.
+-->
+<loginIdentityProviders>
+    <!--
+        Identity Provider for users logging in with username/password against an LDAP server.
+        
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, or START_TLS.
+        
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+            
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully 
+            before the target context is closed. Defaults to false.
+            
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+       
+        'Url' - Url of the LDAP servier (i.e. ldap://<hostname>:<port>).
+        'User Search Base' - Base DN for searching for users (i.e. CN=Users,DC=example,DC=com).
+        'User Search Filter' - Filter for searching for users against the 'User Search Base'.
+            (i.e. sAMAccountName={0}). The user specified name is inserted into '{0}'.
+            
+        'Authentication Expiration' - The duration of how long the user authentication is valid
+            for. If the user never logs out, they will be required to log back in following
+            this duration.
+    -->
+    <!-- To enable the ldap-provider remove 2 lines. This is 1 of 2. 
+    <provider>
+        <identifier>ldap-provider</identifier>
+        <class>org.apache.nifi.ldap.LdapProvider</class>
+        <property name="Authentication Strategy">START_TLS</property>
+
+        <property name="Manager DN"></property>
+        <property name="Manager Password"></property>
+
+        <property name="TLS - Keystore"></property>
+        <property name="TLS - Keystore Password"></property>
+        <property name="TLS - Keystore Type"></property>
+        <property name="TLS - Truststore"></property>
+        <property name="TLS - Truststore Password"></property>
+        <property name="TLS - Truststore Type"></property>
+        <property name="TLS - Client Auth"></property>
+        <property name="TLS - Protocol"></property>
+        <property name="TLS - Shutdown Gracefully"></property>
+        
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url"></property>
+        <property name="User Search Base"></property>
+        <property name="User Search Filter"></property>
+
+        <property name="Authentication Expiration">12 hours</property>
+    </provider>
+    To enable the ldap-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        Identity Provider for users logging in with username/password against a Kerberos KDC server.
+
+        'Default Realm' - Default realm to provide when user enters incomplete user principal (i.e. NIFI.APACHE.ORG).
+        'Kerberos Config File' - Absolute path to Kerberos client configuration file.
+        'Authentication Expiration' - The duration of how long the user authentication is valid for. If the user never logs out, they will be required to log back in following this duration.
+    -->
+    <!-- To enable the kerberos-provider remove 2 lines. This is 1 of 2.
+    <provider>
+        <identifier>kerberos-provider</identifier>
+        <class>org.apache.nifi.kerberos.KerberosProvider</class>
+        <property name="Default Realm">NIFI.APACHE.ORG</property>
+        <property name="Kerberos Config File">/etc/krb5.conf</property>
+        <property name="Authentication Expiration">12 hours</property>
+    </provider>
+    To enable the kerberos-provider remove 2 lines. This is 2 of 2. -->
+</loginIdentityProviders>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/nifi.properties b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/nifi.properties
new file mode 100644
index 0000000..24d2295
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/nifi.properties
@@ -0,0 +1,188 @@
+# 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.
+
+# Core Properties #
+nifi.version=${nifi.version}
+nifi.flow.configuration.file=${nifi.flow.configuration.file}
+nifi.flow.configuration.archive.dir=${nifi.flow.configuration.archive.dir}
+nifi.flowcontroller.autoResumeState=${nifi.flowcontroller.autoResumeState}
+nifi.flowcontroller.graceful.shutdown.period=${nifi.flowcontroller.graceful.shutdown.period}
+nifi.flowservice.writedelay.interval=${nifi.flowservice.writedelay.interval}
+nifi.administrative.yield.duration=${nifi.administrative.yield.duration}
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=${nifi.bored.yield.duration}
+
+nifi.authority.provider.configuration.file=${nifi.authority.provider.configuration.file}
+nifi.login.identity.provider.configuration.file=${nifi.login.identity.provider.configuration.file}
+nifi.templates.directory=${nifi.templates.directory}
+nifi.ui.banner.text=${nifi.ui.banner.text}
+nifi.ui.autorefresh.interval=${nifi.ui.autorefresh.interval}
+nifi.nar.library.directory=${nifi.nar.library.directory}
+nifi.nar.working.directory=${nifi.nar.working.directory}
+nifi.documentation.working.directory=${nifi.documentation.working.directory}
+
+####################
+# State Management #
+####################
+nifi.state.management.configuration.file=${nifi.state.management.configuration.file}
+# The ID of the local state provider
+nifi.state.management.provider.local=${nifi.state.management.provider.local}
+# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
+nifi.state.management.provider.cluster=${nifi.state.management.provider.cluster}
+# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
+nifi.state.management.embedded.zookeeper.start=${nifi.state.management.embedded.zookeeper.start}
+# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
+nifi.state.management.embedded.zookeeper.properties=${nifi.state.management.embedded.zookeeper.properties}
+
+
+# H2 Settings
+nifi.database.directory=${nifi.database.directory}
+nifi.h2.url.append=${nifi.h2.url.append}
+
+# FlowFile Repository
+nifi.flowfile.repository.implementation=${nifi.flowfile.repository.implementation}
+nifi.flowfile.repository.directory=${nifi.flowfile.repository.directory}
+nifi.flowfile.repository.partitions=${nifi.flowfile.repository.partitions}
+nifi.flowfile.repository.checkpoint.interval=${nifi.flowfile.repository.checkpoint.interval}
+nifi.flowfile.repository.always.sync=${nifi.flowfile.repository.always.sync}
+
+nifi.swap.manager.implementation=${nifi.swap.manager.implementation}
+nifi.queue.swap.threshold=${nifi.queue.swap.threshold}
+nifi.swap.in.period=${nifi.swap.in.period}
+nifi.swap.in.threads=${nifi.swap.in.threads}
+nifi.swap.out.period=${nifi.swap.out.period}
+nifi.swap.out.threads=${nifi.swap.out.threads}
+
+# Content Repository
+nifi.content.repository.implementation=${nifi.content.repository.implementation}
+nifi.content.claim.max.appendable.size=${nifi.content.claim.max.appendable.size}
+nifi.content.claim.max.flow.files=${nifi.content.claim.max.flow.files}
+nifi.content.repository.directory.default=${nifi.content.repository.directory.default}
+nifi.content.repository.archive.max.retention.period=${nifi.content.repository.archive.max.retention.period}
+nifi.content.repository.archive.max.usage.percentage=${nifi.content.repository.archive.max.usage.percentage}
+nifi.content.repository.archive.enabled=${nifi.content.repository.archive.enabled}
+nifi.content.repository.always.sync=${nifi.content.repository.always.sync}
+nifi.content.viewer.url=${nifi.content.viewer.url}
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=${nifi.provenance.repository.implementation}
+
+# Persistent Provenance Repository Properties
+nifi.provenance.repository.directory.default=${nifi.provenance.repository.directory.default}
+nifi.provenance.repository.max.storage.time=${nifi.provenance.repository.max.storage.time}
+nifi.provenance.repository.max.storage.size=${nifi.provenance.repository.max.storage.size}
+nifi.provenance.repository.rollover.time=${nifi.provenance.repository.rollover.time}
+nifi.provenance.repository.rollover.size=${nifi.provenance.repository.rollover.size}
+nifi.provenance.repository.query.threads=${nifi.provenance.repository.query.threads}
+nifi.provenance.repository.index.threads=${nifi.provenance.repository.index.threads}
+nifi.provenance.repository.compress.on.rollover=${nifi.provenance.repository.compress.on.rollover}
+nifi.provenance.repository.always.sync=${nifi.provenance.repository.always.sync}
+nifi.provenance.repository.journal.count=${nifi.provenance.repository.journal.count}
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are: 
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
+nifi.provenance.repository.indexed.fields=${nifi.provenance.repository.indexed.fields}
+# FlowFile Attributes that should be indexed and made searchable.  Some examples to consider are filename, uuid, mime.type
+nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.indexed.attributes}
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=${nifi.provenance.repository.index.shard.size}
+# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
+# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
+nifi.provenance.repository.max.attribute.length=${nifi.provenance.repository.max.attribute.length}
+
+# Volatile Provenance Respository Properties
+nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size}
+
+# Component Status Repository
+nifi.components.status.repository.implementation=${nifi.components.status.repository.implementation}
+nifi.components.status.repository.buffer.size=${nifi.components.status.repository.buffer.size}
+nifi.components.status.snapshot.frequency=${nifi.components.status.snapshot.frequency}
+
+# Site to Site properties
+nifi.remote.input.socket.host=
+nifi.remote.input.socket.port=
+nifi.remote.input.secure=true
+
+# web properties #
+nifi.web.war.directory=${nifi.web.war.directory}
+nifi.web.http.host=${nifi.web.http.host}
+nifi.web.http.port=${nifi.web.http.port}
+nifi.web.https.host=${nifi.web.https.host}
+nifi.web.https.port=${nifi.web.https.port}
+nifi.web.jetty.working.directory=${nifi.jetty.work.dir}
+nifi.web.jetty.threads=${nifi.web.jetty.threads}
+
+# security properties #
+nifi.sensitive.props.key=
+nifi.sensitive.props.algorithm=${nifi.sensitive.props.algorithm}
+nifi.sensitive.props.provider=${nifi.sensitive.props.provider}
+
+nifi.security.keystore=${nifi.security.keystore}
+nifi.security.keystoreType=${nifi.security.keystoreType}
+nifi.security.keystorePasswd=${nifi.security.keystorePasswd}
+nifi.security.keyPasswd=${nifi.security.keyPasswd}
+nifi.security.truststore=${nifi.security.truststore}
+nifi.security.truststoreType=${nifi.security.truststoreType}
+nifi.security.truststorePasswd=${nifi.security.truststorePasswd}
+nifi.security.needClientAuth=${nifi.security.needClientAuth}
+nifi.security.user.credential.cache.duration=${nifi.security.user.credential.cache.duration}
+nifi.security.user.authority.provider=${nifi.security.user.authority.provider}
+nifi.security.user.login.identity.provider=${nifi.security.user.login.identity.provider}
+nifi.security.support.new.account.requests=${nifi.security.support.new.account.requests}
+# Valid Authorities include: ROLE_MONITOR,ROLE_DFM,ROLE_ADMIN,ROLE_PROVENANCE,ROLE_NIFI
+nifi.security.anonymous.authorities=${nifi.security.anonymous.authorities}
+nifi.security.ocsp.responder.url=${nifi.security.ocsp.responder.url}
+nifi.security.ocsp.responder.certificate=${nifi.security.ocsp.responder.certificate}
+
+# cluster common properties (cluster manager and nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=${nifi.cluster.protocol.heartbeat.interval}
+nifi.cluster.protocol.is.secure=${nifi.cluster.protocol.is.secure}
+nifi.cluster.protocol.socket.timeout=${nifi.cluster.protocol.socket.timeout}
+nifi.cluster.protocol.connection.handshake.timeout=${nifi.cluster.protocol.connection.handshake.timeout}
+# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured #
+nifi.cluster.protocol.use.multicast=${nifi.cluster.protocol.use.multicast}
+nifi.cluster.protocol.multicast.address=${nifi.cluster.protocol.multicast.address}
+nifi.cluster.protocol.multicast.port=${nifi.cluster.protocol.multicast.port}
+nifi.cluster.protocol.multicast.service.broadcast.delay=${nifi.cluster.protocol.multicast.service.broadcast.delay}
+nifi.cluster.protocol.multicast.service.locator.attempts=${nifi.cluster.protocol.multicast.service.locator.attempts}
+nifi.cluster.protocol.multicast.service.locator.attempts.delay=${nifi.cluster.protocol.multicast.service.locator.attempts.delay}
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=${nifi.cluster.is.node}
+nifi.cluster.node.address=${nifi.cluster.node.address}
+nifi.cluster.node.protocol.port=${nifi.cluster.node.protocol.port}
+nifi.cluster.node.protocol.threads=${nifi.cluster.node.protocol.threads}
+# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx #
+nifi.cluster.node.unicast.manager.address=${nifi.cluster.node.unicast.manager.address}
+nifi.cluster.node.unicast.manager.protocol.port=${nifi.cluster.node.unicast.manager.protocol.port}
+
+# cluster manager properties (only configure for cluster manager) #
+nifi.cluster.is.manager=${nifi.cluster.is.manager}
+nifi.cluster.manager.address=${nifi.cluster.manager.address}
+nifi.cluster.manager.protocol.port=${nifi.cluster.manager.protocol.port}
+nifi.cluster.manager.node.firewall.file=${nifi.cluster.manager.node.firewall.file}
+nifi.cluster.manager.node.event.history.size=${nifi.cluster.manager.node.event.history.size}
+nifi.cluster.manager.node.api.connection.timeout=${nifi.cluster.manager.node.api.connection.timeout}
+nifi.cluster.manager.node.api.read.timeout=${nifi.cluster.manager.node.api.read.timeout}
+nifi.cluster.manager.node.api.request.threads=${nifi.cluster.manager.node.api.request.threads}
+nifi.cluster.manager.flow.retrieval.delay=${nifi.cluster.manager.flow.retrieval.delay}
+nifi.cluster.manager.protocol.threads=${nifi.cluster.manager.protocol.threads}
+nifi.cluster.manager.safemode.duration=${nifi.cluster.manager.safemode.duration}
+
+# kerberos #
+nifi.kerberos.krb5.file=${nifi.kerberos.krb5.file}
+nifi.kerberos.service.principal=${nifi.kerberos.service.principal}
+nifi.kerberos.keytab.location=${nifi.kerberos.keytab.location}
+nifi.kerberos.authentication.expiration=${nifi.kerberos.authentication.expiration}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/state-management.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/state-management.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/state-management.xml
new file mode 100644
index 0000000..665b22b
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/state-management.xml
@@ -0,0 +1,68 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<!--
+  This file provides a mechanism for defining and configuring the State Providers
+  that should be used for storing state locally and across a NiFi cluster. In order
+  to use a specific provider, it must be configured here and its identifier
+  must be specified in the nifi.properties file.
+-->
+<stateManagement>
+    <!--
+        State Provider that stores state locally in a configurable directory. This Provider requires the following properties:
+        
+        Directory - the directory to store components' state in. If the directory being used is a sub-directory of the NiFi installation, it
+                    is important that the directory be copied over to the new version when upgrading NiFi.
+     -->
+    <local-provider>
+        <id>local-provider</id>
+        <class>org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider</class>
+        <property name="Directory">./state/local</property>
+    </local-provider>
+
+    <!--
+        State Provider that is used to store state in ZooKeeper. This Provider requires the following properties:
+        
+        Root Node - the root node in ZooKeeper where state should be stored. The default is '/nifi', but it is advisable to change this to a different value if not using
+                   the embedded ZooKeeper server and if multiple NiFi instances may all be using the same ZooKeeper Server.
+                   
+        Connect String - A comma-separated list of host:port pairs to connect to ZooKeeper. For example, myhost.mydomain:2181,host2.mydomain:5555,host3:6666
+        
+        Session Timeout - Specifies how long this instance of NiFi is allowed to be disconnected from ZooKeeper before creating a new ZooKeeper Session. Default value is "30 seconds"
+        
+        Access Control - Specifies which Access Controls will be applied to the ZooKeeper ZNodes that are created by this State Provider. This value must be set to one of:
+                            - Open  : ZNodes will be open to any ZooKeeper client.
+                            - CreatorOnly  : ZNodes will be accessible only by the creator. The creator will have full access to create children, read, write, delete, and administer the ZNodes.
+                                             This option is available only if access to ZooKeeper is secured via Kerberos or if a Username and Password are set.
+
+        Username - An optional username that can be used to assign Access Controls to ZNodes. ZooKeeper allows users to assign arbitrary usernames and passwords to ZNodes. These usernames
+                   and passwords are not explicitly defined elsewhere but are simply associated with ZNodes, so it is important that all NiFi nodes in a cluster have the same value for the
+                   Username and Password properties.
+
+        Password - An optional password that can be used to assign Access Controls to ZNodes. This property must be set if the Username property is set. NOTE: ZooKeeper transmits passwords
+                   in plain text. As a result, a Username and Password should be used only if communicate with a ZooKeeper on a localhost or over encrypted comms (such as configuring SSL
+                   communications with ZooKeeper).
+    -->
+    <cluster-provider>
+        <id>zk-provider</id>
+        <class>org.apache.nifi.controller.state.providers.zookeeper.ZooKeeperStateProvider</class>
+        <property name="Connect String"></property>
+        <property name="Root Node">/nifi</property>
+        <property name="Session Timeout">30 seconds</property>
+        <property name="Access Control">CreatorOnly</property>
+        <property name="Username">nifi</property>
+        <property name="Password">nifi</property>
+    </cluster-provider>
+</stateManagement>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/zookeeper.properties
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/zookeeper.properties b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/zookeeper.properties
new file mode 100644
index 0000000..fa1d885
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/zookeeper.properties
@@ -0,0 +1,45 @@
+#
+#
+# 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.
+#
+#
+#
+
+clientPort=2181
+initLimit=10
+autopurge.purgeInterval=24
+syncLimit=5
+tickTime=2000
+dataDir=./state/zookeeper
+autopurge.snapRetainCount=30
+
+#
+# Specifies the servers that are part of this zookeeper ensemble. For
+# every NiFi instance running an embedded zookeeper, there needs to be
+# a server entry below. For instance:
+#
+# server.1=nifi-node1-hostname:2888:3888
+# server.2=nifi-node2-hostname:2888:3888
+# server.3=nifi-node3-hostname:2888:3888
+#
+# The index of the server corresponds to the myid file that gets created
+# in the dataDir of each node running an embedded zookeeper. See the
+# administration guide for more details.
+#
+
+server.1=
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
new file mode 100644
index 0000000..d8ec39a
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>minifi-framework</artifactId>
+        <groupId>org.apache.nifi.minifi</groupId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.nifi.minifi</groupId>
+    <artifactId>minifi-runtime</artifactId>
+    <packaging>jar</packaging>
+
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file


[3/3] nifi-minifi git commit: MINIFI-5: Creating a base MiNiFi project to serve as a basis for further extension and design reusing NiFi libraries

Posted by al...@apache.org.
MINIFI-5:  Creating a base MiNiFi project to serve as a basis for further extension and design reusing NiFi libraries

This closes #5


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

Branch: refs/heads/master
Commit: 5751e23d434dd283a8f7a7f3bd009ebce5864974
Parents: 302ae7c
Author: Aldrin Piri <al...@apache.org>
Authored: Tue Apr 5 21:19:57 2016 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Wed Apr 6 14:07:45 2016 -0400

----------------------------------------------------------------------
 minifi-assembly/pom.xml                         | 386 ++++++++++--------
 .../src/main/assembly/dependencies.xml          |  29 +-
 minifi-bootstrap/pom.xml                        |  19 +-
 minifi-framework/minifi-resources/LICENSE       | 202 ----------
 minifi-framework/minifi-resources/NOTICE        |   5 -
 minifi-framework/minifi-resources/pom.xml       |  50 ---
 .../src/main/assembly/dependencies.xml          |  52 ---
 .../src/main/resources/bin/dump-minifi.bat      |  47 ---
 .../src/main/resources/bin/minifi.sh            | 247 ------------
 .../src/main/resources/bin/run-minifi.bat       |  47 ---
 .../src/main/resources/bin/status-minifi.bat    |  47 ---
 .../src/main/resources/conf/bootstrap.conf      |  60 ---
 .../src/main/resources/conf/logback.xml         | 150 -------
 minifi-framework/pom.xml                        |  34 --
 .../minifi-framework-nar/pom.xml                |  67 ++++
 .../minifi-framework/minifi-resources/LICENSE   | 202 ++++++++++
 .../minifi-framework/minifi-resources/NOTICE    |   5 +
 .../minifi-framework/minifi-resources/pom.xml   |  50 +++
 .../src/main/assembly/dependencies.xml          |  52 +++
 .../src/main/resources/bin/dump-minifi.bat      |  47 +++
 .../src/main/resources/bin/minifi.sh            | 247 ++++++++++++
 .../src/main/resources/bin/run-minifi.bat       |  47 +++
 .../src/main/resources/bin/status-minifi.bat    |  47 +++
 .../main/resources/conf/authority-providers.xml |  43 ++
 .../main/resources/conf/authorized-users.xml    |  57 +++
 .../conf/bootstrap-notification-services.xml    |  46 +++
 .../src/main/resources/conf/bootstrap.conf      |  60 +++
 .../src/main/resources/conf/logback.xml         | 150 +++++++
 .../resources/conf/login-identity-providers.xml | 109 +++++
 .../src/main/resources/conf/nifi.properties     | 188 +++++++++
 .../main/resources/conf/state-management.xml    |  68 ++++
 .../main/resources/conf/zookeeper.properties    |  45 +++
 .../minifi-framework/minifi-runtime/pom.xml     |  43 ++
 .../apache/nifi/minifi/BootstrapListener.java   | 395 +++++++++++++++++++
 .../java/org/apache/nifi/minifi/MiNiFi.java     | 234 +++++++++++
 .../minifi-framework/pom.xml                    |  39 ++
 .../minifi-framework-bundle/pom.xml             |  37 ++
 minifi-nar-bundles/pom.xml                      |  29 ++
 pom.xml                                         | 174 +++++++-
 39 files changed, 2737 insertions(+), 1119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/pom.xml b/minifi-assembly/pom.xml
index de4a58d..b076459 100644
--- a/minifi-assembly/pom.xml
+++ b/minifi-assembly/pom.xml
@@ -27,7 +27,6 @@ limitations under the License.
     <artifactId>minifi-assembly</artifactId>
     <packaging>pom</packaging>
     <description>This is the assembly of Apache NiFi - MiNiFi</description>
-
     <build>
         <plugins>
             <plugin>
@@ -60,21 +59,11 @@ limitations under the License.
         </plugins>
     </build>
     <dependencies>
+        <!-- Logging -->
         <dependency>
-            <groupId>org.apache.nifi.minifi</groupId>
-            <artifactId>minifi-bootstrap</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi.minifi</groupId>
-            <artifactId>minifi-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi.minifi</groupId>
-            <artifactId>minifi-resources</artifactId>
-            <classifier>resources</classifier>
-            <scope>runtime</scope>
-            <type>zip</type>
-            <version>0.0.1-SNAPSHOT</version>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>
@@ -96,160 +85,235 @@ limitations under the License.
             <artifactId>slf4j-api</artifactId>
             <scope>compile</scope>
         </dependency>
+
+        <!-- MiNiFi -->
         <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-classic</artifactId>
-            <scope>compile</scope>
+            <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-resources</artifactId>
+            <classifier>resources</classifier>
+            <scope>runtime</scope>
+            <type>zip</type>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-bootstrap</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-runtime</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
         </dependency>
-    </dependencies>
 
+        <!-- MiNiFi NiFi Dependencies -->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+        </dependency>
+
+        <!-- NiFi Assembly Dependencies-->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-docs</artifactId>
+            <classifier>resources</classifier>
+            <scope>runtime</scope>
+            <type>zip</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-provenance-repository-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-jetty-bundle</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-update-attribute-nar</artifactId>
+            <type>nar</type>
+        </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 />
+
+        <!-- nifi.properties: core properties -->
+        <nifi.version>${project.version}</nifi.version>
+        <nifi.flowcontroller.autoResumeState>true</nifi.flowcontroller.autoResumeState>
+        <nifi.flowcontroller.graceful.shutdown.period>10 sec</nifi.flowcontroller.graceful.shutdown.period>
+        <nifi.flowservice.writedelay.interval>500 ms</nifi.flowservice.writedelay.interval>
+        <nifi.administrative.yield.duration>30 sec</nifi.administrative.yield.duration>
+        <nifi.bored.yield.duration>10 millis</nifi.bored.yield.duration>
+
+        <nifi.flow.configuration.file>./conf/flow.xml.gz</nifi.flow.configuration.file>
+        <nifi.flow.configuration.archive.dir>./conf/archive/</nifi.flow.configuration.archive.dir>
+        <nifi.login.identity.provider.configuration.file>./conf/login-identity-providers.xml</nifi.login.identity.provider.configuration.file>
+        <nifi.authority.provider.configuration.file>./conf/authority-providers.xml</nifi.authority.provider.configuration.file>
+        <nifi.templates.directory>./conf/templates</nifi.templates.directory>
+        <nifi.database.directory>./database_repository</nifi.database.directory>
+
+        <nifi.state.management.configuration.file>./conf/state-management.xml</nifi.state.management.configuration.file>
+        <nifi.state.management.embedded.zookeeper.start>false</nifi.state.management.embedded.zookeeper.start>
+        <nifi.state.management.embedded.zookeeper.properties>./conf/zookeeper.properties</nifi.state.management.embedded.zookeeper.properties>
+        <nifi.state.management.provider.local>local-provider</nifi.state.management.provider.local>
+        <nifi.state.management.provider.cluster>zk-provider</nifi.state.management.provider.cluster>
+
+        <nifi.flowfile.repository.implementation>org.apache.nifi.controller.repository.WriteAheadFlowFileRepository</nifi.flowfile.repository.implementation>
+        <nifi.flowfile.repository.directory>./flowfile_repository</nifi.flowfile.repository.directory>
+        <nifi.flowfile.repository.partitions>256</nifi.flowfile.repository.partitions>
+        <nifi.flowfile.repository.checkpoint.interval>2 mins</nifi.flowfile.repository.checkpoint.interval>
+        <nifi.flowfile.repository.always.sync>false</nifi.flowfile.repository.always.sync>
+        <nifi.swap.manager.implementation>org.apache.nifi.controller.FileSystemSwapManager</nifi.swap.manager.implementation>
+        <nifi.queue.swap.threshold>20000</nifi.queue.swap.threshold>
+        <nifi.swap.in.period>5 sec</nifi.swap.in.period>
+        <nifi.swap.in.threads>1</nifi.swap.in.threads>
+        <nifi.swap.out.period>5 sec</nifi.swap.out.period>
+        <nifi.swap.out.threads>4</nifi.swap.out.threads>
+
+        <nifi.content.repository.implementation>org.apache.nifi.controller.repository.FileSystemRepository</nifi.content.repository.implementation>
+        <nifi.content.claim.max.appendable.size>10 MB</nifi.content.claim.max.appendable.size>
+        <nifi.content.claim.max.flow.files>100</nifi.content.claim.max.flow.files>
+        <nifi.content.repository.directory.default>./content_repository</nifi.content.repository.directory.default>
+        <nifi.content.repository.archive.max.retention.period>12 hours</nifi.content.repository.archive.max.retention.period>
+        <nifi.content.repository.archive.max.usage.percentage>50%</nifi.content.repository.archive.max.usage.percentage>
+        <nifi.content.repository.archive.enabled>true</nifi.content.repository.archive.enabled>
+        <nifi.content.repository.always.sync>false</nifi.content.repository.always.sync>
+        <nifi.content.viewer.url>/nifi-content-viewer/</nifi.content.viewer.url>
+
+        <nifi.restore.directory />
+        <nifi.ui.banner.text />
+        <nifi.ui.autorefresh.interval>30 sec</nifi.ui.autorefresh.interval>
+        <nifi.nar.library.directory>./lib</nifi.nar.library.directory>
+        <nifi.nar.working.directory>./work/nar/</nifi.nar.working.directory>
+        <nifi.documentation.working.directory>./work/docs/components</nifi.documentation.working.directory>
+
+        <nifi.sensitive.props.algorithm>PBEWITHMD5AND256BITAES-CBC-OPENSSL</nifi.sensitive.props.algorithm>
+        <nifi.sensitive.props.provider>BC</nifi.sensitive.props.provider>
+        <nifi.h2.url.append>;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE</nifi.h2.url.append>
+
+        <nifi.remote.input.socket.port>9990</nifi.remote.input.socket.port>
+
+        <!-- persistent provenance repository properties -->
+        <nifi.provenance.repository.implementation>org.apache.nifi.provenance.PersistentProvenanceRepository</nifi.provenance.repository.implementation>
+        <nifi.provenance.repository.directory.default>./provenance_repository</nifi.provenance.repository.directory.default>
+        <nifi.provenance.repository.max.storage.time>24 hours</nifi.provenance.repository.max.storage.time>
+        <nifi.provenance.repository.max.storage.size>1 GB</nifi.provenance.repository.max.storage.size>
+        <nifi.provenance.repository.rollover.time>30 secs</nifi.provenance.repository.rollover.time>
+        <nifi.provenance.repository.rollover.size>100 MB</nifi.provenance.repository.rollover.size>
+        <nifi.provenance.repository.query.threads>2</nifi.provenance.repository.query.threads>
+        <nifi.provenance.repository.index.threads>1</nifi.provenance.repository.index.threads>
+        <nifi.provenance.repository.compress.on.rollover>true</nifi.provenance.repository.compress.on.rollover>
+        <nifi.provenance.repository.indexed.fields>EventType, FlowFileUUID, Filename, ProcessorID, Relationship</nifi.provenance.repository.indexed.fields>
+        <nifi.provenance.repository.indexed.attributes />
+        <nifi.provenance.repository.index.shard.size>500 MB</nifi.provenance.repository.index.shard.size>
+        <nifi.provenance.repository.always.sync>false</nifi.provenance.repository.always.sync>
+        <nifi.provenance.repository.journal.count>16</nifi.provenance.repository.journal.count>
+        <nifi.provenance.repository.max.attribute.length>65536</nifi.provenance.repository.max.attribute.length>
+
+        <!-- volatile provenance repository properties -->
+        <nifi.provenance.repository.buffer.size>100000</nifi.provenance.repository.buffer.size>
+
+        <!-- Component status repository properties -->
+        <nifi.components.status.repository.implementation>org.apache.nifi.controller.status.history.VolatileComponentStatusRepository</nifi.components.status.repository.implementation>
+        <nifi.components.status.repository.buffer.size>1440</nifi.components.status.repository.buffer.size>
+        <nifi.components.status.snapshot.frequency>1 min</nifi.components.status.snapshot.frequency>
+
+        <!-- nifi.properties: web properties -->
+        <nifi.web.war.directory>./lib</nifi.web.war.directory>
+        <nifi.web.http.host />
+        <nifi.web.http.port>8080</nifi.web.http.port>
+        <nifi.web.https.host />
+        <nifi.web.https.port />
+        <nifi.jetty.work.dir>./work/jetty</nifi.jetty.work.dir>
+        <nifi.web.jetty.threads>200</nifi.web.jetty.threads>
+
+        <!-- nifi.properties: security properties -->
+        <nifi.security.keystore />
+        <nifi.security.keystoreType />
+        <nifi.security.keystorePasswd />
+        <nifi.security.keyPasswd />
+        <nifi.security.truststore />
+        <nifi.security.truststoreType />
+        <nifi.security.truststorePasswd />
+        <nifi.security.needClientAuth />
+        <nifi.security.authorizedUsers.file>./conf/authorized-users.xml</nifi.security.authorizedUsers.file>
+        <nifi.security.user.credential.cache.duration>24 hours</nifi.security.user.credential.cache.duration>
+        <nifi.security.user.authority.provider>file-provider</nifi.security.user.authority.provider>
+        <nifi.security.user.login.identity.provider />
+        <nifi.security.x509.principal.extractor />
+        <nifi.security.support.new.account.requests />
+        <nifi.security.anonymous.authorities />
+        <nifi.security.ocsp.responder.url />
+        <nifi.security.ocsp.responder.certificate />
+
+        <!-- nifi.properties: cluster common properties (cluster manager and nodes must have same values) -->
+        <nifi.cluster.protocol.heartbeat.interval>5 sec</nifi.cluster.protocol.heartbeat.interval>
+        <nifi.cluster.protocol.is.secure>false</nifi.cluster.protocol.is.secure>
+        <nifi.cluster.protocol.socket.timeout>30 sec</nifi.cluster.protocol.socket.timeout>
+        <nifi.cluster.protocol.connection.handshake.timeout>45 sec</nifi.cluster.protocol.connection.handshake.timeout>
+        <nifi.cluster.protocol.use.multicast>false</nifi.cluster.protocol.use.multicast>
+        <nifi.cluster.protocol.multicast.address />
+        <nifi.cluster.protocol.multicast.port />
+        <nifi.cluster.protocol.multicast.service.broadcast.delay>500 ms</nifi.cluster.protocol.multicast.service.broadcast.delay>
+        <nifi.cluster.protocol.multicast.service.locator.attempts>3</nifi.cluster.protocol.multicast.service.locator.attempts>
+        <nifi.cluster.protocol.multicast.service.locator.attempts.delay>1 sec</nifi.cluster.protocol.multicast.service.locator.attempts.delay>
+
+        <!-- nifi.properties: cluster node properties (only configure for cluster nodes) -->
+        <nifi.cluster.is.node>false</nifi.cluster.is.node>
+        <nifi.cluster.node.address />
+        <nifi.cluster.node.protocol.port />
+        <nifi.cluster.node.protocol.threads>2</nifi.cluster.node.protocol.threads>
+        <nifi.cluster.node.unicast.manager.address />
+        <nifi.cluster.node.unicast.manager.protocol.port />
+
+        <!-- nifi.properties: cluster manager properties (only configure for cluster manager) -->
+        <nifi.cluster.is.manager>false</nifi.cluster.is.manager>
+        <nifi.cluster.manager.address />
+        <nifi.cluster.manager.protocol.port />
+        <nifi.cluster.manager.node.firewall.file />
+        <nifi.cluster.manager.node.event.history.size>10</nifi.cluster.manager.node.event.history.size>
+        <nifi.cluster.manager.node.api.connection.timeout>30 sec</nifi.cluster.manager.node.api.connection.timeout>
+        <nifi.cluster.manager.node.api.read.timeout>30 sec</nifi.cluster.manager.node.api.read.timeout>
+        <nifi.cluster.manager.node.api.request.threads>10</nifi.cluster.manager.node.api.request.threads>
+        <nifi.cluster.manager.flow.retrieval.delay>5 sec</nifi.cluster.manager.flow.retrieval.delay>
+        <nifi.cluster.manager.protocol.threads>10</nifi.cluster.manager.protocol.threads>
+        <nifi.cluster.manager.safemode.duration>0 sec</nifi.cluster.manager.safemode.duration>
+
+        <!-- nifi.properties: kerberos properties -->
+        <nifi.kerberos.krb5.file> </nifi.kerberos.krb5.file>
+        <nifi.kerberos.service.principal />
+        <nifi.kerberos.keytab.location />
+        <nifi.kerberos.authentication.expiration>12 hours</nifi.kerberos.authentication.expiration>
     </properties>
 
-    <profiles>
-        <profile>
-            <id>rpm</id>
-            <activation>
-                <activeByDefault>false</activeByDefault>
-            </activation>
-            <properties>
-                <minifi.run.as>minifi</minifi.run.as>
-            </properties>
-            <build>
-                <plugins>
-                    <plugin>
-                        <artifactId>maven-dependency-plugin</artifactId>
-                        <executions>
-                            <execution>
-                                <id>unpack-shared-resources</id>
-                                <goals>
-                                    <goal>unpack-dependencies</goal>
-                                </goals>
-                                <phase>generate-resources</phase>
-                                <configuration>
-                                    <outputDirectory>${project.build.directory}/generated-resources</outputDirectory>
-                                    <includeArtifactIds>minifi-resources</includeArtifactIds>
-                                    <includeGroupIds>org.apache.nifi.minifi</includeGroupIds>
-                                    <excludeTransitive>false</excludeTransitive>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
-                    <plugin>
-                        <groupId>org.codehaus.mojo</groupId>
-                        <artifactId>rpm-maven-plugin</artifactId>
-                        <configuration>
-                            <name>minifi</name>
-                            <summary>Apache NiFi - MiNiFi</summary>
-                            <description>Apache NiFi - MiNiFi</description>
-                            <license>Apache License, Version 2.0 and others (see included LICENSE file)</license>
-                            <url>http://nifi.apache.org</url>
-                            <group>Utilities</group>
-                            <prefix>/opt/minifi</prefix>
-                            <defineStatements>
-                                <defineStatement>_use_internal_dependency_generator 0</defineStatement>
-                            </defineStatements>
-                            <defaultDirmode>750</defaultDirmode>
-                            <defaultFilemode>640</defaultFilemode>
-                            <defaultUsername>minifi</defaultUsername>
-                            <defaultGroupname>minifi</defaultGroupname>
-                            <preinstallScriptlet>
-                                <script>
-                                    /usr/bin/getent group minifi &gt;/dev/null || /usr/sbin/groupadd -r minifi; /usr/bin/getent passwd minifi &gt;/dev/null || /usr/sbin/useradd -r -g minifi -d /opt/minifi -s /sbin/nologin -c "MiNiFi System User" minifi
-                                </script>
-                            </preinstallScriptlet>
-                        </configuration>
-                        <executions>
-                            <execution>
-                                <id>build-bin-rpm</id>
-                                <goals>
-                                    <goal>attached-rpm</goal>
-                                </goals>
-                                <configuration>
-                                    <classifier>bin</classifier>
-                                    <provides>
-                                        <provide>minifi</provide>
-                                    </provides>
-                                    <mappings>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}</directory>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}</directory>
-                                            <sources>
-                                                <source>
-                                                    <location>./LICENSE</location>
-                                                </source>
-                                                <source>
-                                                    <location>./NOTICE</location>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi</directory>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}/bin</directory>
-                                            <filemode>750</filemode>
-                                            <sources>
-                                                <source>
-                                                    <location>${project.build.directory}/generated-resources/bin/minifi.sh</location>
-                                                    <destination>minifi.sh</destination>
-                                                    <filter>true</filter>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}/conf</directory>
-                                            <configuration>true</configuration>
-                                            <sources>
-                                                <source>
-                                                    <location>${project.build.directory}/generated-resources/conf</location>
-                                                    <filter>true</filter>
-                                                </source>
-                                            </sources>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}/lib</directory>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}/lib</directory>
-                                            <dependency>
-                                                <excludes>
-                                                    <exclude>org.apache.nifi.minifi:minifi-bootstrap</exclude>
-                                                    <exclude>org.apache.nifi.minifi:minifi-resources</exclude>
-                                                    <exclude>commons-io:commons-io</exclude>
-                                                    <exclude>org.apache.commons:commons-lang3</exclude>
-                                                    <exclude>org.antlr:antlr-runtime</exclude>
-                                                    <exclude>javax.activation:activation</exclude>
-                                                    <exclude>javax.mail:mail</exclude>
-                                                </excludes>
-                                            </dependency>
-                                        </mapping>
-                                        <mapping>
-                                            <directory>/opt/minifi/minifi-${project.version}/lib/bootstrap</directory>
-                                            <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>
-                                    </mappings>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
-                </plugins>
-            </build>
-        </profile>
-    </profiles>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/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 4582a55..85d9a9e 100644
--- a/minifi-assembly/src/main/assembly/dependencies.xml
+++ b/minifi-assembly/src/main/assembly/dependencies.xml
@@ -35,9 +35,10 @@
             <excludes>
             	<exclude>minifi-bootstrap</exclude>
                 <exclude>minifi-resources</exclude>
+                <exclude>nifi-docs</exclude>
             </excludes>
         </dependencySet>
-        
+
         <!-- Write out the bootstrap lib component to its own dir -->
         <dependencySet>
             <scope>runtime</scope>
@@ -50,9 +51,11 @@
             	<include>minifi-bootstrap</include>
                 <include>slf4j-api</include>
                 <include>logback-classic</include>
+                <include>nifi-api</include>
+                <include>nifi-utils</include>
             </includes>
         </dependencySet>
-        
+
         <!-- Write out the conf directory contents -->
         <dependencySet>
             <scope>runtime</scope>
@@ -92,7 +95,25 @@
                 </includes>
             </unpackOptions>
         </dependencySet>
-
+        <!-- Writes out the docs directory contents -->
+        <dependencySet>
+            <scope>runtime</scope>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>docs/</outputDirectory>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <includes>
+                <include>nifi-docs</include>
+            </includes>
+            <unpack>true</unpack>
+            <unpackOptions>
+                <filtered>false</filtered>
+                <excludes>
+                    <!-- LICENSE and NOTICE both covered by top-level -->
+                    <exclude>LICENSE</exclude>
+                    <exclude>NOTICE</exclude>
+                </excludes>
+            </unpackOptions>
+        </dependencySet>
     </dependencySets>
     <files>
         <file>
@@ -101,7 +122,7 @@
             <destName>LICENSE</destName>
             <fileMode>0644</fileMode>
             <filtered>true</filtered>
-        </file>       
+        </file>
         <file>
             <source>./NOTICE</source>
             <outputDirectory>./</outputDirectory>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-bootstrap/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/pom.xml b/minifi-bootstrap/pom.xml
index f7929c5..1c3e967 100644
--- a/minifi-bootstrap/pom.xml
+++ b/minifi-bootstrap/pom.xml
@@ -29,20 +29,21 @@ limitations under the License.
 
     <dependencies>
         <dependency>
-            <groupId>org.yaml</groupId>
-            <artifactId>snakeyaml</artifactId>
-            <version>1.17</version>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-            <version>0.6.0</version>
+            <artifactId>nifi-api</artifactId>
+            <scope>compile</scope>
         </dependency>
         <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-            <version>${org.slf4j.version}</version>
-            <scope>compile</scope>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-expression-language</artifactId>
         </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/LICENSE b/minifi-framework/minifi-resources/LICENSE
deleted file mode 100644
index d645695..0000000
--- a/minifi-framework/minifi-resources/LICENSE
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/NOTICE b/minifi-framework/minifi-resources/NOTICE
deleted file mode 100644
index 4885e58..0000000
--- a/minifi-framework/minifi-resources/NOTICE
+++ /dev/null
@@ -1,5 +0,0 @@
-minifi-resources
-Copyright 2014-2016 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/pom.xml b/minifi-framework/minifi-resources/pom.xml
deleted file mode 100644
index 44958a7..0000000
--- a/minifi-framework/minifi-resources/pom.xml
+++ /dev/null
@@ -1,50 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi.minifi</groupId>
-        <artifactId>minifi-framework</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>minifi-resources</artifactId>
-    <packaging>pom</packaging>
-    <description>holds common resources used to build installers</description>
-    <build>
-        <plugins>
-            <plugin>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <configuration>
-                    <attach>true</attach>
-                </configuration>
-                <executions>
-                    <execution>
-                        <id>make shared resource</id>
-                        <goals>
-                            <goal>single</goal>
-                        </goals>
-                        <phase>package</phase>
-                        <configuration>
-                            <descriptors>
-                                <descriptor>src/main/assembly/dependencies.xml</descriptor>
-                            </descriptors>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml b/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml
deleted file mode 100644
index 7c4188d..0000000
--- a/minifi-framework/minifi-resources/src/main/assembly/dependencies.xml
+++ /dev/null
@@ -1,52 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  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.
--->
-<assembly>
-    <id>resources</id>
-    <formats>
-        <format>zip</format>
-    </formats>
-    <includeBaseDirectory>false</includeBaseDirectory>
-    <fileSets>
-        <fileSet>
-            <directory>src/main/resources</directory>
-            <outputDirectory>/</outputDirectory>
-        </fileSet>
-        <fileSet>
-            <directory>src/main/resources/bin</directory>
-            <outputDirectory>/bin/</outputDirectory>
-            <includes>
-                <include>minifi.sh</include>
-            </includes>
-            <fileMode>0750</fileMode>
-        </fileSet>
-    </fileSets>
-    <files>
-        <file>
-            <source>./LICENSE</source>
-            <outputDirectory>./</outputDirectory>
-            <destName>LICENSE</destName>
-            <fileMode>0644</fileMode>
-            <filtered>true</filtered>
-        </file>       
-        <file>
-            <source>./NOTICE</source>
-            <outputDirectory>./</outputDirectory>
-            <destName>NOTICE</destName>
-            <fileMode>0644</fileMode>
-            <filtered>true</filtered>
-        </file>
-    </files>
-</assembly>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat b/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat
deleted file mode 100644
index d55fcfd..0000000
--- a/minifi-framework/minifi-resources/src/main/resources/bin/dump-minifi.bat
+++ /dev/null
@@ -1,47 +0,0 @@
-@echo off
-rem
-rem    Licensed to the Apache Software Foundation (ASF) under one or more
-rem    contributor license agreements.  See the NOTICE file distributed with
-rem    this work for additional information regarding copyright ownership.
-rem    The ASF licenses this file to You under the Apache License, Version 2.0
-rem    (the "License"); you may not use this file except in compliance with
-rem    the License.  You may obtain a copy of the License at
-rem
-rem       http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem    Unless required by applicable law or agreed to in writing, software
-rem    distributed under the License is distributed on an "AS IS" BASIS,
-rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem    See the License for the specific language governing permissions and
-rem    limitations under the License.
-rem
-
-rem Use JAVA_HOME if it's set; otherwise, just use java
-
-if "%JAVA_HOME%" == "" goto noJavaHome
-if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
-set JAVA_EXE=%JAVA_HOME%\bin\java.exe
-goto startMiNifi
-
-:noJavaHome
-echo The JAVA_HOME environment variable is not defined correctly.
-echo Instead the PATH will be used to find the java executable.
-echo.
-set JAVA_EXE=java
-goto startMiNifi
-
-:startMiNifi
-set MINIFI_ROOT=%~dp0..\
-pushd "%MINIFI_ROOT%"
-set LIB_DIR=lib\bootstrap
-set CONF_DIR=conf
-
-set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
-set JAVA_ARGS=-Dorg.apache.nifi.minifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
-
-SET JAVA_PARAMS=-cp %CONF_DIR%;%LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.minifi.bootstrap.RunMiNiFi
-set BOOTSTRAP_ACTION=dump
-
-cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
-
-popd

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh b/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh
deleted file mode 100755
index 51bf67e..0000000
--- a/minifi-framework/minifi-resources/src/main/resources/bin/minifi.sh
+++ /dev/null
@@ -1,247 +0,0 @@
-#!/bin/sh
-#
-#    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.
-#
-# chkconfig: 2345 20 80
-# description: Apache NiFi - MiNiFi
-#
-
-# Script structure inspired from Apache Karaf and other Apache projects with similar startup approaches
-
-SCRIPT_DIR=$(dirname "$0")
-SCRIPT_NAME=$(basename "$0")
-MINIFI_HOME=$(cd "${SCRIPT_DIR}" && cd .. && pwd)
-PROGNAME=$(basename "$0")
-
-
-warn() {
-    echo "${PROGNAME}: $*"
-}
-
-die() {
-    warn "$*"
-    exit 1
-}
-
-detectOS() {
-    # OS specific support (must be 'true' or 'false').
-    cygwin=false;
-    aix=false;
-    os400=false;
-    darwin=false;
-    case "$(uname)" in
-        CYGWIN*)
-            cygwin=true
-            ;;
-        AIX*)
-            aix=true
-            ;;
-        OS400*)
-            os400=true
-            ;;
-        Darwin)
-            darwin=true
-            ;;
-    esac
-    # For AIX, set an environment variable
-    if ${aix}; then
-         export LDR_CNTRL=MAXDATA=0xB0000000@DSA
-         echo ${LDR_CNTRL}
-    fi
-}
-
-unlimitFD() {
-    # Use the maximum available, or set MAX_FD != -1 to use that
-    if [ "x${MAX_FD}" = "x" ]; then
-        MAX_FD="maximum"
-    fi
-
-    # Increase the maximum file descriptors if we can
-    if [ "${os400}" = "false" ] && [ "${cygwin}" = "false" ]; then
-        MAX_FD_LIMIT=$(ulimit -H -n)
-        if [ "${MAX_FD_LIMIT}" != 'unlimited' ]; then
-            if [ $? -eq 0 ]; then
-                if [ "${MAX_FD}" = "maximum" -o "${MAX_FD}" = "max" ]; then
-                    # use the system max
-                    MAX_FD="${MAX_FD_LIMIT}"
-                fi
-
-                ulimit -n ${MAX_FD} > /dev/null
-                # echo "ulimit -n" `ulimit -n`
-                if [ $? -ne 0 ]; then
-                    warn "Could not set maximum file descriptor limit: ${MAX_FD}"
-                fi
-            else
-                warn "Could not query system maximum file descriptor limit: ${MAX_FD_LIMIT}"
-            fi
-        fi
-    fi
-}
-
-
-
-locateJava() {
-    # Setup the Java Virtual Machine
-    if $cygwin ; then
-        [ -n "${JAVA}" ] && JAVA=$(cygpath --unix "${JAVA}")
-        [ -n "${JAVA_HOME}" ] && JAVA_HOME=$(cygpath --unix "${JAVA_HOME}")
-    fi
-
-    if [ "x${JAVA}" = "x" ] && [ -r /etc/gentoo-release ] ; then
-        JAVA_HOME=$(java-config --jre-home)
-    fi
-    if [ "x${JAVA}" = "x" ]; then
-        if [ "x${JAVA_HOME}" != "x" ]; then
-            if [ ! -d "${JAVA_HOME}" ]; then
-                die "JAVA_HOME is not valid: ${JAVA_HOME}"
-            fi
-            JAVA="${JAVA_HOME}/bin/java"
-        else
-            warn "JAVA_HOME not set; results may vary"
-            JAVA=$(type java)
-            JAVA=$(expr "${JAVA}" : '.* \(/.*\)$')
-            if [ "x${JAVA}" = "x" ]; then
-                die "java command not found"
-            fi
-        fi
-    fi
-    # if command is env, attempt to add more to the classpath
-    if [ "$1" = "env" ]; then
-        [ "x${TOOLS_JAR}" =  "x" ] && [ -n "${JAVA_HOME}" ] && TOOLS_JAR=$(find -H "${JAVA_HOME}" -name "tools.jar")
-        [ "x${TOOLS_JAR}" =  "x" ] && [ -n "${JAVA_HOME}" ] && TOOLS_JAR=$(find -H "${JAVA_HOME}" -name "classes.jar")
-        if [ "x${TOOLS_JAR}" =  "x" ]; then
-             warn "Could not locate tools.jar or classes.jar. Please set manually to avail all command features."
-        fi
-    fi
-
-}
-
-init() {
-    # Determine if there is special OS handling we must perform
-    detectOS
-
-    # Unlimit the number of file descriptors if possible
-    unlimitFD
-
-    # Locate the Java VM to execute
-    locateJava "$1"
-}
-
-
-install() {
-        SVC_NAME=minifi
-        if [ "x$2" != "x" ] ; then
-                SVC_NAME=$2
-        fi
-
-        SVC_FILE="/etc/init.d/${SVC_NAME}"
-        cp "$0" "${SVC_FILE}"
-        sed -i s:MINIFI_HOME=.*:MINIFI_HOME="${MINIFI_HOME}": "${SVC_FILE}"
-        sed -i s:PROGNAME=.*:PROGNAME="${SCRIPT_NAME}": "${SVC_FILE}"
-        rm -f "/etc/rc2.d/S65${SVC_NAME}"
-        ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/S65${SVC_NAME}"
-        rm -f "/etc/rc2.d/K65${SVC_NAME}"
-        ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/K65${SVC_NAME}"
-        echo "Service ${SVC_NAME} installed"
-}
-
-
-run() {
-    BOOTSTRAP_CONF_DIR="${MINIFI_HOME}/conf"
-    BOOTSTRAP_CONF="${BOOTSTRAP_CONF_DIR}/bootstrap.conf";
-    BOOTSTRAP_LIBS="${MINIFI_HOME}/lib/bootstrap/*"
-
-    run_as=$(grep run.as "${BOOTSTRAP_CONF}" | cut -d'=' -f2)
-    # If the run as user is the same as that starting the process, ignore this configuration
-    if [ "$run_as" = "$(whoami)" ]; then
-        unset run_as
-    fi
-
-    sudo_cmd_prefix=""
-    if $cygwin; then
-        if [ -n "${run_as}" ]; then
-            echo "The run.as option is not supported in a Cygwin environment. Exiting."
-            exit 1
-        fi;
-
-        MINIFI_HOME=$(cygpath --path --windows "${MINIFI_HOME}")
-        BOOTSTRAP_CONF=$(cygpath --path --windows "${BOOTSTRAP_CONF}")
-        BOOTSTRAP_CONF_DIR=$(cygpath --path --windows "${BOOTSTRAP_CONF_DIR}")
-        BOOTSTRAP_LIBS=$(cygpath --path --windows "${BOOTSTRAP_LIBS}")
-        BOOTSTRAP_CLASSPATH="${BOOTSTRAP_CONF_DIR};${BOOTSTRAP_LIBS}"
-        if [ -n "${TOOLS_JAR}" ]; then
-            TOOLS_JAR=$(cygpath --path --windows "${TOOLS_JAR}")
-            BOOTSTRAP_CLASSPATH="${TOOLS_JAR};${BOOTSTRAP_CLASSPATH}"
-        fi
-    else
-        if [ -n "${run_as}" ]; then
-            if id -u "${run_as}" >/dev/null 2>&1; then
-                sudo_cmd_prefix="sudo -u ${run_as}"
-            else
-                echo "The specified run.as user ${run_as} does not exist. Exiting."
-                exit 1
-            fi
-        fi;
-        BOOTSTRAP_CLASSPATH="${BOOTSTRAP_CONF_DIR}:${BOOTSTRAP_LIBS}"
-        if [ -n "${TOOLS_JAR}" ]; then
-            BOOTSTRAP_CLASSPATH="${TOOLS_JAR}:${BOOTSTRAP_CLASSPATH}"
-        fi
-    fi
-
-    echo
-    echo "Java home: ${JAVA_HOME}"
-    echo "MiNiFi home: ${MINIFI_HOME}"
-    echo
-    echo "Bootstrap Config File: ${BOOTSTRAP_CONF}"
-    echo
-
-    # run 'start' in the background because the process will continue to run, monitoring MiNiFi.
-    # all other commands will terminate quickly so want to just wait for them
-    if [ "$1" = "start" ]; then
-        (cd "${MINIFI_HOME}" && ${sudo_cmd_prefix} "${JAVA}" -cp "${BOOTSTRAP_CLASSPATH}" -Xms12m -Xmx24m -Dorg.apache.nifi.minifi.bootstrap.config.file="${BOOTSTRAP_CONF}" org.apache.nifi.minifi.bootstrap.RunMiNiFi $@ &)
-    else
-        (cd "${MINIFI_HOME}" && ${sudo_cmd_prefix} "${JAVA}" -cp "${BOOTSTRAP_CLASSPATH}" -Xms12m -Xmx24m -Dorg.apache.nifi.minifi.bootstrap.config.file="${BOOTSTRAP_CONF}" org.apache.nifi.minifi.bootstrap.RunMiNiFi $@)
-    fi
-
-    # Wait just a bit (3 secs) to wait for the logging to finish and then echo a new-line.
-    # We do this to avoid having logs spewed on the console after running the command and then not giving
-    # control back to the user
-    sleep 3
-    echo
-}
-
-main() {
-    init "$1"
-    run "$@"
-}
-
-
-case "$1" in
-    install)
-        install "$@"
-        ;;
-    start|stop|run|status|dump|env)
-        main "$@"
-        ;;
-    restart)
-        init
-    run "stop"
-    run "start"
-    ;;
-    *)
-        echo "Usage minifi {start|stop|run|restart|status|dump|install}"
-        ;;
-esac

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat b/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat
deleted file mode 100644
index 8750af6..0000000
--- a/minifi-framework/minifi-resources/src/main/resources/bin/run-minifi.bat
+++ /dev/null
@@ -1,47 +0,0 @@
-@echo off
-rem
-rem    Licensed to the Apache Software Foundation (ASF) under one or more
-rem    contributor license agreements.  See the NOTICE file distributed with
-rem    this work for additional information regarding copyright ownership.
-rem    The ASF licenses this file to You under the Apache License, Version 2.0
-rem    (the "License"); you may not use this file except in compliance with
-rem    the License.  You may obtain a copy of the License at
-rem
-rem       http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem    Unless required by applicable law or agreed to in writing, software
-rem    distributed under the License is distributed on an "AS IS" BASIS,
-rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem    See the License for the specific language governing permissions and
-rem    limitations under the License.
-rem
-
-rem Use JAVA_HOME if it's set; otherwise, just use java
-
-if "%JAVA_HOME%" == "" goto noJavaHome
-if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
-set JAVA_EXE=%JAVA_HOME%\bin\java.exe
-goto startMiNifi
-
-:noJavaHome
-echo The JAVA_HOME environment variable is not defined correctly.
-echo Instead the PATH will be used to find the java executable.
-echo.
-set JAVA_EXE=java
-goto startMiNifi
-
-:startMiNifi
-set MiNIFI_ROOT=%~dp0..\
-pushd "%MiNIFI_ROOT%"
-set LIB_DIR=lib\bootstrap
-set CONF_DIR=conf
-
-set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
-set JAVA_ARGS=-Dorg.apache.nifi.minifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
-
-SET JAVA_PARAMS=-cp %CONF_DIR%;%LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.minifi.bootstrap.RunMiNiFi
-set BOOTSTRAP_ACTION=run
-
-cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
-
-popd

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat b/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat
deleted file mode 100644
index 8551baa..0000000
--- a/minifi-framework/minifi-resources/src/main/resources/bin/status-minifi.bat
+++ /dev/null
@@ -1,47 +0,0 @@
-@echo off
-rem
-rem    Licensed to the Apache Software Foundation (ASF) under one or more
-rem    contributor license agreements.  See the NOTICE file distributed with
-rem    this work for additional information regarding copyright ownership.
-rem    The ASF licenses this file to You under the Apache License, Version 2.0
-rem    (the "License"); you may not use this file except in compliance with
-rem    the License.  You may obtain a copy of the License at
-rem
-rem       http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem    Unless required by applicable law or agreed to in writing, software
-rem    distributed under the License is distributed on an "AS IS" BASIS,
-rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem    See the License for the specific language governing permissions and
-rem    limitations under the License.
-rem
-
-rem Use JAVA_HOME if it's set; otherwise, just use java
-
-if "%JAVA_HOME%" == "" goto noJavaHome
-if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
-set JAVA_EXE=%JAVA_HOME%\bin\java.exe
-goto startMiNifi
-
-:noJavaHome
-echo The JAVA_HOME environment variable is not defined correctly.
-echo Instead the PATH will be used to find the java executable.
-echo.
-set JAVA_EXE=java
-goto startMiNifi
-
-:startMiNifi
-set MINIFI_ROOT=%~dp0..\
-pushd "%MINIFI_ROOT%"
-set LIB_DIR=lib\bootstrap
-set CONF_DIR=conf
-
-set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
-set JAVA_ARGS=-Dorg.apache.nifi.minifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
-
-set JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.minifi.bootstrap.RunNiFiMi
-set BOOTSTRAP_ACTION=status
-
-cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
-
-popd
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf b/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
deleted file mode 100644
index 5e676ff..0000000
--- a/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
+++ /dev/null
@@ -1,60 +0,0 @@
-#
-# 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.
-#
-
-# Java command to use when running MiNiFi
-java=java
-
-# Username to use when running MiNiFi. This value will be ignored on Windows.
-run.as=${minifi.run.as}
-
-# Configure where MiNiFi's lib and conf directories live
-lib.dir=./lib
-conf.dir=./conf
-
-# How long to wait after telling MiNiFi to shutdown before explicitly killing the Process
-graceful.shutdown.seconds=20
-
-# Disable JSR 199 so that we can use JSP's without running a JDK
-java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
-
-# JVM memory settings
-java.arg.2=-Xms${minifi.jvm.heap.mb}m
-java.arg.3=-Xmx${minifi.jvm.heap.mb}m
-
-# Enable Remote Debugging
-#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
-
-java.arg.4=-Djava.net.preferIPv4Stack=true
-
-# allowRestrictedHeaders is required for Cluster/Node communications to work properly
-java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
-java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
-
-# Java 7 and below have issues with Code Cache. The following lines allow us to run well even with
-# many classes loaded in the JVM.
-#java.arg.7=-XX:ReservedCodeCacheSize=256m
-#java.arg.8=-XX:CodeCacheFlushingMinimumFreeSpace=10m
-#java.arg.9=-XX:+UseCodeCacheFlushing
-#java.arg.11=-XX:PermSize=${minifi.jvm.permgen.mb}M
-#java.arg.12=-XX:MaxPermSize=${minifi.jvm.permgen.mb}M
-
-# The G1GC is still considered experimental but has proven to be very advantageous in providing great
-# performance without significant "stop-the-world" delays.
-#java.arg.13=-XX:+UseG1GC
-
-#Set headless mode by default
-java.arg.14=-Djava.awt.headless=true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/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
deleted file mode 100644
index e071858..0000000
--- a/minifi-framework/minifi-resources/src/main/resources/conf/logback.xml
+++ /dev/null
@@ -1,150 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
-<configuration scan="true" scanPeriod="30 seconds">
-    <contextListener class="ch.qos.logback.classic.jul.LevelChangePropagator">
-        <resetJUL>true</resetJUL>
-    </contextListener>
-    
-    <appender name="APP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/minifi-app.log</file>
-        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
-            <!--
-              For daily rollover, use 'app_%d.log'.
-              For hourly rollover, use 'app_%d{yyyy-MM-dd_HH}.log'.
-              To GZIP rolled files, replace '.log' with '.log.gz'.
-              To ZIP rolled files, replace '.log' with '.log.zip'.
-            -->
-            <fileNamePattern>./logs/minifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
-            <timeBasedFileNamingAndTriggeringPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedFNATP">
-                <maxFileSize>100MB</maxFileSize>
-            </timeBasedFileNamingAndTriggeringPolicy>
-            <!-- keep 30 log files worth of history -->
-            <maxHistory>30</maxHistory>
-        </rollingPolicy>
-        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
-            <immediateFlush>true</immediateFlush>
-        </encoder>
-    </appender>
-    
-    <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/minifi-user.log</file>
-        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
-            <!--
-              For daily rollover, use 'user_%d.log'.
-              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
-              To GZIP rolled files, replace '.log' with '.log.gz'.
-              To ZIP rolled files, replace '.log' with '.log.zip'.
-            -->
-            <fileNamePattern>./logs/minifi-user_%d.log</fileNamePattern>
-            <!-- keep 30 log files worth of history -->
-            <maxHistory>30</maxHistory>
-        </rollingPolicy>
-        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/minifi-bootstrap.log</file>
-        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
-            <!--
-              For daily rollover, use 'user_%d.log'.
-              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
-              To GZIP rolled files, replace '.log' with '.log.gz'.
-              To ZIP rolled files, replace '.log' with '.log.zip'.
-            -->
-            <fileNamePattern>./logs/minifi-bootstrap_%d.log</fileNamePattern>
-            <!-- keep 5 log files worth of history -->
-            <maxHistory>5</maxHistory>
-        </rollingPolicy>
-        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
-        </encoder>
-    </appender>
-	
-	<appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
-		<encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
-        </encoder>
-	</appender>
-    
-    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
-    
-    <logger name="org.apache.nifi" 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.server.JettyServer" level="INFO"/>
-
-    <!-- Logger for managing logging statements for jetty -->
-    <logger name="org.eclipse.jetty" level="INFO"/>
-
-    <!-- Suppress non-error messages due to excessive logging by class or library -->
-    <logger name="com.sun.jersey.spi.container.servlet.WebComponent" level="ERROR"/>
-    <logger name="com.sun.jersey.spi.spring" level="ERROR"/>
-    <logger name="org.springframework" level="ERROR"/>
-    
-    <!-- Suppress non-error messages due to known warning about redundant path annotation (NIFI-574) -->
-    <logger name="com.sun.jersey.spi.inject.Errors" level="ERROR"/>
-
-    <!--
-        Logger for capturing user events. We do not want to propagate these
-        log events to the root logger. These messages are only sent to the
-        user-log appender.
-    -->
-    <logger name="org.apache.nifi.web.security" level="INFO" additivity="false">
-        <appender-ref ref="USER_FILE"/>
-    </logger>
-    <logger name="org.apache.nifi.web.api.config" level="INFO" additivity="false">
-        <appender-ref ref="USER_FILE"/>
-    </logger>
-    <logger name="org.apache.nifi.authorization" level="INFO" additivity="false">
-        <appender-ref ref="USER_FILE"/>
-    </logger>
-    <logger name="org.apache.nifi.web.filter.RequestLogger" level="INFO" additivity="false">
-        <appender-ref ref="USER_FILE"/>
-    </logger>
-
-
-    <!--
-        Logger for capturing Bootstrap logs and MiNiFi's standard error and standard out.
-    -->
-    <logger name="org.apache.nifi.minifi.bootstrap" level="INFO" additivity="false">
-        <appender-ref ref="BOOTSTRAP_FILE" />
-    </logger>
-    <logger name="org.apache.nifi.minifi.bootstrap.Command" level="INFO" additivity="false">
-        <appender-ref ref="CONSOLE" />
-        <appender-ref ref="BOOTSTRAP_FILE" />
-    </logger>
-
-    <!-- Everything written to MiNiFi's Standard Out will be logged with the logger org.apache.nifi.minifi.StdOut at INFO level -->
-    <logger name="org.apache.nifi.minifi.StdOut" level="INFO" additivity="false">
-        <appender-ref ref="BOOTSTRAP_FILE" />
-    </logger>
-    
-    <!-- Everything written to MiNiFi's Standard Error will be logged with the logger org.apache.nifi.minifi.StdErr at ERROR level -->
-	<logger name="org.apache.nifi.minifi.StdErr" level="ERROR" additivity="false">
-    	<appender-ref ref="BOOTSTRAP_FILE" />
-    </logger>
-
-
-    <root level="INFO">
-        <appender-ref ref="APP_FILE"/>
-    </root>
-    
-</configuration>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-framework/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-framework/pom.xml b/minifi-framework/pom.xml
deleted file mode 100644
index c602931..0000000
--- a/minifi-framework/pom.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
- -->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <artifactId>minifi</artifactId>
-        <groupId>org.apache.nifi.minifi</groupId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <artifactId>minifi-framework</artifactId>
-    <packaging>pom</packaging>
-
-    <modules>
-        <module>minifi-resources</module>
-    </modules>
-
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/5751e23d/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/pom.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/pom.xml
new file mode 100644
index 0000000..cd78d0b
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/pom.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>minifi-framework-bundle</artifactId>
+        <groupId>org.apache.nifi.minifi</groupId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.nifi.minifi</groupId>
+    <artifactId>minifi-framework-nar</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>nar</packaging>
+
+    <description>MiNiFi: Framework NAR</description>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+    <dependencies>
+        <!-- mark these nifi artifacts as provided since it is included in the lib -->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-runtime</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-nar-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file