You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jp...@apache.org on 2016/11/30 22:12:44 UTC

[1/3] nifi-minifi git commit: MINIFI-142 Upgrading NiFi dependencies to 1.1.0 and adjusting NAR and ClassLoader code to reflect changes introduced in NiFi.

Repository: nifi-minifi
Updated Branches:
  refs/heads/master bcf6c6cb2 -> 967d40243


MINIFI-142 Upgrading NiFi dependencies to 1.1.0 and adjusting NAR and ClassLoader code to reflect changes introduced in NiFi.

Signed-off-by: Joseph Percivall <JP...@apache.org>


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

Branch: refs/heads/master
Commit: 85d69eadae5bffb570a28fcb4a05649d58605ce6
Parents: bcf6c6c
Author: Aldrin Piri <al...@apache.org>
Authored: Tue Nov 29 16:36:46 2016 -0500
Committer: Joseph Percivall <JP...@apache.org>
Committed: Wed Nov 30 17:13:13 2016 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/nar/ExtensionManager.java   | 103 ++++++++++--
 .../apache/nifi/nar/InstanceClassLoader.java    | 160 +++++++++++++++++++
 .../java/org/apache/nifi/nar/NarCloseable.java  |  27 ++--
 .../nifi/nar/NarThreadContextClassLoader.java   |  22 ++-
 .../java/org/apache/nifi/nar/NarUnpacker.java   |  28 ++--
 .../java/org/apache/nifi/util/FileUtils.java    | 105 ++++--------
 .../pom.xml                                     |   5 +
 .../MiNiFiPersistentProvenanceRepository.java   |  52 +++---
 ...estMiNiFiPersistentProvenanceRepository.java | 156 +++---------------
 pom.xml                                         |   2 +-
 10 files changed, 385 insertions(+), 275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
index 787fb3c..9fd9e66 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
@@ -16,14 +16,8 @@
  */
 package org.apache.nifi.nar;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.ServiceLoader;
-import java.util.Set;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
 import org.apache.nifi.authentication.LoginIdentityProvider;
-
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.repository.ContentRepository;
@@ -34,10 +28,21 @@ import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.provenance.ProvenanceRepository;
 import org.apache.nifi.reporting.ReportingTask;
-
+import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
 /**
  * Scans through the classpath to load all FlowFileProcessors, FlowFileComparators, and ReportingTasks using the service provider API and running through all classloaders (root, NARs).
  *
@@ -53,6 +58,9 @@ public class ExtensionManager {
 
     private static final Map<String, ClassLoader> extensionClassloaderLookup = new HashMap<>();
 
+    private static final Set<String> requiresInstanceClassLoading = new HashSet<>();
+    private static final Map<String, ClassLoader> instanceClassloaderLookup = new ConcurrentHashMap<>();
+
     static {
         definitionMap.put(Processor.class, new HashSet<>());
         definitionMap.put(FlowFilePrioritizer.class, new HashSet<>());
@@ -127,6 +135,12 @@ public class ExtensionManager {
         if (registeredClassLoader == null) {
             classloaderMap.put(className, classLoader);
             classes.add(type);
+
+            // keep track of which classes require a class loader per component instance
+            if (type.isAnnotationPresent(RequiresInstanceClassLoading.class)) {
+                requiresInstanceClassLoading.add(className);
+            }
+
         } else {
             boolean loadedFromAncestor = false;
 
@@ -159,6 +173,77 @@ public class ExtensionManager {
         return extensionClassloaderLookup.get(classType);
     }
 
+    /**
+     * Determines the effective ClassLoader for the instance of the given type.
+     *
+     * @param classType the type of class to lookup the ClassLoader for
+     * @param instanceIdentifier the identifier of the specific instance of the classType to look up the ClassLoader for
+     * @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type
+     */
+    public static ClassLoader getClassLoader(final String classType, final String instanceIdentifier) {
+        if (StringUtils.isEmpty(classType) || StringUtils.isEmpty(instanceIdentifier)) {
+            throw new IllegalArgumentException("Class Type and Instance Identifier must be provided");
+        }
+
+        // Check if we already have a ClassLoader for this instance
+        ClassLoader instanceClassLoader = instanceClassloaderLookup.get(instanceIdentifier);
+
+        // If we don't then we'll create a new ClassLoader for this instance and add it to the map for future lookups
+        if (instanceClassLoader == null) {
+            final ClassLoader registeredClassLoader = getClassLoader(classType);
+            if (registeredClassLoader == null) {
+                return null;
+            }
+
+            // If the class is annotated with @RequiresInstanceClassLoading and the registered ClassLoader is a URLClassLoader
+            // then make a new InstanceClassLoader that is a full copy of the NAR Class Loader, otherwise create an empty
+            // InstanceClassLoader that has the NAR ClassLoader as a parent
+            if (requiresInstanceClassLoading.contains(classType) && (registeredClassLoader instanceof URLClassLoader)) {
+                final URLClassLoader registeredUrlClassLoader = (URLClassLoader) registeredClassLoader;
+                instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, registeredUrlClassLoader.getURLs(), registeredUrlClassLoader.getParent());
+            } else {
+                instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, new URL[0], registeredClassLoader);
+            }
+
+            instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader);
+        }
+
+        return instanceClassLoader;
+    }
+
+    /**
+     * Removes the ClassLoader for the given instance and closes it if necessary.
+     *
+     * @param instanceIdentifier the identifier of a component to remove the ClassLoader for
+     * @return the removed ClassLoader for the given instance, or null if not found
+     */
+    public static ClassLoader removeInstanceClassLoaderIfExists(final String instanceIdentifier) {
+        if (instanceIdentifier == null) {
+            return null;
+        }
+
+        final ClassLoader classLoader = instanceClassloaderLookup.remove(instanceIdentifier);
+        if (classLoader != null && (classLoader instanceof URLClassLoader)) {
+            final URLClassLoader urlClassLoader = (URLClassLoader) classLoader;
+            try {
+                urlClassLoader.close();
+            } catch (IOException e) {
+                logger.warn("Unable to class URLClassLoader for " + instanceIdentifier);
+            }
+        }
+        return classLoader;
+    }
+
+    /**
+     * Checks if the given class type requires per-instance class loading (i.e. contains the @RequiresInstanceClassLoading annotation)
+     *
+     * @param classType the class to check
+     * @return true if the class is found in the set of classes requiring instance level class loading, false otherwise
+     */
+    public static boolean requiresInstanceClassLoading(final String classType) {
+        return requiresInstanceClassLoading.contains(classType);
+    }
+
     public static Set<Class> getExtensions(final Class<?> definition) {
         final Set<Class> extensions = definitionMap.get(definition);
         return (extensions == null) ? Collections.<Class>emptySet() : extensions;
@@ -180,4 +265,4 @@ public class ExtensionManager {
 
         logger.info(builder.toString());
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
new file mode 100644
index 0000000..8aff08f
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
@@ -0,0 +1,160 @@
+/*
+ * 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.nar;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+/**
+ * A ClassLoader created for an instance of a component which lets a client add resources to an intermediary ClassLoader
+ * that will be checked first when loading/finding classes.
+ *
+ * Typically an instance of this ClassLoader will be created by passing in the URLs and parent from a NARClassLoader in
+ * order to create a copy of the NARClassLoader without modifying it.
+ */
+public class InstanceClassLoader extends URLClassLoader {
+
+    private static final Logger logger = LoggerFactory.getLogger(InstanceClassLoader.class);
+
+    private final String identifier;
+    private final String instanceType;
+    private ShimClassLoader shimClassLoader;
+
+    /**
+     * @param identifier the id of the component this ClassLoader was created for
+     * @param urls the URLs for the ClassLoader
+     * @param parent the parent ClassLoader
+     */
+    public InstanceClassLoader(final String identifier, final String type, final URL[] urls, final ClassLoader parent) {
+        super(urls, parent);
+        this.identifier = identifier;
+        this.instanceType = type;
+    }
+
+    /**
+     * Initializes a new ShimClassLoader for the provided resources, closing the previous ShimClassLoader if one existed.
+     *
+     * @param urls the URLs for the ShimClassLoader
+     * @throws IOException if the previous ShimClassLoader existed and couldn't be closed
+     */
+    public synchronized void setInstanceResources(final URL[] urls) {
+        if (shimClassLoader != null) {
+            try {
+                shimClassLoader.close();
+            } catch (IOException e) {
+                logger.warn("Unable to close inner URLClassLoader for " + identifier);
+            }
+        }
+
+        shimClassLoader = new ShimClassLoader(urls, getParent());
+    }
+
+    /**
+     * @return the URLs for the instance resources that have been set
+     */
+    public synchronized URL[] getInstanceResources() {
+        if (shimClassLoader != null) {
+            return shimClassLoader.getURLs();
+        }
+        return new URL[0];
+    }
+
+    @Override
+    public Class<?> loadClass(String name) throws ClassNotFoundException {
+        return this.loadClass(name, false);
+    }
+
+    @Override
+    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+        Class<?> c = null;
+        // first try the shim
+        if (shimClassLoader != null) {
+            try {
+                c = shimClassLoader.loadClass(name, resolve);
+            } catch (ClassNotFoundException e) {
+                c = null;
+            }
+        }
+        // if it wasn't in the shim try our self
+        if (c == null) {
+            return super.loadClass(name, resolve);
+        } else {
+            return c;
+        }
+    }
+
+    @Override
+    protected Class<?> findClass(String name) throws ClassNotFoundException {
+        Class<?> c = null;
+        // first try the shim
+        if (shimClassLoader != null) {
+            try {
+                c = shimClassLoader.findClass(name);
+            } catch (ClassNotFoundException cnf) {
+                c = null;
+            }
+        }
+        // if it wasn't in the shim try our self
+        if (c == null) {
+            return super.findClass(name);
+        } else {
+            return c;
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (shimClassLoader != null) {
+            try {
+                shimClassLoader.close();
+            } catch (IOException e) {
+                logger.warn("Unable to close inner URLClassLoader for " + identifier);
+            }
+        }
+        super.close();
+    }
+
+    /**
+     * Extend URLClassLoader to increase visibility of protected methods so that InstanceClassLoader can delegate.
+     */
+    private static class ShimClassLoader extends URLClassLoader {
+
+        public ShimClassLoader(URL[] urls, ClassLoader parent) {
+            super(urls, parent);
+        }
+
+        public ShimClassLoader(URL[] urls) {
+            super(urls);
+        }
+
+        @Override
+        public Class<?> findClass(String name) throws ClassNotFoundException {
+            return super.findClass(name);
+        }
+
+        @Override
+        public Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+            return super.loadClass(name, resolve);
+        }
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
index 639d032..56aff9e 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
@@ -35,18 +35,25 @@ public class NarCloseable implements Closeable {
     }
 
     /**
-     * Sets the current thread context class loader to the specific appropriate
-     * Nar class loader for the given configurable component. Restores to the
-     * previous classloader once complete. If the given class is not assignable
-     * from ConfigurableComponent then the NarThreadContextClassLoader is used.
+     * Sets the current thread context class loader to the specific appropriate class loader for the given
+     * component. If the component requires per-instance class loading then the class loader will be the
+     * specific class loader for instance with the given identifier, otherwise the class loader will be
+     * the NARClassLoader.
      *
-     * @param componentClass componentClass
-     * @return NarCloseable with current thread context classloader jailed to
-     * the nar of the component
+     * @param componentClass the component class
+     * @param componentIdentifier the identifier of the component
+     * @return NarCloseable with the current thread context classloader jailed to the Nar
+     *              or instance class loader of the component
      */
-    public static NarCloseable withComponentNarLoader(final Class componentClass) {
+    public static NarCloseable withComponentNarLoader(final Class componentClass, final String componentIdentifier) {
         final ClassLoader current = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(componentClass.getClassLoader());
+
+        ClassLoader componentClassLoader = ExtensionManager.getClassLoader(componentClass.getName(), componentIdentifier);
+        if (componentClassLoader == null) {
+            componentClassLoader = componentClass.getClassLoader();
+        }
+
+        Thread.currentThread().setContextClassLoader(componentClassLoader);
         return new NarCloseable(current);
     }
 
@@ -88,4 +95,4 @@ public class NarCloseable implements Closeable {
             Thread.currentThread().setContextClassLoader(toSet);
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
index 5c70927..6a66ba2 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -203,14 +203,28 @@ public class NarThreadContextClassLoader extends URLClassLoader {
                 return typeDefinition.cast(desiredClass.newInstance());
             }
             Constructor<?> constructor = null;
+
             try {
                 constructor = desiredClass.getConstructor(NiFiProperties.class);
-                return typeDefinition.cast(constructor.newInstance(nifiProperties));
-            } catch (final NoSuchMethodException | InvocationTargetException ex) {
-                return typeDefinition.cast(desiredClass.newInstance());
+            } catch (NoSuchMethodException nsme) {
+                try {
+                    constructor = desiredClass.getConstructor();
+                } catch (NoSuchMethodException nsme2) {
+                    throw new IllegalStateException("Failed to find constructor which takes NiFiProperties as argument as well as the default constructor on "
+                            + desiredClass.getName(), nsme2);
+                }
+            }
+            try {
+                if (constructor.getParameterTypes().length == 0) {
+                    return typeDefinition.cast(constructor.newInstance());
+                } else {
+                    return typeDefinition.cast(constructor.newInstance(nifiProperties));
+                }
+            } catch (InvocationTargetException ite) {
+                throw new IllegalStateException("Failed to instantiate a component due to (see target exception)", ite);
             }
         } finally {
             Thread.currentThread().setContextClassLoader(originalClassLoader);
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
index 2af1090..13108c1 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
@@ -50,7 +50,7 @@ import java.util.jar.Manifest;
  */
 public final class NarUnpacker {
 
-    private static final Logger logger = LoggerFactory.getLogger(org.apache.nifi.nar.NarUnpacker.class);
+    private static final Logger logger = LoggerFactory.getLogger(NarUnpacker.class);
     private static String HASH_FILENAME = "nar-md5sum";
     private static final FileFilter NAR_FILTER = new FileFilter() {
         @Override
@@ -72,14 +72,16 @@ public final class NarUnpacker {
             final List<File> narFiles = new ArrayList<>();
 
             // make sure the nar directories are there and accessible
-            FileUtils.ensureDirectoryExistAndCanAccess(frameworkWorkingDir);
-            FileUtils.ensureDirectoryExistAndCanAccess(extensionsWorkingDir);
-            FileUtils.ensureDirectoryExistAndCanAccess(docsWorkingDir);
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(frameworkWorkingDir);
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(extensionsWorkingDir);
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(docsWorkingDir);
 
             for (Path narLibraryDir : narLibraryDirs) {
 
                 File narDir = narLibraryDir.toFile();
-                FileUtils.ensureDirectoryExistAndCanAccess(narDir);
+
+                // Test if the source NARs can be read
+                FileUtils.ensureDirectoryExistAndCanRead(narDir);
 
                 File[] dirFiles = narDir.listFiles(NAR_FILTER);
                 if (dirFiles != null) {
@@ -170,7 +172,7 @@ public final class NarUnpacker {
     }
 
     private static void mapExtensions(final File workingDirectory, final File docsDirectory,
-            final ExtensionMapping mapping) throws IOException {
+                                      final ExtensionMapping mapping) throws IOException {
         final File[] directoryContents = workingDirectory.listFiles();
         if (directoryContents != null) {
             for (final File file : directoryContents) {
@@ -256,7 +258,7 @@ public final class NarUnpacker {
     }
 
     private static void unpackDocumentation(final File jar, final File docsDirectory,
-            final ExtensionMapping extensionMapping) throws IOException {
+                                            final ExtensionMapping extensionMapping) throws IOException {
         // determine the components that may have documentation
         determineDocumentedNiFiComponents(jar, extensionMapping);
 
@@ -298,7 +300,7 @@ public final class NarUnpacker {
     }
 
     private static void determineDocumentedNiFiComponents(final File jar,
-            final ExtensionMapping extensionMapping) throws IOException {
+                                                          final ExtensionMapping extensionMapping) throws IOException {
         try (final JarFile jarFile = new JarFile(jar)) {
             final JarEntry processorEntry = jarFile
                     .getJarEntry("META-INF/services/org.apache.nifi.processor.Processor");
@@ -317,7 +319,7 @@ public final class NarUnpacker {
     }
 
     private static List<String> determineDocumentedNiFiComponents(final JarFile jarFile,
-            final JarEntry jarEntry) throws IOException {
+                                                                  final JarEntry jarEntry) throws IOException {
         final List<String> componentNames = new ArrayList<>();
 
         if (jarEntry == null) {
@@ -325,8 +327,8 @@ public final class NarUnpacker {
         }
 
         try (final InputStream entryInputStream = jarFile.getInputStream(jarEntry);
-                final BufferedReader reader = new BufferedReader(new InputStreamReader(
-                        entryInputStream))) {
+             final BufferedReader reader = new BufferedReader(new InputStreamReader(
+                     entryInputStream))) {
             String line;
             while ((line = reader.readLine()) != null) {
                 final String trimmedLine = line.trim();
@@ -355,7 +357,7 @@ public final class NarUnpacker {
      */
     private static void makeFile(final InputStream inputStream, final File file) throws IOException {
         try (final InputStream in = inputStream;
-                final FileOutputStream fos = new FileOutputStream(file)) {
+             final FileOutputStream fos = new FileOutputStream(file)) {
             byte[] bytes = new byte[65536];
             int numRead;
             while ((numRead = in.read(bytes)) != -1) {
@@ -393,4 +395,4 @@ public final class NarUnpacker {
 
     private NarUnpacker() {
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java
index 5462f23..5e8a3c3 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java
@@ -33,7 +33,13 @@ public class FileUtils {
 
     public static final long MILLIS_BETWEEN_ATTEMPTS = 50L;
 
+    /* Superseded by renamed class bellow */
+    @Deprecated
     public static void ensureDirectoryExistAndCanAccess(final File dir) throws IOException {
+        ensureDirectoryExistAndCanReadAndWrite(dir);
+    }
+
+    public static void ensureDirectoryExistAndCanReadAndWrite(final File dir) throws IOException {
         if (dir.exists() && !dir.isDirectory()) {
             throw new IOException(dir.getAbsolutePath() + " is not a directory");
         } else if (!dir.exists()) {
@@ -47,6 +53,20 @@ public class FileUtils {
         }
     }
 
+    public static void ensureDirectoryExistAndCanRead(final File dir) throws IOException {
+        if (dir.exists() && !dir.isDirectory()) {
+            throw new IOException(dir.getAbsolutePath() + " is not a directory");
+        } else if (!dir.exists()) {
+            final boolean made = dir.mkdirs();
+            if (!made) {
+                throw new IOException(dir.getAbsolutePath() + " could not be created");
+            }
+        }
+        if (!dir.canRead()) {
+            throw new IOException(dir.getAbsolutePath() + " directory does not have read privilege");
+        }
+    }
+
     /**
      * Deletes the given file. If the given file exists but could not be deleted
      * this will be printed as a warning to the given logger
@@ -103,79 +123,8 @@ public class FileUtils {
      * @param directory to delete contents of
      * @param filter if null then no filter is used
      * @param logger to notify
-     * @deprecated As of release 0.6.0, replaced by
-     * {@link #deleteFilesInDirectory(File, FilenameFilter, Logger)}
-     */
-    @Deprecated
-    public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger) {
-        FileUtils.deleteFilesInDir(directory, filter, logger, false);
-    }
-
-    /**
-     * Deletes all files (not directories) in the given directory (recursive)
-     * that match the given filename filter. If any file cannot be deleted then
-     * this is printed at warn to the given logger.
-     *
-     * @param directory to delete contents of
-     * @param filter if null then no filter is used
-     * @param logger to notify
-     * @param recurse true if should recurse
-     * @deprecated As of release 0.6.0, replaced by
-     * {@link #deleteFilesInDirectory(File, FilenameFilter, Logger, boolean)}
-     */
-    @Deprecated
-    public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse) {
-        FileUtils.deleteFilesInDir(directory, filter, logger, recurse, false);
-    }
-
-    /**
-     * Deletes all files (not directories) in the given directory (recursive)
-     * that match the given filename filter. If any file cannot be deleted then
-     * this is printed at warn to the given logger.
-     *
-     * @param directory to delete contents of
-     * @param filter if null then no filter is used
-     * @param logger to notify
-     * @param recurse will look for contents of sub directories.
-     * @param deleteEmptyDirectories default is false; if true will delete
-     * directories found that are empty
-     * @deprecated As of release 0.6.0, replaced by
-     * {@link #deleteFilesInDirectory(File, FilenameFilter, Logger, boolean, boolean)}
-     */
-    @Deprecated
-    public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse, final boolean deleteEmptyDirectories) {
-        // ensure the specified directory is actually a directory and that it exists
-        if (null != directory && directory.isDirectory()) {
-            final File ingestFiles[] = directory.listFiles();
-            if (ingestFiles == null) {
-                // null if abstract pathname does not denote a directory, or if an I/O error occurs
-                logger.error("Unable to list directory content in: " + directory.getAbsolutePath());
-            }
-            for (File ingestFile : ingestFiles) {
-                boolean process = (filter == null) ? true : filter.accept(directory, ingestFile.getName());
-                if (ingestFile.isFile() && process) {
-                    FileUtils.deleteFile(ingestFile, logger, 3);
-                }
-                if (ingestFile.isDirectory() && recurse) {
-                    FileUtils.deleteFilesInDir(ingestFile, filter, logger, recurse, deleteEmptyDirectories);
-                    if (deleteEmptyDirectories && ingestFile.list().length == 0) {
-                        FileUtils.deleteFile(ingestFile, logger, 3);
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Deletes all files (not directories..) in the given directory (non
-     * recursive) that match the given filename filter. If any file cannot be
-     * deleted then this is printed at warn to the given logger.
-     *
-     * @param directory to delete contents of
-     * @param filter if null then no filter is used
-     * @param logger to notify
-     * @throws IOException if abstract pathname does not denote a directory,
-     * or if an I/O error occurs
+     * @throws IOException if abstract pathname does not denote a directory, or
+     * if an I/O error occurs
      */
     public static void deleteFilesInDirectory(final File directory, final FilenameFilter filter, final Logger logger) throws IOException {
         FileUtils.deleteFilesInDirectory(directory, filter, logger, false);
@@ -190,8 +139,8 @@ public class FileUtils {
      * @param filter if null then no filter is used
      * @param logger to notify
      * @param recurse true if should recurse
-     * @throws IOException if abstract pathname does not denote a directory,
-     * or if an I/O error occurs
+     * @throws IOException if abstract pathname does not denote a directory, or
+     * if an I/O error occurs
      */
     public static void deleteFilesInDirectory(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse) throws IOException {
         FileUtils.deleteFilesInDirectory(directory, filter, logger, recurse, false);
@@ -208,8 +157,8 @@ public class FileUtils {
      * @param recurse will look for contents of sub directories.
      * @param deleteEmptyDirectories default is false; if true will delete
      * directories found that are empty
-     * @throws IOException if abstract pathname does not denote a directory,
-     * or if an I/O error occurs
+     * @throws IOException if abstract pathname does not denote a directory, or
+     * if an I/O error occurs
      */
     public static void deleteFilesInDirectory(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse, final boolean deleteEmptyDirectories) throws IOException {
         // ensure the specified directory is actually a directory and that it exists
@@ -265,4 +214,4 @@ public class FileUtils {
             /* do nothing */
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/pom.xml b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/pom.xml
index dca11b0..94108c4 100644
--- a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/pom.xml
+++ b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/pom.xml
@@ -51,5 +51,10 @@ limitations under the License.
             <artifactId>nifi-persistent-provenance-repository</artifactId>
             <scope>compile</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <scope>provided</scope>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/MiNiFiPersistentProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/MiNiFiPersistentProvenanceRepository.java b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/MiNiFiPersistentProvenanceRepository.java
index a015dca..5ce83a6 100644
--- a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/MiNiFiPersistentProvenanceRepository.java
+++ b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/MiNiFiPersistentProvenanceRepository.java
@@ -16,7 +16,29 @@
  */
 package org.apache.nifi.provenance;
 
-import static org.apache.nifi.provenance.toc.TocUtil.getTocFile;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.provenance.expiration.ExpirationAction;
+import org.apache.nifi.provenance.expiration.FileRemovalAction;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.apache.nifi.provenance.serialization.RecordReader;
+import org.apache.nifi.provenance.serialization.RecordReaders;
+import org.apache.nifi.provenance.serialization.RecordWriter;
+import org.apache.nifi.provenance.serialization.RecordWriters;
+import org.apache.nifi.provenance.toc.TocReader;
+import org.apache.nifi.provenance.toc.TocUtil;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.util.StopWatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.EOFException;
 import java.io.File;
@@ -54,29 +76,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Pattern;
 
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.provenance.expiration.ExpirationAction;
-import org.apache.nifi.provenance.expiration.FileRemovalAction;
-import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
-import org.apache.nifi.provenance.search.Query;
-import org.apache.nifi.provenance.search.QuerySubmission;
-import org.apache.nifi.provenance.search.SearchableField;
-import org.apache.nifi.provenance.serialization.RecordReader;
-import org.apache.nifi.provenance.serialization.RecordReaders;
-import org.apache.nifi.provenance.serialization.RecordWriter;
-import org.apache.nifi.provenance.serialization.RecordWriters;
-import org.apache.nifi.provenance.toc.TocReader;
-import org.apache.nifi.provenance.toc.TocUtil;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.StopWatch;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static org.apache.nifi.provenance.toc.TocUtil.getTocFile;
 
 
 // TODO: When API, FlowController, and supporting classes are refactored/reimplemented migrate this class and its accompanying imports to minifi package structure
@@ -320,7 +320,7 @@ public class MiNiFiPersistentProvenanceRepository implements ProvenanceRepositor
             final File journalDirectory = new File(storageDirectory, "journals");
             final File journalFile = new File(journalDirectory, String.valueOf(initialRecordId) + ".journal." + i);
 
-            writers[i] = RecordWriters.newRecordWriter(journalFile, false, false);
+            writers[i] = RecordWriters.newSchemaRecordWriter(journalFile, false, false);
             writers[i].writeHeader(initialRecordId);
         }
 
@@ -1367,7 +1367,7 @@ public class MiNiFiPersistentProvenanceRepository implements ProvenanceRepositor
 
             // loop over each entry in the map, persisting the records to the merged file in order, and populating the map
             // with the next entry from the journal file from which the previous record was written.
-            try (final RecordWriter writer = RecordWriters.newRecordWriter(writerFile, configuration.isCompressOnRollover(), true)) {
+            try (final RecordWriter writer = RecordWriters.newSchemaRecordWriter(writerFile, configuration.isCompressOnRollover(), true)) {
                 writer.writeHeader(minEventId);
 
                 while (!recordToReaderMap.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestMiNiFiPersistentProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestMiNiFiPersistentProvenanceRepository.java b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestMiNiFiPersistentProvenanceRepository.java
index fbeeeb6..0dd5f65 100644
--- a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestMiNiFiPersistentProvenanceRepository.java
+++ b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestMiNiFiPersistentProvenanceRepository.java
@@ -16,30 +16,6 @@
  */
 package org.apache.nifi.provenance;
 
-import static org.apache.nifi.provenance.TestUtil.createFlowFile;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.zip.GZIPOutputStream;
-
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.flowfile.FlowFile;
@@ -57,9 +33,28 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.zip.GZIPOutputStream;
+
+import static org.apache.nifi.provenance.TestUtil.createFlowFile;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public class TestMiNiFiPersistentProvenanceRepository {
 
@@ -558,113 +553,6 @@ public class TestMiNiFiPersistentProvenanceRepository {
         assertEquals(0, reportedEvents.size());
     }
 
-
-    @Test
-    public void testBehaviorOnOutOfMemory() throws IOException, InterruptedException {
-        final RepositoryConfiguration config = createConfiguration();
-        config.setMaxEventFileLife(3, TimeUnit.MINUTES);
-        config.setJournalCount(4);
-
-        // Create a repository that overrides the createWriters() method so that we can return writers that will throw
-        // OutOfMemoryError where we want to
-        final AtomicBoolean causeOOME = new AtomicBoolean(false);
-        repo = new MiNiFiPersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
-            @Override
-            protected RecordWriter[] createWriters(RepositoryConfiguration config, long initialRecordId) throws IOException {
-                final RecordWriter[] recordWriters = super.createWriters(config, initialRecordId);
-
-                // Spy on each of the writers so that a call to writeUUID throws an OutOfMemoryError if we set the
-                // causeOOME flag to true
-                final StandardRecordWriter[] spiedWriters = new StandardRecordWriter[recordWriters.length];
-                for (int i = 0; i < recordWriters.length; i++) {
-                    final StandardRecordWriter writer = (StandardRecordWriter) recordWriters[i];
-
-                    spiedWriters[i] = Mockito.spy(writer);
-                    Mockito.doAnswer(new Answer<Object>() {
-                        @Override
-                        public Object answer(final InvocationOnMock invocation) throws Throwable {
-                            if (causeOOME.get()) {
-                                throw new OutOfMemoryError();
-                            } else {
-                                writer.writeUUID(invocation.getArgumentAt(0, DataOutputStream.class), invocation.getArgumentAt(1, String.class));
-                            }
-                            return null;
-                        }
-                    }).when(spiedWriters[i]).writeUUID(Mockito.any(DataOutputStream.class), Mockito.any(String.class));
-                }
-
-                // return the writers that we are spying on
-                return spiedWriters;
-            }
-        };
-        repo.initialize(getEventReporter(), null, null);
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("75chars", "123456789012345678901234567890123456789012345678901234567890123456789012345");
-
-        final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
-        builder.setEventTime(System.currentTimeMillis());
-        builder.setEventType(ProvenanceEventType.RECEIVE);
-        builder.setTransitUri("nifi://unit-test");
-        attributes.put("uuid", "12345678-0000-0000-0000-012345678912");
-        builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
-        builder.setComponentId("1234");
-        builder.setComponentType("dummy processor");
-
-        // first make sure that we are able to write to the repo successfully.
-        for (int i = 0; i < 4; i++) {
-            final ProvenanceEventRecord record = builder.build();
-            repo.registerEvent(record);
-        }
-
-        // cause OOME to occur
-        causeOOME.set(true);
-
-        // write 4 times to make sure that we mark all partitions as dirty
-        for (int i = 0; i < 4; i++) {
-            final ProvenanceEventRecord record = builder.build();
-            try {
-                repo.registerEvent(record);
-                Assert.fail("Expected OutOfMmeoryError but was able to register event");
-            } catch (final OutOfMemoryError oome) {
-            }
-        }
-
-        // now that all partitions are dirty, ensure that as we keep trying to write, we get an IllegalStateException
-        // and that we don't corrupt the repository by writing partial records
-        for (int i = 0; i < 8; i++) {
-            final ProvenanceEventRecord record = builder.build();
-            try {
-                repo.registerEvent(record);
-                Assert.fail("Expected OutOfMmeoryError but was able to register event");
-            } catch (final IllegalStateException ise) {
-            }
-        }
-
-        // close repo so that we can create a new one to recover records
-        repo.close();
-
-        // make sure we can recover
-        final MiNiFiPersistentProvenanceRepository recoveryRepo = new MiNiFiPersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
-            @Override
-            protected Set<File> recoverJournalFiles() throws IOException {
-                try {
-                    return super.recoverJournalFiles();
-                } catch (final IOException ioe) {
-                    Assert.fail("Failed to recover properly");
-                    return null;
-                }
-            }
-        };
-
-        try {
-            recoveryRepo.initialize(getEventReporter(), null, null);
-        } finally {
-            recoveryRepo.close();
-        }
-    }
-
-
     private static class ReportedEvent {
         private final Severity severity;
         private final String category;

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/85d69ead/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5678666..27075db 100644
--- a/pom.xml
+++ b/pom.xml
@@ -95,7 +95,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>1.0.0</org.apache.nifi.version>
+        <org.apache.nifi.version>1.1.0</org.apache.nifi.version>
         <logback.version>1.1.7</logback.version>
         <jetty.version>9.3.9.v20160517</jetty.version>
         <jersey.version>1.19</jersey.version>


[3/3] nifi-minifi git commit: MINIFI-142 Updating License and Notice information for NiFi 1.1.0 upgrade

Posted by jp...@apache.org.
MINIFI-142 Updating License and Notice information for NiFi 1.1.0 upgrade

This closes #60

Signed-off-by: Aldrin Piri <al...@apache.org>


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

Branch: refs/heads/master
Commit: 967d402434d448f08411c1933eea21a9dd4c53cb
Parents: 85d69ea
Author: Joseph Percivall <JP...@apache.org>
Authored: Tue Nov 29 23:21:47 2016 -0500
Committer: Joseph Percivall <JP...@apache.org>
Committed: Wed Nov 30 17:13:15 2016 -0500

----------------------------------------------------------------------
 minifi-assembly/LICENSE                         |  43 +++-
 minifi-assembly/NOTICE                          |  62 ++++-
 .../src/main/resources/META-INF/LICENSE         | 202 ++++++++++++++++
 .../src/main/resources/META-INF/NOTICE          |  32 +++
 .../src/main/resources/META-INF/LICENSE         | 202 ++++++++++++++++
 .../src/main/resources/META-INF/NOTICE          |  28 +++
 .../src/main/resources/META-INF/LICENSE         | 202 ++++++++++++++++
 .../src/main/resources/META-INF/NOTICE          | 193 +---------------
 .../src/main/resources/META-INF/LICENSE         | 135 -----------
 .../src/main/resources/META-INF/NOTICE          | 174 +-------------
 .../src/main/resources/META-INF/LICENSE         |  27 +--
 .../src/main/resources/META-INF/NOTICE          |  50 ++--
 .../src/main/resources/META-INF/LICENSE         | 135 -----------
 .../src/main/resources/META-INF/NOTICE          | 172 --------------
 .../src/main/resources/META-INF/LICENSE         | 202 ++++++++++++++++
 .../src/main/resources/META-INF/NOTICE          |  13 ++
 minifi-toolkit/minifi-toolkit-assembly/LICENSE  | 106 ++++++++-
 minifi-toolkit/minifi-toolkit-assembly/NOTICE   | 229 ++++++++++++++++++-
 18 files changed, 1333 insertions(+), 874 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-assembly/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-assembly/LICENSE b/minifi-assembly/LICENSE
index dda1222..af59c3f 100644
--- a/minifi-assembly/LICENSE
+++ b/minifi-assembly/LICENSE
@@ -202,9 +202,9 @@
    limitations under the License.
 
 
-APACHE NIFI - MINIFI SUBCOMPONENTS:
+APACHE MINIFI SUBCOMPONENTS:
 
-The Apache NiFi - MiNiFi project contains subcomponents with separate copyright
+The Apache MiNiFi project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these
 subcomponents is subject to the terms and conditions of the following
 licenses.
@@ -483,4 +483,41 @@ For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE
     ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
     WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
     ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
-    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
\ No newline at end of file
+    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+
+
+The binary distribution of this product bundles 'JLine' under a BSD
+style license.
+
+    Copyright (c) 2002-2006, Marc Prud'hommeaux <mw...@cornell.edu>
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or
+    without modification, are permitted provided that the following
+    conditions are met:
+
+    Redistributions of source code must retain the above copyright
+    notice, this list of conditions and the following disclaimer.
+
+    Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions and the following disclaimer
+    in the documentation and/or other materials provided with
+    the distribution.
+
+    Neither the name of JLine nor the names of its contributors
+    may be used to endorse or promote products derived from this
+    software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+    "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING,
+    BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
+    AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
+    EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
+    FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
+    OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+    PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+    DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
+    AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+    LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+    IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+    OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-assembly/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-assembly/NOTICE b/minifi-assembly/NOTICE
index a193179..a374875 100644
--- a/minifi-assembly/NOTICE
+++ b/minifi-assembly/NOTICE
@@ -12,15 +12,15 @@ The following binary components are provided under the Apache Software License v
 
   (ASLv2) Apache NiFi
     The following NOTICE information applies:
-      Apache NiFi
-      Copyright 2014-2016 The Apache Software Foundation
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
 
-      This product includes software developed at
-      The Apache Software Foundation (http://www.apache.org/).
+        This product includes software developed at
+        The Apache Software Foundation (http://www.apache.org/).
 
-      This product includes the following work from the Apache Hadoop project:
+        This product includes the following work from the Apache Hadoop project:
 
-      BoundedByteArrayOutputStream.java adapted to SoftLimitBoundedByteArrayOutputStream.java
+        BoundedByteArrayOutputStream.java adapted to SoftLimitBoundedByteArrayOutputStream.java
 
   (ASLv2) Jetty
     The following NOTICE information applies:
@@ -294,6 +294,10 @@ The following binary components are provided under the Apache Software License v
       Apache Avro
       Copyright 2009-2013 The Apache Software Foundation
 
+  (ASLv2) Jasypt
+    The following NOTICE information applies:
+	  Copyright (c) 2007-2010, The JASYPT team (http://www.jasypt.org)
+
   (ASLv2) Snappy Java
     The following NOTICE information applies:
       This product includes software developed by Google
@@ -591,6 +595,43 @@ The following binary components are provided under the Apache Software License v
       is a distributed tracing system that is Apache 2.0 Licensed.
       Copyright 2012 Twitter, Inc.
 
+  (ASLv2) Jettison
+    The following NOTICE information applies:
+         Copyright 2006 Envoi Solutions LLC
+
+    (ASLv2) Apache Curator
+      The following NOTICE information applies:
+        Curator Framework
+        Copyright 2011-2014 The Apache Software Foundation
+
+        Curator Client
+        Copyright 2011-2014 The Apache Software Foundation
+
+        Curator Recipes
+        Copyright 2011-2014 The Apache Software Foundation
+
+
+    (ASLv2) Quartz
+      The following NOTICE information applies:
+        Copyright Declaration:
+        Copyright � 2003-2016 Software AG, Darmstadt, Germany and/or Software AG USA Inc., Reston, VA, USA, and/or its subsidiaries and/or its affiliates and/or their licensors.
+
+        Trademark and Patent declaration
+        The name Software AG and all Software AG product names are either trademarks or registered trademarks of Software AG and/or Software AG USA Inc. and/or its subsidiaries and/or its affiliates
+        and/or their licensors. Other company and product names mentioned herein may be trademarks of their respective owners.
+
+        Detailed information on trademarks and patents owned by Software AG and/or its subsidiaries is located at http://softwareag.com/licenses.
+
+        Third Party declaration
+        This software may include portions of third-party products. For third-party copyright notices, license terms, additional rights or restrictions, please refer to "License Texts, Copyright
+        Notices and Disclaimers of Third Party Products". For certain specific third-party license restrictions, please refer to section E of the Legal Notices available under "License Terms and
+        Conditions for Use of Software AG Products / Copyright and Trademark Notices of Software AG Products". These documents are part of the product documentation, located at
+        http://softwareag.com/licenses and/or in the root installation directory of the licensed product(s).
+
+        Confidentiality Disclaimer:
+        Use, reproduction, transfer, publication or disclosure is prohibited except as specifically provided for in your License Agreement with Software AG.
+        Contact GitHub API Training Shop Blog About
+
 ************************
 Common Development and Distribution License 1.1
 ************************
@@ -631,6 +672,15 @@ The following binary components are provided under the Common Development and Di
     (CDDL 1.1) (GPL2 w/ CPE) jersey-repackaged-guava (org.glassfish.jersey.bundles.repackaged:jersey-guava:bundle:2.19 - https://jersey.java.net/project/project/jersey-guava/)
 
 ************************
+Common Development and Distribution License 1.0
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details.
+
+    (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
+    (CDDL 1.0) (GPL3) Streaming API For XML (javax.xml.stream:stax-api:jar:1.0-2 - no url provided)
+
+************************
 Eclipse Public License 1.0
 ************************
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/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/967d4024/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..982f250
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,32 @@
+minifi-framework-nar
+Copyright 2014-2016 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache NiFi
+    The following NOTICE information applies:
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
+
+        This product includes software developed at
+        The Apache Software Foundation (http://www.apache.org/).
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2012 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/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/967d4024/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..e53d061
--- /dev/null
+++ b/minifi-nar-bundles/minifi-provenance-reporting-bundle/minifi-provenance-reporting-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,28 @@
+minifi-provenance-reporting-nar
+Copyright 2014-2016 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+===========================================
+Apache Software License v2
+===========================================
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache NiFi
+    The following NOTICE information applies:
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
+
+        This product includes software developed at
+        The Apache Software Foundation (http://www.apache.org/).
+
+************************
+Common Development and Distribution License 1.1
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
+
+    (CDDL 1.1) (GPL2 w/ CPE) JSON Processing API (javax.json:javax.json-api:jar:1.0 - http://json-processing-spec.java.net)
+    (CDDL 1.1) (GPL2 w/ CPE) JSON Processing Default Provider (org.glassfish:javax.json:jar:1.0.4 - https://jsonp.java.net)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/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/967d4024/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/NOTICE
index 9cfe874..0ace367 100644
--- a/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/NOTICE
+++ b/minifi-nar-bundles/minifi-provenance-repository-bundle/minifi-provenance-repository-nar/src/main/resources/META-INF/NOTICE
@@ -10,193 +10,10 @@ Apache Software License v2
 
 The following binary components are provided under the Apache Software License v2
 
-  (ASLv2) Apache Lucene
+  (ASLv2) Apache NiFi
     The following NOTICE information applies:
-      Apache Lucene
-      Copyright 2014 The Apache Software Foundation
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
 
-      Includes software from other Apache Software Foundation projects,
-      including, but not limited to:
-       - Apache Ant
-       - Apache Jakarta Regexp
-       - Apache Commons
-       - Apache Xerces
-
-      ICU4J, (under analysis/icu) is licensed under an MIT styles license
-      and Copyright (c) 1995-2008 International Business Machines Corporation and others
-
-      Some data files (under analysis/icu/src/data) are derived from Unicode data such
-      as the Unicode Character Database. See http://unicode.org/copyright.html for more
-      details.
-
-      Brics Automaton (under core/src/java/org/apache/lucene/util/automaton) is
-      BSD-licensed, created by Anders M�ller. See http://www.brics.dk/automaton/
-
-      The levenshtein automata tables (under core/src/java/org/apache/lucene/util/automaton) were
-      automatically generated with the moman/finenight FSA library, created by
-      Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
-      see http://sites.google.com/site/rrettesite/moman and
-      http://bitbucket.org/jpbarrette/moman/overview/
-
-      The class org.apache.lucene.util.WeakIdentityMap was derived from
-      the Apache CXF project and is Apache License 2.0.
-
-      The Google Code Prettify is Apache License 2.0.
-      See http://code.google.com/p/google-code-prettify/
-
-      JUnit (junit-4.10) is licensed under the Common Public License v. 1.0
-      See http://junit.sourceforge.net/cpl-v10.html
-
-      This product includes code (JaspellTernarySearchTrie) from Java Spelling Checkin
-      g Package (jaspell): http://jaspell.sourceforge.net/
-      License: The BSD License (http://www.opensource.org/licenses/bsd-license.php)
-
-      The snowball stemmers in
-        analysis/common/src/java/net/sf/snowball
-      were developed by Martin Porter and Richard Boulton.
-      The snowball stopword lists in
-        analysis/common/src/resources/org/apache/lucene/analysis/snowball
-      were developed by Martin Porter and Richard Boulton.
-      The full snowball package is available from
-        http://snowball.tartarus.org/
-
-      The KStem stemmer in
-        analysis/common/src/org/apache/lucene/analysis/en
-      was developed by Bob Krovetz and Sergio Guzman-Lara (CIIR-UMass Amherst)
-      under the BSD-license.
-
-      The Arabic,Persian,Romanian,Bulgarian, and Hindi analyzers (common) come with a default
-      stopword list that is BSD-licensed created by Jacques Savoy.  These files reside in:
-      analysis/common/src/resources/org/apache/lucene/analysis/ar/stopwords.txt,
-      analysis/common/src/resources/org/apache/lucene/analysis/fa/stopwords.txt,
-      analysis/common/src/resources/org/apache/lucene/analysis/ro/stopwords.txt,
-      analysis/common/src/resources/org/apache/lucene/analysis/bg/stopwords.txt,
-      analysis/common/src/resources/org/apache/lucene/analysis/hi/stopwords.txt
-      See http://members.unine.ch/jacques.savoy/clef/index.html.
-
-      The German,Spanish,Finnish,French,Hungarian,Italian,Portuguese,Russian and Swedish light stemmers
-      (common) are based on BSD-licensed reference implementations created by Jacques Savoy and
-      Ljiljana Dolamic. These files reside in:
-      analysis/common/src/java/org/apache/lucene/analysis/de/GermanLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/de/GermanMinimalStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/es/SpanishLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchMinimalStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/it/ItalianLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/ru/RussianLightStemmer.java
-      analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishLightStemmer.java
-
-      The Stempel analyzer (stempel) includes BSD-licensed software developed
-      by the Egothor project http://egothor.sf.net/, created by Leo Galambos, Martin Kvapil,
-      and Edmond Nolan.
-
-      The Polish analyzer (stempel) comes with a default
-      stopword list that is BSD-licensed created by the Carrot2 project. The file resides
-      in stempel/src/resources/org/apache/lucene/analysis/pl/stopwords.txt.
-      See http://project.carrot2.org/license.html.
-
-      The SmartChineseAnalyzer source code (smartcn) was
-      provided by Xiaoping Gao and copyright 2009 by www.imdict.net.
-
-      WordBreakTestUnicode_*.java (under modules/analysis/common/src/test/)
-      is derived from Unicode data such as the Unicode Character Database.
-      See http://unicode.org/copyright.html for more details.
-
-      The Morfologik analyzer (morfologik) includes BSD-licensed software
-      developed by Dawid Weiss and Marcin Mi\u0142kowski (http://morfologik.blogspot.com/).
-
-      Morfologik uses data from Polish ispell/myspell dictionary
-      (http://www.sjp.pl/slownik/en/) licenced on the terms of (inter alia)
-      LGPL and Creative Commons ShareAlike.
-
-      Morfologic includes data from BSD-licensed dictionary of Polish (SGJP)
-      (http://sgjp.pl/morfeusz/)
-
-      Servlet-api.jar and javax.servlet-*.jar are under the CDDL license, the original
-      source code for this can be found at http://www.eclipse.org/jetty/downloads.php
-
-      ===========================================================================
-      Kuromoji Japanese Morphological Analyzer - Apache Lucene Integration
-      ===========================================================================
-
-      This software includes a binary and/or source version of data from
-
-        mecab-ipadic-2.7.0-20070801
-
-      which can be obtained from
-
-        http://atilika.com/releases/mecab-ipadic/mecab-ipadic-2.7.0-20070801.tar.gz
-
-      or
-
-        http://jaist.dl.sourceforge.net/project/mecab/mecab-ipadic/2.7.0-20070801/mecab-ipadic-2.7.0-20070801.tar.gz
-
-      ===========================================================================
-      mecab-ipadic-2.7.0-20070801 Notice
-      ===========================================================================
-
-      Nara Institute of Science and Technology (NAIST),
-      the copyright holders, disclaims all warranties with regard to this
-      software, including all implied warranties of merchantability and
-      fitness, in no event shall NAIST be liable for
-      any special, indirect or consequential damages or any damages
-      whatsoever resulting from loss of use, data or profits, whether in an
-      action of contract, negligence or other tortuous action, arising out
-      of or in connection with the use or performance of this software.
-
-      A large portion of the dictionary entries
-      originate from ICOT Free Software.  The following conditions for ICOT
-      Free Software applies to the current dictionary as well.
-
-      Each User may also freely distribute the Program, whether in its
-      original form or modified, to any third party or parties, PROVIDED
-      that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear
-      on, or be attached to, the Program, which is distributed substantially
-      in the same form as set out herein and that such intended
-      distribution, if actually made, will neither violate or otherwise
-      contravene any of the laws and regulations of the countries having
-      jurisdiction over the User or the intended distribution itself.
-
-      NO WARRANTY
-
-      The program was produced on an experimental basis in the course of the
-      research and development conducted during the project and is provided
-      to users as so produced on an experimental basis.  Accordingly, the
-      program is provided without any warranty whatsoever, whether express,
-      implied, statutory or otherwise.  The term "warranty" used herein
-      includes, but is not limited to, any warranty of the quality,
-      performance, merchantability and fitness for a particular purpose of
-      the program and the nonexistence of any infringement or violation of
-      any right of any third party.
-
-      Each user of the program will agree and understand, and be deemed to
-      have agreed and understood, that there is no warranty whatsoever for
-      the program and, accordingly, the entire risk arising from or
-      otherwise connected with the program is assumed by the user.
-
-      Therefore, neither ICOT, the copyright holder, or any other
-      organization that participated in or was otherwise related to the
-      development of the program and their respective officials, directors,
-      officers and other employees shall be held liable for any and all
-      damages, including, without limitation, general, special, incidental
-      and consequential damages, arising out of or otherwise in connection
-      with the use or inability to use the program or any product, material
-      or result produced or otherwise obtained by using the program,
-      regardless of whether they have been advised of, or otherwise had
-      knowledge of, the possibility of such damages at any time during the
-      project or thereafter.  Each user will be deemed to have agreed to the
-      foregoing by his or her commencement of use of the program.  The term
-      "use" as used herein includes, but is not limited to, the use,
-      modification, copying and distribution of the program and the
-      production of secondary products from the program.
-
-      In the case where the program, whether in its original form or
-      modified, was distributed or delivered to or received by a user from
-      any person, organization or entity other than ICOT, unless it makes or
-      grants independently of ICOT any specific warranty to the user in
-      writing, such person, organization or entity, will also be exempted
-      from and not be held liable to the user for any such damages as noted
-      above as far as the program is concerned.
\ No newline at end of file
+        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/967d4024/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/LICENSE
index a38d06b..d645695 100644
--- a/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/LICENSE
+++ b/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/LICENSE
@@ -200,138 +200,3 @@
    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.
-
-APACHE NIFI SUBCOMPONENTS:
-
-The Apache NiFi project contains subcomponents with separate copyright
-notices and license terms. Your use of the source code for the these
-subcomponents is subject to the terms and conditions of the following
-licenses. 
-
-The binary distribution of this product bundles 'Bouncy Castle JDK 1.5 Provider'
-under an MIT style license.
-
-    Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
-
-    Permission is hereby granted, free of charge, to any person obtaining a copy
-    of this software and associated documentation files (the "Software"), to deal
-    in the Software without restriction, including without limitation the rights
-    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-    copies of the Software, and to permit persons to whom the Software is
-    furnished to do so, subject to the following conditions:
-
-    The above copyright notice and this permission notice shall be included in
-    all copies or substantial portions of the Software.
-
-    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-    THE SOFTWARE.
-
-This product bundes 'JCraft Jsch' which is available under a 3-Clause BSD 
-License.
-
-    Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. 
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions are met:
-
-      1. Redistributions of source code must retain the above copyright notice,
-         this list of conditions and the following disclaimer.
-
-      2. Redistributions in binary form must reproduce the above copyright 
-         notice, this list of conditions and the following disclaimer in 
-         the documentation and/or other materials provided with the distribution.
-
-      3. The names of the authors may not be used to endorse or promote products
-         derived from this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
-    INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-    FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT,
-    INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT,
-    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-    LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA,
-    OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
-    EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-This product bundles 'JCraft Jzlib' which is available under a 3-Clause BSD License.
-
-    Copyright (c) 2002-2014 Atsuhiko Yamanaka, JCraft,Inc. 
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions are met:
-
-      1. Redistributions of source code must retain the above copyright notice,
-         this list of conditions and the following disclaimer.
-
-      2. Redistributions in binary form must reproduce the above copyright 
-         notice, this list of conditions and the following disclaimer in 
-         the documentation and/or other materials provided with the distribution.
-
-      3. The names of the authors may not be used to endorse or promote products
-         derived from this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
-    INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-    FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT,
-    INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT,
-    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-    LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA,
-    OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
-    EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-This product bundles 'asm' which is available under a 3-Clause BSD style license.
-For details see http://asm.ow2.org/asmdex-license.html
-
-    Copyright (c) 2012 France T�l�com
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions
-    are met:
-    1. Redistributions of source code must retain the above copyright
-       notice, this list of conditions and the following disclaimer.
-    2. Redistributions in binary form must reproduce the above copyright
-       notice, this list of conditions and the following disclaimer in the
-       documentation and/or other materials provided with the distribution.
-    3. Neither the name of the copyright holders nor the names of its
-       contributors may be used to endorse or promote products derived from
-       this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
-    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
-    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
-    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
-    THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles 'jBCrypt' which is available under a BSD license. For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE
-
-    Copyright (c) 2006 Damien Miller <dj...@mindrot.org>
-
-    Permission to use, copy, modify, and distribute this software for any
-    purpose with or without fee is hereby granted, provided that the above
-    copyright notice and this permission notice appear in all copies.
-
-    THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
-    WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
-    MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
-    ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
-    WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
-    ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
-    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/NOTICE
index 5718321..b7d1e8b 100644
--- a/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/NOTICE
+++ b/minifi-nar-bundles/minifi-ssl-context-service-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-minifi-standard-nar
+minifi-ssl-context-service-nar
 Copyright 2014-2016 The Apache Software Foundation
 
 This product includes software developed at
@@ -22,175 +22,3 @@ The following binary components are provided under the Apache Software License v
 
       This product includes software from the Spring Framework,
       under the Apache License 2.0 (see: StringUtils.containsWhitespace())
-
-  (ASLv2) Apache Commons Net
-    The following NOTICE information applies:
-      Apache Commons Net
-      Copyright 2001-2013 The Apache Software Foundation
-
-  (ASLv2) Apache Commons Compress
-    The following NOTICE information applies:
-      Apache Commons Compress
-      Copyright 2002-2014 The Apache Software Foundation
-
-      The files in the package org.apache.commons.compress.archivers.sevenz
-      were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/),
-      which has been placed in the public domain:
-
-      "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
-
-  (ASLv2) Apache HttpComponents
-    The following NOTICE information applies:
-      Apache HttpClient
-      Copyright 1999-2015 The Apache Software Foundation
-      
-      Apache HttpCore
-      Copyright 2005-2015 The Apache Software Foundation
-
-      This project contains annotations derived from JCIP-ANNOTATIONS
-      Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net
-
-  (ASLv2) Apache Commons Logging
-    The following NOTICE information applies:
-      Apache Commons Logging
-      Copyright 2003-2014 The Apache Software Foundation
-
-  (ASLv2) Joda Time
-    The following NOTICE information applies:
-      This product includes software developed by
-      Joda.org (http://www.joda.org/).
-
-  (ASLv2) Apache ActiveMQ
-    The following NOTICE information applies:
-      ActiveMQ :: Client
-      Copyright 2005-2015 The Apache Software Foundation
-
-  (ASLv2) Apache Geronimo
-    The following NOTICE information applies:
-      Apache Geronimo 
-      Copyright 2003-2008 The Apache Software Foundation
-
-  (ASLv2) Yammer Metrics
-    The following NOTICE information applies:
-      Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-      This product includes software developed by Coda Hale and Yammer, Inc.
-
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
-
-          Written by Doug Lea with assistance from members of JCP JSR-166
-          Expert Group and released to the public domain, as explained at
-          http://creativecommons.org/publicdomain/zero/1.0/
-
-  (ASLv2) JSON-SMART
-    The following NOTICE information applies:
-      Copyright 2011 JSON-SMART authors
-
-  (ASLv2) JsonPath
-    The following NOTICE information applies:
-      Copyright 2011 JsonPath authors
-
-  (ASLv2) Apache Commons Codec
-    The following NOTICE information applies:
-      Apache Commons Codec
-      Copyright 2002-2014 The Apache Software Foundation
-
-      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
-      contains test data from http://aspell.net/test/orig/batch0.tab.
-      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
-
-      ===============================================================================
-
-      The content of package org.apache.commons.codec.language.bm has been translated
-      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
-      with permission from the original authors.
-      Original source copyright:
-      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
-
-  (ASLv2) Apache Tika
-    The following NOTICE information applies:
-      Apache Tika Core
-      Copyright 2007-2015 The Apache Software Foundation
-
-  (ASLv2) Jackson JSON processor
-    The following NOTICE information applies:
-      # Jackson JSON processor
-
-      Jackson is a high-performance, Free/Open Source JSON processing library.
-      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
-      been in development since 2007.
-      It is currently developed by a community of developers, as well as supported
-      commercially by FasterXML.com.
-
-      ## Licensing
-
-      Jackson core and extension components may licensed under different licenses.
-      To find the details that apply to this artifact see the accompanying LICENSE file.
-      For more information, including possible other licensing options, contact
-      FasterXML.com (http://fasterxml.com).
-
-      ## Credits
-
-      A list of contributors may be found from CREDITS file, which is included
-      in some artifacts (usually source distributions); but is always available
-      from the source code management (SCM) system project uses.
-
-  (ASLv2) Apache Avro
-    The following NOTICE information applies:
-      Apache Avro
-      Copyright 2009-2013 The Apache Software Foundation
-
-  (ASLv2) Snappy Java
-    The following NOTICE information applies:
-      This product includes software developed by Google
-       Snappy: http://code.google.com/p/snappy/ (New BSD License)
-      
-      This product includes software developed by Apache
-       PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
-       (Apache 2.0 license)
-
-      This library containd statically linked libstdc++. This inclusion is allowed by 
-      "GCC RUntime Library Exception" 
-      http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
-
-************************
-Common Development and Distribution License 1.1
-************************
-
-The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
-
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.19 - https://jersey.java.net/jersey-core/)
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.19 - https://jersey.java.net/jersey-client/)
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.19 - https://jersey.java.net/jersey-server/)
-    (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
-    (CDDL 1.1) (GPL2 w/ CPE) Javax JMS Api (javax.jms:javax.jms-api:jar:2.0.1 - http://java.net/projects/jms-spec/pages/Home)
-
-*****************
-Common Development and Distribution License v1.0:
-*****************
-
-The following binary components are provided under the Common Development and Distribution License v1.0.  See project link for details.
-
-    (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
-    (CDDL 1.0) JSR311 API (javax.ws.rs:jsr311-api:jar:1.1.1 - https://jsr311.dev.java.net)
-
-*****************
-Mozilla Public License v2.0
-*****************
-
-The following binary components are provided under the Mozilla Public License v2.0.  See project link for details.
-
-    (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:9.6.0-5 - http://www.saxonica.com/)
-
-*****************
-Public Domain
-*****************
-
-The following binary components are provided to the 'Public Domain'.  See project link for details.
-
-    (Public Domain) XZ for Java (org.tukaani:xz:jar:1.5 - http://tukaani.org/xz/java.html
-
-
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/LICENSE
index a38d06b..43fcd47 100644
--- a/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/LICENSE
+++ b/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/LICENSE
@@ -201,36 +201,13 @@
    See the License for the specific language governing permissions and
    limitations under the License.
 
-APACHE NIFI SUBCOMPONENTS:
+APACHE MINIFI SUBCOMPONENTS:
 
-The Apache NiFi project contains subcomponents with separate copyright
+The Apache MiNiFi project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these
 subcomponents is subject to the terms and conditions of the following
 licenses. 
 
-The binary distribution of this product bundles 'Bouncy Castle JDK 1.5 Provider'
-under an MIT style license.
-
-    Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
-
-    Permission is hereby granted, free of charge, to any person obtaining a copy
-    of this software and associated documentation files (the "Software"), to deal
-    in the Software without restriction, including without limitation the rights
-    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-    copies of the Software, and to permit persons to whom the Software is
-    furnished to do so, subject to the following conditions:
-
-    The above copyright notice and this permission notice shall be included in
-    all copies or substantial portions of the Software.
-
-    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-    THE SOFTWARE.
-
 This product bundes 'JCraft Jsch' which is available under a 3-Clause BSD 
 License.
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/NOTICE
index 5718321..554311d 100644
--- a/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/NOTICE
+++ b/minifi-nar-bundles/minifi-standard-nar/src/main/resources/META-INF/NOTICE
@@ -1,6 +1,11 @@
 minifi-standard-nar
 Copyright 2014-2016 The Apache Software Foundation
 
+This includes derived works from the Apache Software License V2 library Jolt (https://github.com/bazaarvoice/jolt)
+Copyright 2013-2014 Bazaarvoice, Inc
+The derived work is adapted from com.bazaarvoice.jolt.chainr.ChainrBuilder.java, com.bazaarvoice.jolt.chainr.spec.ChainrSpec.java, com.bazaarvoice.jolt.chainr.spec.ChainrEntry.java and can be found in the org.apache.nifi.processors.standard.util.TransformFactory.java class.
+
+
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
 
@@ -10,19 +15,23 @@ Apache Software License v2
 
 The following binary components are provided under the Apache Software License v2
 
+  (ASLv2) Apache NiFi
+    The following NOTICE information applies:
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
+
+        This product includes software developed at
+        The Apache Software Foundation (http://www.apache.org/).
+
+        This product includes the following work from the Apache Hadoop project:
+
+        BoundedByteArrayOutputStream.java adapted to SoftLimitBoundedByteArrayOutputStream.java
+
   (ASLv2) Apache Commons IO
     The following NOTICE information applies:
       Apache Commons IO
       Copyright 2002-2012 The Apache Software Foundation
 
-  (ASLv2) Apache Commons Lang
-    The following NOTICE information applies:
-      Apache Commons Lang
-      Copyright 2001-2015 The Apache Software Foundation
-
-      This product includes software from the Spring Framework,
-      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
-
   (ASLv2) Apache Commons Net
     The following NOTICE information applies:
       Apache Commons Net
@@ -39,22 +48,6 @@ The following binary components are provided under the Apache Software License v
 
       "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
 
-  (ASLv2) Apache HttpComponents
-    The following NOTICE information applies:
-      Apache HttpClient
-      Copyright 1999-2015 The Apache Software Foundation
-      
-      Apache HttpCore
-      Copyright 2005-2015 The Apache Software Foundation
-
-      This project contains annotations derived from JCIP-ANNOTATIONS
-      Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net
-
-  (ASLv2) Apache Commons Logging
-    The following NOTICE information applies:
-      Apache Commons Logging
-      Copyright 2003-2014 The Apache Software Foundation
-
   (ASLv2) Joda Time
     The following NOTICE information applies:
       This product includes software developed by
@@ -155,6 +148,14 @@ The following binary components are provided under the Apache Software License v
       "GCC RUntime Library Exception" 
       http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
 
+  (ASLv2) Jolt
+    The following NOTICE information applies:
+      Copyright 2013-2014 Bazaarvoice, Inc
+
+  (ASLv2) ParCEFone
+    The following NOTICE information applies:
+      ParCEFone
+      Copyright 2016 Fluenda
 ************************
 Common Development and Distribution License 1.1
 ************************
@@ -163,7 +164,6 @@ The following binary components are provided under the Common Development and Di
 
     (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.19 - https://jersey.java.net/jersey-core/)
     (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.19 - https://jersey.java.net/jersey-client/)
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.19 - https://jersey.java.net/jersey-server/)
     (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
     (CDDL 1.1) (GPL2 w/ CPE) Javax JMS Api (javax.jms:javax.jms-api:jar:2.0.1 - http://java.net/projects/jms-spec/pages/Home)
 


[2/3] nifi-minifi git commit: MINIFI-142 Updating License and Notice information for NiFi 1.1.0 upgrade

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/LICENSE
index a38d06b..d645695 100644
--- a/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/LICENSE
+++ b/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/LICENSE
@@ -200,138 +200,3 @@
    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.
-
-APACHE NIFI SUBCOMPONENTS:
-
-The Apache NiFi project contains subcomponents with separate copyright
-notices and license terms. Your use of the source code for the these
-subcomponents is subject to the terms and conditions of the following
-licenses. 
-
-The binary distribution of this product bundles 'Bouncy Castle JDK 1.5 Provider'
-under an MIT style license.
-
-    Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
-
-    Permission is hereby granted, free of charge, to any person obtaining a copy
-    of this software and associated documentation files (the "Software"), to deal
-    in the Software without restriction, including without limitation the rights
-    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-    copies of the Software, and to permit persons to whom the Software is
-    furnished to do so, subject to the following conditions:
-
-    The above copyright notice and this permission notice shall be included in
-    all copies or substantial portions of the Software.
-
-    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-    THE SOFTWARE.
-
-This product bundes 'JCraft Jsch' which is available under a 3-Clause BSD 
-License.
-
-    Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. 
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions are met:
-
-      1. Redistributions of source code must retain the above copyright notice,
-         this list of conditions and the following disclaimer.
-
-      2. Redistributions in binary form must reproduce the above copyright 
-         notice, this list of conditions and the following disclaimer in 
-         the documentation and/or other materials provided with the distribution.
-
-      3. The names of the authors may not be used to endorse or promote products
-         derived from this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
-    INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-    FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT,
-    INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT,
-    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-    LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA,
-    OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
-    EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-This product bundles 'JCraft Jzlib' which is available under a 3-Clause BSD License.
-
-    Copyright (c) 2002-2014 Atsuhiko Yamanaka, JCraft,Inc. 
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions are met:
-
-      1. Redistributions of source code must retain the above copyright notice,
-         this list of conditions and the following disclaimer.
-
-      2. Redistributions in binary form must reproduce the above copyright 
-         notice, this list of conditions and the following disclaimer in 
-         the documentation and/or other materials provided with the distribution.
-
-      3. The names of the authors may not be used to endorse or promote products
-         derived from this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
-    INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-    FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT,
-    INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT,
-    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-    LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA,
-    OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
-    EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-This product bundles 'asm' which is available under a 3-Clause BSD style license.
-For details see http://asm.ow2.org/asmdex-license.html
-
-    Copyright (c) 2012 France T�l�com
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions
-    are met:
-    1. Redistributions of source code must retain the above copyright
-       notice, this list of conditions and the following disclaimer.
-    2. Redistributions in binary form must reproduce the above copyright
-       notice, this list of conditions and the following disclaimer in the
-       documentation and/or other materials provided with the distribution.
-    3. Neither the name of the copyright holders nor the names of its
-       contributors may be used to endorse or promote products derived from
-       this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
-    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
-    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
-    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
-    THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles 'jBCrypt' which is available under a BSD license. For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE
-
-    Copyright (c) 2006 Damien Miller <dj...@mindrot.org>
-
-    Permission to use, copy, modify, and distribute this software for any
-    purpose with or without fee is hereby granted, provided that the above
-    copyright notice and this permission notice appear in all copies.
-
-    THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
-    WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
-    MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
-    ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
-    WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
-    ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
-    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/NOTICE
index 5718321..7eb13a7 100644
--- a/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/NOTICE
+++ b/minifi-nar-bundles/minifi-standard-services-api-nar/src/main/resources/META-INF/NOTICE
@@ -22,175 +22,3 @@ The following binary components are provided under the Apache Software License v
 
       This product includes software from the Spring Framework,
       under the Apache License 2.0 (see: StringUtils.containsWhitespace())
-
-  (ASLv2) Apache Commons Net
-    The following NOTICE information applies:
-      Apache Commons Net
-      Copyright 2001-2013 The Apache Software Foundation
-
-  (ASLv2) Apache Commons Compress
-    The following NOTICE information applies:
-      Apache Commons Compress
-      Copyright 2002-2014 The Apache Software Foundation
-
-      The files in the package org.apache.commons.compress.archivers.sevenz
-      were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/),
-      which has been placed in the public domain:
-
-      "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
-
-  (ASLv2) Apache HttpComponents
-    The following NOTICE information applies:
-      Apache HttpClient
-      Copyright 1999-2015 The Apache Software Foundation
-      
-      Apache HttpCore
-      Copyright 2005-2015 The Apache Software Foundation
-
-      This project contains annotations derived from JCIP-ANNOTATIONS
-      Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net
-
-  (ASLv2) Apache Commons Logging
-    The following NOTICE information applies:
-      Apache Commons Logging
-      Copyright 2003-2014 The Apache Software Foundation
-
-  (ASLv2) Joda Time
-    The following NOTICE information applies:
-      This product includes software developed by
-      Joda.org (http://www.joda.org/).
-
-  (ASLv2) Apache ActiveMQ
-    The following NOTICE information applies:
-      ActiveMQ :: Client
-      Copyright 2005-2015 The Apache Software Foundation
-
-  (ASLv2) Apache Geronimo
-    The following NOTICE information applies:
-      Apache Geronimo 
-      Copyright 2003-2008 The Apache Software Foundation
-
-  (ASLv2) Yammer Metrics
-    The following NOTICE information applies:
-      Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-      This product includes software developed by Coda Hale and Yammer, Inc.
-
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
-
-          Written by Doug Lea with assistance from members of JCP JSR-166
-          Expert Group and released to the public domain, as explained at
-          http://creativecommons.org/publicdomain/zero/1.0/
-
-  (ASLv2) JSON-SMART
-    The following NOTICE information applies:
-      Copyright 2011 JSON-SMART authors
-
-  (ASLv2) JsonPath
-    The following NOTICE information applies:
-      Copyright 2011 JsonPath authors
-
-  (ASLv2) Apache Commons Codec
-    The following NOTICE information applies:
-      Apache Commons Codec
-      Copyright 2002-2014 The Apache Software Foundation
-
-      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
-      contains test data from http://aspell.net/test/orig/batch0.tab.
-      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
-
-      ===============================================================================
-
-      The content of package org.apache.commons.codec.language.bm has been translated
-      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
-      with permission from the original authors.
-      Original source copyright:
-      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
-
-  (ASLv2) Apache Tika
-    The following NOTICE information applies:
-      Apache Tika Core
-      Copyright 2007-2015 The Apache Software Foundation
-
-  (ASLv2) Jackson JSON processor
-    The following NOTICE information applies:
-      # Jackson JSON processor
-
-      Jackson is a high-performance, Free/Open Source JSON processing library.
-      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
-      been in development since 2007.
-      It is currently developed by a community of developers, as well as supported
-      commercially by FasterXML.com.
-
-      ## Licensing
-
-      Jackson core and extension components may licensed under different licenses.
-      To find the details that apply to this artifact see the accompanying LICENSE file.
-      For more information, including possible other licensing options, contact
-      FasterXML.com (http://fasterxml.com).
-
-      ## Credits
-
-      A list of contributors may be found from CREDITS file, which is included
-      in some artifacts (usually source distributions); but is always available
-      from the source code management (SCM) system project uses.
-
-  (ASLv2) Apache Avro
-    The following NOTICE information applies:
-      Apache Avro
-      Copyright 2009-2013 The Apache Software Foundation
-
-  (ASLv2) Snappy Java
-    The following NOTICE information applies:
-      This product includes software developed by Google
-       Snappy: http://code.google.com/p/snappy/ (New BSD License)
-      
-      This product includes software developed by Apache
-       PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
-       (Apache 2.0 license)
-
-      This library containd statically linked libstdc++. This inclusion is allowed by 
-      "GCC RUntime Library Exception" 
-      http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
-
-************************
-Common Development and Distribution License 1.1
-************************
-
-The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
-
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.19 - https://jersey.java.net/jersey-core/)
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.19 - https://jersey.java.net/jersey-client/)
-    (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.19 - https://jersey.java.net/jersey-server/)
-    (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
-    (CDDL 1.1) (GPL2 w/ CPE) Javax JMS Api (javax.jms:javax.jms-api:jar:2.0.1 - http://java.net/projects/jms-spec/pages/Home)
-
-*****************
-Common Development and Distribution License v1.0:
-*****************
-
-The following binary components are provided under the Common Development and Distribution License v1.0.  See project link for details.
-
-    (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
-    (CDDL 1.0) JSR311 API (javax.ws.rs:jsr311-api:jar:1.1.1 - https://jsr311.dev.java.net)
-
-*****************
-Mozilla Public License v2.0
-*****************
-
-The following binary components are provided under the Mozilla Public License v2.0.  See project link for details.
-
-    (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:9.6.0-5 - http://www.saxonica.com/)
-
-*****************
-Public Domain
-*****************
-
-The following binary components are provided to the 'Public Domain'.  See project link for details.
-
-    (Public Domain) XZ for Java (org.tukaani:xz:jar:1.5 - http://tukaani.org/xz/java.html
-
-
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/LICENSE b/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/NOTICE b/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/NOTICE
index 9d96bed..56d1f0d 100644
--- a/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/NOTICE
+++ b/minifi-nar-bundles/minifi-update-attribute-nar/src/main/resources/META-INF/NOTICE
@@ -10,6 +10,14 @@ Apache Software License v2
 
 The following binary components are provided under the Apache Software License v2
 
+  (ASLv2) Apache NiFi
+    The following NOTICE information applies:
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
+
+        This product includes software developed at
+        The Apache Software Foundation (http://www.apache.org/).
+
   (ASLv2) Apache Commons Lang
     The following NOTICE information applies:
       Apache Commons Lang
@@ -17,3 +25,8 @@ The following binary components are provided under the Apache Software License v
 
       This product includes software from the Spring Framework,
       under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-toolkit/minifi-toolkit-assembly/LICENSE
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-assembly/LICENSE b/minifi-toolkit/minifi-toolkit-assembly/LICENSE
index 9864230..a4b4e87 100644
--- a/minifi-toolkit/minifi-toolkit-assembly/LICENSE
+++ b/minifi-toolkit/minifi-toolkit-assembly/LICENSE
@@ -202,16 +202,15 @@
    limitations under the License.
 
 
-APACHE NIFI - MINIFI SUBCOMPONENTS:
+APACHE MINIFI SUBCOMPONENTS:
 
-The Apache NiFi - MiNiFi project contains subcomponents with separate copyright
+The Apache MiNiFi project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these
 subcomponents is subject to the terms and conditions of the following
 licenses.
 
-
-  The binary distribution of this product bundles 'Slf4j' which is available
-  under a "3-clause BSD" license.  For details see http://www.slf4j.org/
+The binary distribution of this product bundles 'Slf4j' which is available
+under a "3-clause BSD" license.  For details see http://www.slf4j.org/
 
     Copyright (c) 2004-2013 QOS.ch
      All rights reserved.
@@ -233,4 +232,99 @@ licenses.
      NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
      LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
      OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
-     WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
\ No newline at end of file
+     WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+The binary distribution of this product bundles 'Antlr 3' which is available
+under a "3-clause BSD" license.  For details see http://www.antlr3.org/license.html
+
+    Copyright (c) 2010 Terence Parr
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions are met:
+
+    Redistributions of source code must retain the above copyright notice, this
+    list of conditions and the following disclaimer.
+    Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+    Neither the name of the author nor the names of its contributors may be used
+    to endorse or promote products derived from this software without specific
+    prior written permission.
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
+
+This product bundles 'asm' which is available under a 3-Clause BSD style license.
+For details see http://asm.ow2.org/asmdex-license.html
+
+    Copyright (c) 2012 France T�l�com
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles 'JLine' under a BSD
+style license.
+
+    Copyright (c) 2002-2006, Marc Prud'hommeaux <mw...@cornell.edu>
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or
+    without modification, are permitted provided that the following
+    conditions are met:
+
+    Redistributions of source code must retain the above copyright
+    notice, this list of conditions and the following disclaimer.
+
+    Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions and the following disclaimer
+    in the documentation and/or other materials provided with
+    the distribution.
+
+    Neither the name of JLine nor the names of its contributors
+    may be used to endorse or promote products derived from this
+    software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+    "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING,
+    BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
+    AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
+    EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
+    FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
+    OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+    PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+    DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
+    AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+    LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+    IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+    OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/967d4024/minifi-toolkit/minifi-toolkit-assembly/NOTICE
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-assembly/NOTICE b/minifi-toolkit/minifi-toolkit-assembly/NOTICE
index 4ec8741..7d89810 100644
--- a/minifi-toolkit/minifi-toolkit-assembly/NOTICE
+++ b/minifi-toolkit/minifi-toolkit-assembly/NOTICE
@@ -12,12 +12,229 @@ The following binary components are provided under the Apache Software License v
 
   (ASLv2) Apache NiFi
     The following NOTICE information applies:
-      Apache NiFi
-      Copyright 2014-2016 The Apache Software Foundation
+        Apache NiFi
+        Copyright 2014-2016 The Apache Software Foundation
 
-      This product includes software developed at
-      The Apache Software Foundation (http://www.apache.org/).
+        This product includes software developed at
+        The Apache Software Foundation (http://www.apache.org/).
 
-      This product includes the following work from the Apache Hadoop project:
+        This product includes the following work from the Apache Hadoop project:
 
-      BoundedByteArrayOutputStream.java adapted to SoftLimitBoundedByteArrayOutputStream.java
\ No newline at end of file
+        BoundedByteArrayOutputStream.java adapted to SoftLimitBoundedByteArrayOutputStream.java
+
+        This includes derived works from the Apache Software License V2 library python-evtx (https://github.com/williballenthin/python-evtx)
+        Copyright 2012, 2013 Willi Ballenthin william.ballenthin@mandiant.com
+        while at Mandiant http://www.mandiant.com
+        The derived work is adapted from Evtx/Evtx.py, Evtx/BinaryParser.py, Evtx/Nodes.py, Evtx/Views.py and can be found in the org.apache.nifi.processors.evtx.parser package.
+
+
+
+        This includes derived works from the Apache Storm (ASLv2 licensed) project (https://github.com/apache/storm):
+        Copyright 2015 The Apache Software Foundation
+        The derived work is adapted from
+          org/apache/storm/hive/common/HiveWriter.java
+          org/apache/storm/hive/common/HiveOptions.java
+        and can be found in the org.apache.nifi.util.hive package
+
+        This includes derived works from the Apache Hive (ASLv2 licensed) project (https://github.com/apache/hive):
+        Copyright 2008-2016 The Apache Software Foundation
+        The derived work is adapted from
+          release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
+        and can be found in the org.apache.hadoop.hive.ql.io.orc package
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Apache Commons Codec
+    The following NOTICE information applies:
+      Apache Commons Codec
+      Copyright 2002-2014 The Apache Software Foundation
+
+      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
+      contains test data from http://aspell.net/test/orig/batch0.tab.
+      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+
+      ===============================================================================
+
+      The content of package org.apache.commons.codec.language.bm has been translated
+      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+      with permission from the original authors.
+      Original source copyright:
+      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+   (ASLv2) JsonPath
+     The following NOTICE information applies:
+       Copyright 2011 JsonPath authors
+
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+      Copyright 2011 JSON-SMART authors
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (ASLv2) Apache Commons Collections
+    The following NOTICE information applies:
+      Apache Commons Collections
+      Copyright 2001-2013 The Apache Software Foundation
+
+  (ASLv2) Jettison
+    The following NOTICE information applies:
+         Copyright 2006 Envoi Solutions LLC
+
+  (ASLv2) Apache HttpComponents
+    The following NOTICE information applies:
+      Apache HttpClient
+      Copyright 1999-2015 The Apache Software Foundation
+
+      Apache HttpCore
+      Copyright 2005-2015 The Apache Software Foundation
+
+      Apache HttpMime
+      Copyright 1999-2013 The Apache Software Foundation
+
+      This project contains annotations derived from JCIP-ANNOTATIONS
+      Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net
+
+  (ASLv2) Apache Commons Logging
+    The following NOTICE information applies:
+      Apache Commons Logging
+      Copyright 2003-2014 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Net
+    The following NOTICE information applies:
+      Apache Commons Net
+      Copyright 2001-2013 The Apache Software Foundation
+
+  (ASLv2) Swagger Core
+    The following NOTICE information applies:
+      Swagger Core 1.5.3-M1
+      Copyright 2015 Reverb Technologies, Inc.
+
+  (ASLv2) Spring Security
+    The following NOTICE information applies:
+          Spring Framework 4.0.3.RELEASE
+          Copyright (c) 2002-2015 Pivotal, Inc.
+
+  (ASLv2) Spring Framework
+    The following NOTICE information applies:
+      Spring Framework 4.1.4.RELEASE
+      Copyright (c) 2002-2015 Pivotal, Inc.
+
+    (ASLv2) Quartz
+      The following NOTICE information applies:
+        Copyright Declaration:
+        Copyright � 2003-2016 Software AG, Darmstadt, Germany and/or Software AG USA Inc., Reston, VA, USA, and/or its subsidiaries and/or its affiliates and/or their licensors.
+
+        Trademark and Patent declaration
+        The name Software AG and all Software AG product names are either trademarks or registered trademarks of Software AG and/or Software AG USA Inc. and/or its subsidiaries and/or its affiliates
+        and/or their licensors. Other company and product names mentioned herein may be trademarks of their respective owners.
+
+        Detailed information on trademarks and patents owned by Software AG and/or its subsidiaries is located at http://softwareag.com/licenses.
+
+        Third Party declaration
+        This software may include portions of third-party products. For third-party copyright notices, license terms, additional rights or restrictions, please refer to "License Texts, Copyright
+        Notices and Disclaimers of Third Party Products". For certain specific third-party license restrictions, please refer to section E of the Legal Notices available under "License Terms and
+        Conditions for Use of Software AG Products / Copyright and Trademark Notices of Software AG Products". These documents are part of the product documentation, located at
+        http://softwareag.com/licenses and/or in the root installation directory of the licensed product(s).
+
+        Confidentiality Disclaimer:
+        Use, reproduction, transfer, publication or disclosure is prohibited except as specifically provided for in your License Agreement with Software AG.
+        Contact GitHub API Training Shop Blog About
+
+  (ASLv2) Jasypt
+    The following NOTICE information applies:
+	  Copyright (c) 2007-2010, The JASYPT team (http://www.jasypt.org)
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache ZooKeeper
+    The following NOTICE information applies:
+      Apache ZooKeeper
+      Copyright 2009-2012 The Apache Software Foundation
+
+  (ASLv2) Apache log4j
+    The following NOTICE information applies:
+      Apache log4j
+      Copyright 2007 The Apache Software Foundation
+
+  (ASLv2) The Netty Project
+    The following NOTICE information applies:
+      The Netty Project
+      Copyright 2011 The Netty Project
+
+  (ASLv2) Apache Curator
+    The following NOTICE information applies:
+      Curator Framework
+      Copyright 2011-2014 The Apache Software Foundation
+
+      Curator Client
+      Copyright 2011-2014 The Apache Software Foundation
+
+      Curator Recipes
+      Copyright 2011-2014 The Apache Software Foundation
+
+************************
+Common Development and Distribution License 1.1
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
+
+    (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:jar:1.19 - https://jersey.java.net/jersey-json/)
+    (CDDL 1.1) (GPL2 w/ CPE) Old JAXB Runtime (com.sun.xml.bind:jaxb-impl:jar:2.2.3-1 - http://jaxb.java.net/)
+    (CDDL 1.1) (GPL2 w/ CPE) Java Architecture For XML Binding (javax.xml.bind:jaxb-api:jar:2.2.2 - https://jaxb.dev.java.net/)
+    (CDDL 1.1) (GPL2 w/ CPE) aopalliance version 1.0 repackaged as a module (org.glassfish.hk2.external:aopalliance-repackaged:jar:2.4.0-b25 - https://hk2.java.net/external/aopalliance-repackaged)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.19 - https://jersey.java.net/jersey-client/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.19 - https://jersey.java.net/jersey-core/)
+
+
+************************
+Common Development and Distribution License 1.0
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.0.  See project link for details.
+
+    (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
+    (CDDL 1.0) (GPL3) Streaming API For XML (javax.xml.stream:stax-api:jar:1.0-2 - no url provided)
+    (CDDL 1.0) JSR311 API (javax.ws.rs:jsr311-api:jar:1.1.1 - https://jsr311.dev.java.net)
+
+
+************************
+Eclipse Public License 1.0
+************************
+
+The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
+
+    (EPL 1.0) AspectJ Weaver (org.aspectj:aspectjweaver:jar:1.8.5 - http://www.eclipse.org/aspectj/)
+    (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.1.3 - http://logback.qos.ch/)
+    (EPL 1.0)(LGPL 2.1) Logback Core (ch.qos.logback:logback-core:jar:1.1.3 - http://logback.qos.ch/)
+    (EPL 1.0)(MPL 2.0) H2 Database (com.h2database:h2:jar:1.3.176 - http://www.h2database.com/html/license.html)
\ No newline at end of file