You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by kd...@apache.org on 2019/02/13 20:07:06 UTC

[nifi-maven] 19/20: NIFI-5859: Build NAR Extension Definitions/docs at build time

This is an automated email from the ASF dual-hosted git repository.

kdoran pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi-maven.git

commit f9f372e731441f1288933111586a937bbae3a65a
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed Nov 14 15:09:25 2018 -0500

    NIFI-5859: Build NAR Extension Definitions/docs at build time
---
 pom.xml                                            |  36 +-
 src/main/java/org/apache/nifi/NarMojo.java         | 342 ++++++++++++++++-
 .../apache/nifi/PropertiesDefinitionWriter.java    |  75 ++++
 .../java/org/apache/nifi/XmlDefinitionWriter.java  | 165 ++++++++
 .../extension/definition/ExtensionDefinition.java  |  52 +++
 .../nifi/extension/definition/ExtensionType.java   |  27 ++
 .../nifi/extension/definition/Restriction.java     |  29 ++
 .../nifi/extension/definition/Restrictions.java    |  31 ++
 .../extension/definition/ServiceAPIDefinition.java |  27 ++
 .../extraction/ExtensionClassLoader.java           | 114 ++++++
 .../extraction/ExtensionClassLoaderFactory.java    | 426 +++++++++++++++++++++
 .../extraction/ExtensionDefinitionFactory.java     | 247 ++++++++++++
 .../extraction/StandardExtensionDefinition.java    |  78 ++++
 .../definition/extraction/StandardRestriction.java |  39 ++
 .../extraction/StandardRestrictions.java           |  42 ++
 .../extraction/StandardServiceAPIDefinition.java   |  53 +++
 16 files changed, 1754 insertions(+), 29 deletions(-)

diff --git a/pom.xml b/pom.xml
index daa17cb..35f54ab 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,7 +80,7 @@
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <inceptionYear>2014</inceptionYear>
-    </properties>    
+    </properties>
     <build>
         <pluginManagement>
             <plugins>
@@ -224,7 +224,7 @@
                                 <requireMavenVersion>
                                     <version>${maven.min-version}</version>
                                 </requireMavenVersion>
-                            </rules>    
+                            </rules>
                         </configuration>
                     </execution>
                 </executions>
@@ -313,7 +313,7 @@
                     <includeTestSourceDirectory>true</includeTestSourceDirectory>
                     <excludes>**/HelpMojo.java</excludes>
                 </configuration>
-            </plugin>      
+            </plugin>
             <plugin>
                 <groupId>org.apache.rat</groupId>
                 <artifactId>apache-rat-plugin</artifactId>
@@ -338,7 +338,7 @@
                         </exclusions>
                     </dependency>
                 </dependencies>
-            </plugin>             
+            </plugin>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-plugin-plugin</artifactId>
@@ -358,21 +358,27 @@
                         <phase>generate-sources</phase>
                     </execution>
                 </executions>
-            </plugin>
-        </plugins>        
+            </plugin><plugin><groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+                <source>8</source>
+                <target>8</target>
+            </configuration>
+        </plugin>
+        </plugins>
     </build>
     <dependencies>
-        <dependency>          
+        <dependency>
             <groupId>org.apache.maven</groupId>
             <artifactId>maven-plugin-api</artifactId>
             <version>3.1.1</version>
         </dependency>
-        <dependency>          
+        <dependency>
             <groupId>org.apache.maven</groupId>
             <artifactId>maven-artifact</artifactId>
             <version>3.1.1</version>
         </dependency>
-        <dependency>          
+        <dependency>
             <groupId>org.apache.maven</groupId>
             <artifactId>maven-compat</artifactId>
             <version>3.1.1</version>
@@ -383,7 +389,7 @@
             <type>maven-plugin</type>
             <version>2.9</version>
         </dependency>
-        <dependency>          
+        <dependency>
             <groupId>org.apache.maven.shared</groupId>
             <artifactId>maven-dependency-tree</artifactId>
             <version>2.2</version>
@@ -404,9 +410,9 @@
     </dependencies>
     <profiles>
         <profile>
-            <!-- 
-            Checks style and licensing requirements.  This is a good idea to 
-            run for contributions and for the release process.  While it 
+            <!--
+            Checks style and licensing requirements.  This is a good idea to
+            run for contributions and for the release process.  While it
             would be nice to run always these plugins can considerably slow
             the build and have proven to create unstable builds in our
             multi-module project and when building using multiple threads.
@@ -437,10 +443,10 @@
                                     <goal>check</goal>
                                 </goals>
                             </execution>
-                        </executions>                
+                        </executions>
                     </plugin>
                 </plugins>
             </build>
         </profile>
-    </profiles>    
+    </profiles>
 </project>
diff --git a/src/main/java/org/apache/nifi/NarMojo.java b/src/main/java/org/apache/nifi/NarMojo.java
index 4110fa6..85ebfcf 100644
--- a/src/main/java/org/apache/nifi/NarMojo.java
+++ b/src/main/java/org/apache/nifi/NarMojo.java
@@ -16,18 +16,13 @@
  */
 package org.apache.nifi;
 
-import java.io.File;
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 import org.apache.maven.archiver.MavenArchiveConfiguration;
 import org.apache.maven.archiver.MavenArchiver;
 import org.apache.maven.artifact.Artifact;
 import org.apache.maven.artifact.DependencyResolutionRequiredException;
 import org.apache.maven.artifact.factory.ArtifactFactory;
+import org.apache.maven.artifact.handler.ArtifactHandler;
+import org.apache.maven.artifact.handler.manager.ArtifactHandlerManager;
 import org.apache.maven.artifact.installer.ArtifactInstaller;
 import org.apache.maven.artifact.metadata.ArtifactMetadataSource;
 import org.apache.maven.artifact.repository.ArtifactRepository;
@@ -36,6 +31,7 @@ import org.apache.maven.artifact.resolver.ArtifactCollector;
 import org.apache.maven.artifact.resolver.ArtifactNotFoundException;
 import org.apache.maven.artifact.resolver.ArtifactResolutionException;
 import org.apache.maven.artifact.resolver.ArtifactResolver;
+import org.apache.maven.execution.MavenSession;
 import org.apache.maven.plugin.AbstractMojo;
 import org.apache.maven.plugin.MojoExecutionException;
 import org.apache.maven.plugin.MojoFailureException;
@@ -46,29 +42,70 @@ import org.apache.maven.plugin.dependency.utils.resolvers.ArtifactsResolver;
 import org.apache.maven.plugin.dependency.utils.resolvers.DefaultArtifactsResolver;
 import org.apache.maven.plugin.dependency.utils.translators.ArtifactTranslator;
 import org.apache.maven.plugin.dependency.utils.translators.ClassifierTypeTranslator;
+import org.apache.maven.plugins.annotations.Component;
 import org.apache.maven.plugins.annotations.LifecyclePhase;
 import org.apache.maven.plugins.annotations.Mojo;
 import org.apache.maven.plugins.annotations.Parameter;
 import org.apache.maven.plugins.annotations.ResolutionScope;
 import org.apache.maven.project.MavenProject;
-import org.apache.maven.execution.MavenSession;
-import org.apache.maven.plugins.annotations.Component;
 import org.apache.maven.project.MavenProjectHelper;
+import org.apache.maven.project.ProjectBuilder;
 import org.apache.maven.shared.artifact.filter.collection.ArtifactFilterException;
 import org.apache.maven.shared.artifact.filter.collection.ArtifactIdFilter;
 import org.apache.maven.shared.artifact.filter.collection.ArtifactsFilter;
 import org.apache.maven.shared.artifact.filter.collection.ClassifierFilter;
 import org.apache.maven.shared.artifact.filter.collection.FilterArtifacts;
 import org.apache.maven.shared.artifact.filter.collection.GroupIdFilter;
-import org.apache.maven.shared.artifact.filter.collection.ScopeFilter;
 import org.apache.maven.shared.artifact.filter.collection.ProjectTransitivityFilter;
+import org.apache.maven.shared.artifact.filter.collection.ScopeFilter;
 import org.apache.maven.shared.artifact.filter.collection.TypeFilter;
+import org.apache.maven.shared.dependency.tree.DependencyTreeBuilder;
+import org.apache.nifi.extension.definition.ExtensionDefinition;
+import org.apache.nifi.extension.definition.ExtensionType;
+import org.apache.nifi.extension.definition.ServiceAPIDefinition;
+import org.apache.nifi.extension.definition.extraction.ExtensionClassLoader;
+import org.apache.nifi.extension.definition.extraction.ExtensionClassLoaderFactory;
+import org.apache.nifi.extension.definition.extraction.ExtensionDefinitionFactory;
 import org.codehaus.plexus.archiver.ArchiverException;
 import org.codehaus.plexus.archiver.jar.JarArchiver;
 import org.codehaus.plexus.archiver.jar.ManifestException;
 import org.codehaus.plexus.archiver.manager.ArchiverManager;
 import org.codehaus.plexus.util.FileUtils;
 import org.codehaus.plexus.util.StringUtils;
+import org.eclipse.aether.RepositorySystemSession;
+
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamWriter;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.stream.Collectors;
 
 /**
  * Packages the current project as an Apache NiFi Archive (NAR).
@@ -80,6 +117,8 @@ import org.codehaus.plexus.util.StringUtils;
  */
 @Mojo(name = "nar", defaultPhase = LifecyclePhase.PACKAGE, threadSafe = true, requiresDependencyResolution = ResolutionScope.RUNTIME)
 public class NarMojo extends AbstractMojo {
+    private static final String SERVICES_DIRECTORY = "META-INF/services/";
+    private static final String DOCUMENTATION_WRITER_CLASS_NAME = "org.apache.nifi.documentation.xml.XmlDocumentationWriter";
 
     private static final String[] DEFAULT_EXCLUDES = new String[]{"**/package.html"};
     private static final String[] DEFAULT_INCLUDES = new String[]{"**/**"};
@@ -357,6 +396,20 @@ public class NarMojo extends AbstractMojo {
     public boolean silent;
 
     /**
+     * The dependency tree builder to use for verbose output.
+     */
+    @Component
+    private DependencyTreeBuilder dependencyTreeBuilder;
+
+    /**
+     * *
+     * The {@link ArtifactHandlerManager} into which any extension {@link ArtifactHandler} instances should have been injected when the extensions were loaded.
+     */
+    @Component
+    private ArtifactHandlerManager artifactHandlerManager;
+
+
+    /**
      * Output absolute filename for resolved artifacts
      *
      */
@@ -426,12 +479,260 @@ public class NarMojo extends AbstractMojo {
     protected boolean cloneDuringInstanceClassLoading;
 
 
+    /**
+     * The {@link RepositorySystemSession} used for obtaining the local and remote artifact repositories.
+     */
+    @Parameter(defaultValue = "${repositorySystemSession}", readonly = true)
+    private RepositorySystemSession repoSession;
+
+
+    /**
+     * The {@link ProjectBuilder} used to generate the {@link MavenProject} for the nar artifact the dependency tree is being generated for.
+     */
+    @Component
+    private ProjectBuilder projectBuilder;
+
+
+
     @Override
-    public void execute() throws MojoExecutionException, MojoFailureException {
+    public void execute() throws MojoExecutionException {
         copyDependencies();
+
+        try {
+            generateDocumentation();
+        } catch (final Throwable t) { // Catch Throwable in case a linkage error such as NoClassDefFoundError occurs
+            getLog().warn("Could not generate extensions' documentation", t);
+        }
+
         makeNar();
     }
 
+    private File getExtensionsDocumentationFile() {
+        final File directory = new File(projectBuildDirectory, "META-INF/docs");
+        return new File(directory, "extension-docs.xml");
+    }
+
+    private void generateDocumentation() throws MojoExecutionException {
+        getLog().info("Generating documentation for NiFi extensions in the NAR...");
+
+        // Create the ClassLoader for the NAR
+        final ExtensionClassLoaderFactory classLoaderFactory = createClassLoaderFactory();
+
+        final ExtensionClassLoader extensionClassLoader;
+        try {
+            extensionClassLoader = classLoaderFactory.createExtensionClassLoader();
+        } catch (final Exception e) {
+            if (getLog().isDebugEnabled()) {
+                getLog().debug("Unable to create a ClassLoader for documenting extensions. If this NAR contains any NiFi Extensions, those extensions will not be documented.", e);
+            } else {
+                getLog().warn("Unable to create a ClassLoader for documenting extensions. If this NAR contains any NiFi Extensions, those extensions will not be documented. " +
+                    "Enable mvn DEBUG output for more information (mvn -X).");
+            }
+
+            return;
+        }
+
+
+        final File docsFile = getExtensionsDocumentationFile();
+        createDirectory(docsFile.getParentFile());
+
+        final File additionalDetailsDir = new File(docsFile.getParentFile(), "additional-details");
+        createDirectory(additionalDetailsDir);
+
+        try (final OutputStream out = new FileOutputStream(docsFile)) {
+
+            final XMLStreamWriter xmlWriter = XMLOutputFactory.newInstance().createXMLStreamWriter(out, "UTF-8");
+            try {
+                xmlWriter.writeStartElement("extensions");
+
+                final String nifiApiVersion = extensionClassLoader.getNiFiApiVersion();
+                xmlWriter.writeStartElement("nifiApiVersion");
+                xmlWriter.writeCharacters(nifiApiVersion);
+                xmlWriter.writeEndElement();
+
+                final Class<?> docWriterClass;
+                try {
+                    docWriterClass = Class.forName(DOCUMENTATION_WRITER_CLASS_NAME, false, extensionClassLoader);
+                } catch (ClassNotFoundException e) {
+                    getLog().warn("Cannot locate class " + DOCUMENTATION_WRITER_CLASS_NAME + ", so no documentation will be generated for the extensions in this NAR");
+                    return;
+                }
+
+                getLog().debug("Creating Extension Definition Factory for NiFi API version " + nifiApiVersion);
+
+                final ExtensionDefinitionFactory extensionDefinitionFactory = new ExtensionDefinitionFactory(extensionClassLoader);
+
+                final ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
+                try {
+                    Thread.currentThread().setContextClassLoader(extensionClassLoader);
+
+                    final Set<ExtensionDefinition> processorDefinitions = extensionDefinitionFactory.discoverExtensions(ExtensionType.PROCESSOR);
+                    writeDocumentation(processorDefinitions, extensionClassLoader, docWriterClass, xmlWriter, additionalDetailsDir);
+
+                    final Set<ExtensionDefinition> controllerServiceDefinitions = extensionDefinitionFactory.discoverExtensions(ExtensionType.CONTROLLER_SERVICE);
+                    writeDocumentation(controllerServiceDefinitions, extensionClassLoader, docWriterClass, xmlWriter, additionalDetailsDir);
+
+                    final Set<ExtensionDefinition> reportingTaskDefinitions = extensionDefinitionFactory.discoverExtensions(ExtensionType.REPORTING_TASK);
+                    writeDocumentation(reportingTaskDefinitions, extensionClassLoader, docWriterClass, xmlWriter, additionalDetailsDir);
+                } finally {
+                    if (currentContextClassLoader != null) {
+                        Thread.currentThread().setContextClassLoader(currentContextClassLoader);
+                    }
+                }
+
+                xmlWriter.writeEndElement();
+            } finally {
+                xmlWriter.close();
+            }
+        } catch (final Exception ioe) {
+            throw new MojoExecutionException("Failed to create Extension Documentation", ioe);
+        }
+    }
+
+    private void writeDocumentation(final Set<ExtensionDefinition> extensionDefinitions, final ExtensionClassLoader classLoader,
+                                    final Class<?> docWriterClass, final XMLStreamWriter xmlWriter, final File additionalDetailsDir)
+        throws InvocationTargetException, NoSuchMethodException, ClassNotFoundException, InstantiationException, IllegalAccessException, IOException {
+
+        for (final ExtensionDefinition definition : extensionDefinitions) {
+            writeDocumentation(definition, classLoader, docWriterClass, xmlWriter);
+        }
+
+        final Set<String> extensionNames = extensionDefinitions.stream()
+            .map(ExtensionDefinition::getExtensionName)
+            .collect(Collectors.toSet());
+
+        try {
+            writeAdditionalDetails(classLoader, extensionNames, additionalDetailsDir);
+        } catch (final Exception e) {
+            throw new IOException("Unable to extract Additional Details", e);
+        }
+
+    }
+
+    private void writeDocumentation(final ExtensionDefinition extensionDefinition, final ExtensionClassLoader classLoader,
+                                    final Class<?> docWriterClass, final XMLStreamWriter xmlWriter)
+        throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, IOException {
+
+        getLog().debug("Generating documentation for " + extensionDefinition.getExtensionName() + " using ClassLoader:\n" + classLoader.toTree());
+        final Object docWriter = docWriterClass.getConstructor(XMLStreamWriter.class).newInstance(xmlWriter);
+        final Class<?> configurableComponentClass = Class.forName("org.apache.nifi.components.ConfigurableComponent", false, classLoader);
+
+        final Class<?> extensionClass = Class.forName(extensionDefinition.getExtensionName(), false, classLoader);
+        final Object extensionInstance = extensionClass.newInstance();
+
+        final Set<ServiceAPIDefinition> serviceDefinitions = extensionDefinition.getProvidedServiceAPIs();
+
+        if (serviceDefinitions == null || serviceDefinitions.isEmpty()) {
+            final Method writeMethod = docWriterClass.getMethod("write", configurableComponentClass);
+            writeMethod.invoke(docWriter, extensionInstance);
+        } else {
+            final Class<?> providedServiceApiClass = Class.forName("org.apache.nifi.documentation.StandardProvidedServiceAPI", false, classLoader);
+            final Constructor<?> ctr = providedServiceApiClass.getConstructor(String.class, String.class, String.class, String.class);
+
+            final List<Object> providedServices = new ArrayList<>();
+
+            for (final ServiceAPIDefinition definition : serviceDefinitions) {
+                final Object serviceApi = ctr.newInstance(definition.getServiceAPIClassName(), definition.getServiceGroupId(), definition.getServiceArtifactId(), definition.getServiceVersion());
+                providedServices.add(serviceApi);
+            }
+
+            final Method writeMethod = docWriterClass.getMethod("write", configurableComponentClass, Collection.class);
+            writeMethod.invoke(docWriter, extensionInstance, providedServices);
+        }
+    }
+
+    private void writeAdditionalDetails(final ExtensionClassLoader classLoader, final Set<String> extensionNames, final File additionalDetailsDir)
+        throws URISyntaxException, IOException, MojoExecutionException {
+
+        for (final URL url : classLoader.getURLs()) {
+            final File file = new File(url.toURI());
+            final String filename = file.getName();
+            if (!filename.endsWith(".jar")) {
+                continue;
+            }
+
+            writeAdditionalDetails(file, extensionNames, additionalDetailsDir);
+        }
+    }
+
+    private void writeAdditionalDetails(final File file, final Set<String> extensionNames, final File additionalDetailsDir) throws IOException, MojoExecutionException {
+        final JarFile jarFile = new JarFile(file);
+
+        for (final Enumeration<JarEntry> jarEnumeration = jarFile.entries(); jarEnumeration.hasMoreElements();) {
+            final JarEntry jarEntry = jarEnumeration.nextElement();
+
+            final String entryName = jarEntry.getName();
+            if (!entryName.startsWith("docs/")) {
+                continue;
+            }
+
+            final int nextSlashIndex = entryName.indexOf("/", 5);
+            if (nextSlashIndex < 0) {
+                continue;
+            }
+
+            final String componentName = entryName.substring(5, nextSlashIndex);
+            if (!extensionNames.contains(componentName)) {
+                continue;
+            }
+
+            if (jarEntry.isDirectory()) {
+                continue;
+            }
+
+            if (entryName.length() < nextSlashIndex + 1) {
+                continue;
+            }
+
+            getLog().debug("Found file " + entryName + " in " + file + " that consists of documentation for " + componentName);
+            final File componentDirectory = new File(additionalDetailsDir, componentName);
+            final String remainingPath = entryName.substring(nextSlashIndex + 1);
+            final File destinationFile = new File(componentDirectory, remainingPath);
+
+            createDirectory(destinationFile.getParentFile());
+
+            try (final InputStream in = jarFile.getInputStream(jarEntry);
+                 final OutputStream out = new FileOutputStream(destinationFile)) {
+                copy(in, out);
+            }
+        }
+    }
+
+    private void copy(final InputStream in, final OutputStream out) throws IOException {
+        final byte[] buffer = new byte[8192];
+        int len;
+        while ((len = in.read(buffer)) >= 0) {
+            out.write(buffer, 0, len);
+        }
+    }
+
+
+    private ExtensionClassLoaderFactory createClassLoaderFactory() {
+        return new ExtensionClassLoaderFactory.Builder()
+            .artifactResolver(resolver)
+            .dependencyTreeBuilder(dependencyTreeBuilder)
+            .localRepository(local)
+            .log(getLog())
+            .project(project)
+            .projectBuilder(projectBuilder)
+            .remoteRepositories(remoteRepos)
+            .repositorySession(repoSession)
+            .artifactHandlerManager(artifactHandlerManager)
+            .build();
+    }
+
+
+    private void createDirectory(final File file) throws MojoExecutionException {
+        if (!file.exists()) {
+            try {
+                Files.createDirectories(file.toPath());
+            } catch (IOException e) {
+                throw new MojoExecutionException("Could not create directory " + file, e);
+            }
+        }
+    }
+
+
     private void copyDependencies() throws MojoExecutionException {
         DependencyStatusSets dss = getDependencySets(this.failOnMissingClassifierArtifact);
         Set artifacts = dss.getResolvedDependencies();
@@ -443,7 +744,7 @@ public class NarMojo extends AbstractMojo {
         artifacts = dss.getSkippedDependencies();
         for (Object artifactOjb : artifacts) {
             Artifact artifact = (Artifact) artifactOjb;
-            getLog().info(artifact.getFile().getName() + " already exists in destination.");
+            getLog().debug(artifact.getFile().getName() + " already exists in destination.");
         }
     }
 
@@ -469,6 +770,7 @@ public class NarMojo extends AbstractMojo {
         return new DestFileFilter(this.overWriteReleases, this.overWriteSnapshots, this.overWriteIfNewer, false, false, false, false, false, getDependenciesDirectory());
     }
 
+
     protected DependencyStatusSets getDependencySets(boolean stopOnFailure) throws MojoExecutionException {
         // add filters in well known order, least specific to most specific
         FilterArtifacts filter = new FilterArtifacts();
@@ -597,10 +899,22 @@ public class NarMojo extends AbstractMojo {
 
         try {
             File contentDirectory = getClassesDirectory();
-            if (!contentDirectory.exists()) {
+            if (contentDirectory.exists()) {
+                archiver.getArchiver().addDirectory(contentDirectory, getIncludes(), getExcludes());
+            } else {
                 getLog().warn("NAR will be empty - no content was marked for inclusion!");
+            }
+
+            File extensionDocsFile = getExtensionsDocumentationFile();
+            if (extensionDocsFile.exists()) {
+                archiver.getArchiver().addFile(extensionDocsFile, "META-INF/docs/" + extensionDocsFile.getName());
             } else {
-                archiver.getArchiver().addDirectory(contentDirectory, getIncludes(), getExcludes());
+                getLog().warn("NAR will not contain any Extensions' documentation - no META-INF/" + extensionDocsFile.getName() + " file found!");
+            }
+
+            File additionalDetailsDirectory = new File(getExtensionsDocumentationFile().getParentFile(), "additional-details");
+            if (additionalDetailsDirectory.exists()) {
+                archiver.getArchiver().addDirectory(additionalDetailsDirectory, "META-INF/docs/additional-details/");
             }
 
             File existingManifest = defaultManifestFile;
diff --git a/src/main/java/org/apache/nifi/PropertiesDefinitionWriter.java b/src/main/java/org/apache/nifi/PropertiesDefinitionWriter.java
new file mode 100644
index 0000000..402d18c
--- /dev/null
+++ b/src/main/java/org/apache/nifi/PropertiesDefinitionWriter.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi;
+
+import org.apache.nifi.extension.definition.ExtensionDefinition;
+import org.apache.nifi.extension.definition.Restriction;
+import org.apache.nifi.extension.definition.Restrictions;
+import org.apache.nifi.extension.definition.ServiceAPIDefinition;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Objects;
+import java.util.Properties;
+
+public class PropertiesDefinitionWriter {
+
+    public void writeDefinition(final ExtensionDefinition definition, final File file) throws IOException {
+        Objects.requireNonNull(definition);
+
+        final String capabilityDescription = definition.getCapabilityDescription();
+
+        final Properties properties = new Properties();
+        if (capabilityDescription != null) {
+            properties.setProperty("capability.description", capabilityDescription);
+        }
+
+        int i=0;
+        for (final String tag : definition.getTags()) {
+            properties.setProperty("tags." + (i++), tag);
+        }
+
+        final Restrictions restrictions = definition.getRestrictions();
+        if (restrictions != null) {
+            final String restrictedDescription = restrictions.getGeneralRestrictionExplanation();
+
+            if (restrictedDescription != null) {
+                properties.setProperty("restricted.description", restrictedDescription);
+            }
+
+            for (final Restriction restriction : restrictions.getRestrictions()) {
+                properties.setProperty("restriction." + restriction.getIdentifier(), restriction.getExplanation());
+            }
+        }
+
+        int serviceIndex = 0;
+        for (final ServiceAPIDefinition apiDefinition : definition.getProvidedServiceAPIs()) {
+            properties.setProperty("service.definition." + serviceIndex + ".class", apiDefinition.getServiceAPIClassName());
+            properties.setProperty("service.definition." + serviceIndex + ".groupId", apiDefinition.getServiceGroupId());
+            properties.setProperty("service.definition." + serviceIndex + ".artifactId", apiDefinition.getServiceArtifactId());
+            properties.setProperty("service.definition." + serviceIndex + ".version", apiDefinition.getServiceVersion());
+
+            serviceIndex++;
+        }
+
+        try (final OutputStream fos = new FileOutputStream(file)) {
+            properties.store(fos, null);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/nifi/XmlDefinitionWriter.java b/src/main/java/org/apache/nifi/XmlDefinitionWriter.java
new file mode 100644
index 0000000..3e91061
--- /dev/null
+++ b/src/main/java/org/apache/nifi/XmlDefinitionWriter.java
@@ -0,0 +1,165 @@
+/*
+ * 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;
+
+import org.apache.nifi.extension.definition.ExtensionDefinition;
+import org.apache.nifi.extension.definition.ExtensionType;
+import org.apache.nifi.extension.definition.Restriction;
+import org.apache.nifi.extension.definition.Restrictions;
+import org.apache.nifi.extension.definition.ServiceAPIDefinition;
+
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class XmlDefinitionWriter {
+
+    public void writeDefinition(final Collection<ExtensionDefinition> definitions, final File file) throws IOException {
+        final Map<ExtensionType, List<ExtensionDefinition>> definitionMap = definitions.stream()
+            .collect(Collectors.groupingBy(ExtensionDefinition::getExtensionType));
+
+        writeDefinition(definitionMap, file);
+    }
+
+    public void writeDefinition(final Map<ExtensionType, ? extends Collection<ExtensionDefinition>> definitions, final File file) throws IOException {
+        Objects.requireNonNull(definitions);
+        Objects.requireNonNull(file);
+
+        if (definitions.isEmpty()) {
+            return;
+        }
+
+        try (final OutputStream fileOut = new FileOutputStream(file)) {
+             final XMLStreamWriter writer = XMLOutputFactory.newInstance().createXMLStreamWriter(fileOut, "UTF-8");
+             try {
+                 writer.writeStartElement("extensions");
+
+                 writer.writeStartElement("processors");
+                 writeDefinitions(ExtensionType.PROCESSOR, definitions.get(ExtensionType.PROCESSOR), writer);
+                 writer.writeEndElement();
+
+                 writer.writeStartElement("controllerServices");
+                 writeDefinitions(ExtensionType.CONTROLLER_SERVICE, definitions.get(ExtensionType.CONTROLLER_SERVICE), writer);
+                 writer.writeEndElement();
+
+                 writer.writeStartElement("reportingTasks");
+                 writeDefinitions(ExtensionType.REPORTING_TASK, definitions.get(ExtensionType.REPORTING_TASK), writer);
+                 writer.writeEndElement();
+
+                 writer.writeEndElement();
+             } finally {
+                 writer.close();
+             }
+        } catch (XMLStreamException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void writeDefinitions(final ExtensionType extensionType, final Collection<ExtensionDefinition> definitions, final XMLStreamWriter writer) throws XMLStreamException {
+        if (definitions == null) {
+            return;
+        }
+
+        final String tagName;
+        switch (extensionType) {
+            case PROCESSOR:
+                tagName = "processor";
+                break;
+            case CONTROLLER_SERVICE:
+                tagName = "controllerService";
+                break;
+            case REPORTING_TASK:
+                tagName = "reportingTask";
+                break;
+            default:
+                throw new AssertionError("Encountered unknown Extension Type " + extensionType);
+        }
+
+        for (final ExtensionDefinition definition : definitions) {
+            writer.writeStartElement(tagName);
+
+            writeTextElement(writer, "name", definition.getExtensionName());
+            writeTextElement(writer, "description", definition.getCapabilityDescription());
+
+            writer.writeStartElement("tags");
+            for (final String tag : definition.getTags()) {
+                writeTextElement(writer, "tag", tag);
+            }
+            writer.writeEndElement();
+
+            final Restrictions restrictions = definition.getRestrictions();
+            if (restrictions == null) {
+                writer.writeEmptyElement("restrictions");
+            } else {
+                writer.writeStartElement("restrictions");
+
+                writeTextElement(writer, "explanation", restrictions.getGeneralRestrictionExplanation());
+                final Set<Restriction> specificRestrictions = restrictions.getRestrictions();
+                for (final Restriction restriction : specificRestrictions) {
+                    writer.writeStartElement("restriction");
+                    writeTextElement(writer, "identifier", restriction.getIdentifier());
+                    writeTextElement(writer, "explanation", restriction.getExplanation());
+                    writer.writeEndElement();
+                }
+
+                writer.writeEndElement();
+            }
+
+            if (extensionType == ExtensionType.CONTROLLER_SERVICE) {
+                writer.writeStartElement("providedServiceAPIs");
+
+                final Set<ServiceAPIDefinition> serviceDefinitions = definition.getProvidedServiceAPIs();
+                if (serviceDefinitions != null) {
+                    for (final ServiceAPIDefinition serviceDefinition : serviceDefinitions) {
+                        writer.writeStartElement("service");
+
+                        writeTextElement(writer, "className", serviceDefinition.getServiceAPIClassName());
+                        writeTextElement(writer, "groupId", serviceDefinition.getServiceGroupId());
+                        writeTextElement(writer, "artifactId", serviceDefinition.getServiceArtifactId());
+                        writeTextElement(writer, "version", serviceDefinition.getServiceVersion());
+
+                        writer.writeEndElement();
+                    }
+                }
+
+                writer.writeEndElement();
+            }
+
+            writer.writeEndElement();
+        }
+    }
+
+    private void writeTextElement(final XMLStreamWriter writer, final String tagName, final String text) throws XMLStreamException {
+        writer.writeStartElement(tagName);
+
+        if (text != null) {
+            writer.writeCharacters(text);
+        }
+
+        writer.writeEndElement();
+    }
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/ExtensionDefinition.java b/src/main/java/org/apache/nifi/extension/definition/ExtensionDefinition.java
new file mode 100644
index 0000000..8c0bc40
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/ExtensionDefinition.java
@@ -0,0 +1,52 @@
+/*
+ * 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.extension.definition;
+
+import java.util.Set;
+
+public interface ExtensionDefinition {
+    /**
+     * @return the extension's capability description
+     */
+    String getCapabilityDescription();
+
+    /**
+     * @return the set of Tags associated with the extension
+     */
+    Set<String> getTags();
+
+    /**
+     * @return the Restrictions that are placed on the Extension
+     */
+    Restrictions getRestrictions();
+
+    /**
+     * @return the type of Extension
+     */
+    ExtensionType getExtensionType();
+
+    /**
+     * @return the Set of all Services API's that this extension provides. Note that this will be an empty set for
+     * any Extension for which {@link #getExtensionType()} is not {@link ExtensionType#CONTROLLER_SERVICE}.
+     */
+    Set<ServiceAPIDefinition> getProvidedServiceAPIs();
+
+    /**
+     * @return the name of the Extension
+     */
+    String getExtensionName();
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/ExtensionType.java b/src/main/java/org/apache/nifi/extension/definition/ExtensionType.java
new file mode 100644
index 0000000..55f76f5
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/ExtensionType.java
@@ -0,0 +1,27 @@
+/*
+ * 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.extension.definition;
+
+public enum ExtensionType {
+
+    PROCESSOR,
+
+    CONTROLLER_SERVICE,
+
+    REPORTING_TASK;
+
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/Restriction.java b/src/main/java/org/apache/nifi/extension/definition/Restriction.java
new file mode 100644
index 0000000..9b5064c
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/Restriction.java
@@ -0,0 +1,29 @@
+/*
+ * 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.extension.definition;
+
+public interface Restriction {
+    /**
+     * @return the Restriction identifier used by NiFi
+     */
+    String getIdentifier();
+
+    /**
+     * @return an Explanation of why the Restriction exists.
+     */
+    String getExplanation();
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/Restrictions.java b/src/main/java/org/apache/nifi/extension/definition/Restrictions.java
new file mode 100644
index 0000000..5e098fb
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/Restrictions.java
@@ -0,0 +1,31 @@
+/*
+ * 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.extension.definition;
+
+import java.util.Set;
+
+public interface Restrictions {
+    /**
+     * @return if the extension has a General Restriction on it, this explains why the Restriction is in place.
+     */
+    String getGeneralRestrictionExplanation();
+
+    /**
+     * @return the set of all specific Restrictions that are placed on this extension.
+     */
+    Set<Restriction> getRestrictions();
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/ServiceAPIDefinition.java b/src/main/java/org/apache/nifi/extension/definition/ServiceAPIDefinition.java
new file mode 100644
index 0000000..0167f40
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/ServiceAPIDefinition.java
@@ -0,0 +1,27 @@
+/*
+ * 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.extension.definition;
+
+public interface ServiceAPIDefinition {
+    String getServiceAPIClassName();
+
+    String getServiceGroupId();
+
+    String getServiceArtifactId();
+
+    String getServiceVersion();
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionClassLoader.java b/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionClassLoader.java
new file mode 100644
index 0000000..12f02ac
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionClassLoader.java
@@ -0,0 +1,114 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.maven.artifact.Artifact;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+
+public class ExtensionClassLoader extends URLClassLoader {
+    private final URL[] urls;
+    private final Artifact narArtifact;
+    private final Collection<Artifact> allArtifacts;
+
+    public ExtensionClassLoader(final URL[] urls, final ClassLoader parent, final Artifact narArtifact, final Collection<Artifact> otherArtifacts) {
+        super(urls, parent);
+        this.urls = urls;
+        this.narArtifact = narArtifact;
+        this.allArtifacts = new ArrayList<>(otherArtifacts);
+        allArtifacts.add(narArtifact);
+    }
+
+    public ExtensionClassLoader(final URL[] urls, final Artifact narArtifact, final Collection<Artifact> otherArtifacts) {
+        super(urls);
+        this.urls = urls;
+        this.narArtifact = narArtifact;
+        this.allArtifacts = new ArrayList<>(otherArtifacts);
+        allArtifacts.add(narArtifact);
+    }
+
+    public String getNiFiApiVersion() {
+        final Collection<Artifact> artifacts = getAllArtifacts();
+        for (final Artifact artifact : artifacts) {
+            if (artifact.getArtifactId().equals("nifi-api") && artifact.getGroupId().equals("org.apache.nifi")) {
+                return artifact.getVersion();
+            }
+        }
+
+        final ClassLoader parent = getParent();
+        if (parent instanceof ExtensionClassLoader) {
+            ((ExtensionClassLoader) parent).getNiFiApiVersion();
+        }
+
+        return null;
+    }
+
+    public Artifact getNarArtifact() {
+        return narArtifact;
+    }
+
+    public Collection<Artifact> getAllArtifacts() {
+        return allArtifacts;
+    }
+
+    @Override
+    public String toString() {
+        return "ExtensionClassLoader[" + narArtifact + ", Dependencies=" + Arrays.asList(urls) + "]";
+    }
+
+    public String toTree() {
+        final StringBuilder sb = new StringBuilder();
+        sb.append("ClassLoader for ").append(narArtifact).append(" with nifi-api version ").append(getNiFiApiVersion()).append(":\n");
+
+        for (final URL url : urls) {
+            sb.append(url).append("\n");
+        }
+
+        final ClassLoader parent = getParent();
+        if (parent instanceof ExtensionClassLoader) {
+            sb.append("\n\n-------- Parent:\n");
+            sb.append(((ExtensionClassLoader) parent).toTree());
+        } else if (parent instanceof URLClassLoader) {
+            sb.append(toTree((URLClassLoader) parent));
+        }
+
+        return sb.toString();
+    }
+
+    private String toTree(final URLClassLoader classLoader) {
+        final StringBuilder sb = new StringBuilder();
+
+        sb.append("\n\n-------- Parent:\n");
+        final URL[] urls = classLoader.getURLs();
+
+        for (final URL url : urls) {
+            sb.append(url).append("\n");
+        }
+
+        final ClassLoader parent = classLoader.getParent();
+        if (parent instanceof URLClassLoader) {
+            final URLClassLoader urlParent = (URLClassLoader) parent;
+            sb.append(toTree(urlParent));
+        }
+
+        return sb.toString();
+    }
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionClassLoaderFactory.java b/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionClassLoaderFactory.java
new file mode 100644
index 0000000..7b2e5f7
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionClassLoaderFactory.java
@@ -0,0 +1,426 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.maven.artifact.Artifact;
+import org.apache.maven.artifact.DefaultArtifact;
+import org.apache.maven.artifact.handler.ArtifactHandler;
+import org.apache.maven.artifact.handler.manager.ArtifactHandlerManager;
+import org.apache.maven.artifact.repository.ArtifactRepository;
+import org.apache.maven.artifact.resolver.ArtifactResolutionRequest;
+import org.apache.maven.artifact.resolver.ArtifactResolutionResult;
+import org.apache.maven.artifact.resolver.ArtifactResolver;
+import org.apache.maven.artifact.versioning.VersionRange;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.logging.Log;
+import org.apache.maven.project.DefaultProjectBuildingRequest;
+import org.apache.maven.project.MavenProject;
+import org.apache.maven.project.ProjectBuilder;
+import org.apache.maven.project.ProjectBuildingException;
+import org.apache.maven.project.ProjectBuildingRequest;
+import org.apache.maven.project.ProjectBuildingResult;
+import org.apache.maven.shared.dependency.tree.DependencyNode;
+import org.apache.maven.shared.dependency.tree.DependencyTreeBuilder;
+import org.apache.maven.shared.dependency.tree.DependencyTreeBuilderException;
+import org.apache.maven.shared.dependency.tree.traversal.DependencyNodeVisitor;
+import org.eclipse.aether.RepositorySystemSession;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+public class ExtensionClassLoaderFactory {
+    private final Log log;
+    private final MavenProject project;
+    private final RepositorySystemSession repoSession;
+    private final ProjectBuilder projectBuilder;
+    private final ArtifactRepository localRepo;
+    private final List<ArtifactRepository> remoteRepos;
+    private final DependencyTreeBuilder dependencyTreeBuilder;
+    private final ArtifactResolver artifactResolver;
+    private final ArtifactHandlerManager artifactHandlerManager;
+
+    private ExtensionClassLoaderFactory(final Builder builder) {
+        this.log = builder.log;
+        this.project = builder.project;
+        this.repoSession = builder.repositorySession;
+        this.projectBuilder = builder.projectBuilder;
+        this.localRepo = builder.localRepo;
+        this.remoteRepos = new ArrayList<>(builder.remoteRepos);
+        this.dependencyTreeBuilder = builder.dependencyTreeBuilder;
+        this.artifactResolver = builder.artifactResolver;
+        this.artifactHandlerManager = builder.artifactHandlerManager;
+    }
+
+    private Log getLog() {
+        return log;
+    }
+
+    public ExtensionClassLoader createExtensionClassLoader() throws MojoExecutionException {
+        final Set<Artifact> artifacts = new HashSet<>();
+        gatherArtifacts(project, artifacts);
+
+        getLog().debug("Project artifacts: ");
+        artifacts.forEach(artifact -> getLog().debug(artifact.toString()));
+
+        final Artifact narArtifact = project.getArtifact();
+
+        final VersionLookup versionLookup = (group, artifact) -> {
+            try {
+                return determineProvidedEntityVersion(artifacts, group, artifact);
+            } catch (final Exception e) {
+                throw new RuntimeException("Failed to determine provided version of NiFi dependencies", e);
+            }
+        };
+
+        final ClassLoader parentClassLoader = createClassLoader(artifacts, versionLookup);
+        final ExtensionClassLoader classLoader = createClassLoader(artifacts, parentClassLoader, narArtifact);
+
+        if (getLog().isDebugEnabled()) {
+            getLog().debug("Full ClassLoader is:\n" + classLoader.toTree());
+        }
+
+        return classLoader;
+    }
+
+    private ClassLoader createClassLoader(final Set<Artifact> artifacts, final VersionLookup versionLookup) throws MojoExecutionException {
+        final Artifact nar = removeNarArtifact(artifacts);
+        if (nar == null) {
+            final ClassLoader providedEntityClassLoader = createProvidedEntitiesClassLoader(versionLookup);
+            return createUrlClassLoader(artifacts, providedEntityClassLoader);
+        }
+
+        final Set<Artifact> narDependencies = getNarDependencies(nar);
+        return createClassLoader(narDependencies, createClassLoader(narDependencies, versionLookup), nar);
+    }
+
+
+    private Artifact removeNarArtifact(final Set<Artifact> artifacts) {
+        final Iterator<Artifact> itr = artifacts.iterator();
+        while (itr.hasNext()) {
+            final Artifact artifact = itr.next();
+
+            if (artifact.equals(project.getArtifact())) {
+                continue;
+            }
+
+            if ("nar".equalsIgnoreCase(artifact.getType())) {
+                getLog().info("Found NAR dependency of " + artifact);
+                itr.remove();
+
+                return artifact;
+            }
+        }
+
+        return null;
+    }
+
+    private Set<Artifact> getNarDependencies(final Artifact narArtifact) throws MojoExecutionException {
+        final ProjectBuildingRequest narRequest = new DefaultProjectBuildingRequest();
+        narRequest.setRepositorySession(repoSession);
+        narRequest.setSystemProperties(System.getProperties());
+
+        final Set<Artifact> narDependencies = new HashSet<>();
+
+        try {
+            final ProjectBuildingResult narResult = projectBuilder.build(narArtifact, narRequest);
+            gatherArtifacts(narResult.getProject(), narDependencies);
+            narDependencies.remove(narArtifact);
+            narDependencies.remove(project.getArtifact());
+
+            getLog().debug("Found NAR dependency of " + narArtifact + ", which resolved to the following artifacts: " + narDependencies);
+        } catch (ProjectBuildingException e) {
+            throw new MojoExecutionException("Could not build parent nar project");
+        }
+
+        return narDependencies;
+    }
+
+    private String determineProvidedEntityVersion(final Set<Artifact> artifacts, final String groupId, final String artifactId) throws ProjectBuildingException, MojoExecutionException {
+        getLog().debug("Determining provided entities for " + groupId + ":" + artifactId);
+
+        for (final Artifact artifact : artifacts) {
+            if (artifact.getGroupId().equals(groupId) && artifact.getArtifactId().equals(artifactId)) {
+                return artifact.getVersion();
+            }
+        }
+
+        return findProvidedDependencyVersion(artifacts, groupId, artifactId);
+    }
+
+    private String findProvidedDependencyVersion(final Set<Artifact> artifacts, final String groupId, final String artifactId) throws ProjectBuildingException, MojoExecutionException {
+        final ProjectBuildingRequest narRequest = new DefaultProjectBuildingRequest();
+        narRequest.setRepositorySession(repoSession);
+        narRequest.setSystemProperties(System.getProperties());
+
+        for (final Artifact artifact : artifacts) {
+            final Set<Artifact> artifactDependencies = new HashSet<>();
+
+            try {
+                final ProjectBuildingResult projectResult = projectBuilder.build(artifact, narRequest);
+                gatherArtifacts(projectResult.getProject(), artifactDependencies);
+
+                getLog().debug("For Artifact " + artifact + ", found the following dependencies:");
+                artifactDependencies.forEach(dep -> getLog().debug(dep.toString()));
+
+                for (final Artifact dependency : artifactDependencies) {
+                    if (dependency.getGroupId().equals(groupId) && dependency.getArtifactId().equals(artifactId)) {
+                        getLog().debug("Found version of " + groupId + ":" + artifactId + " to be " + artifact.getVersion());
+                        return artifact.getVersion();
+                    }
+                }
+            } catch (final Exception e) {
+                getLog().warn("Unable to construct Maven Project for " + artifact + " when attempting to determine the expected version of NiFi API");
+                getLog().debug("Unable to construct Maven Project for " + artifact + " when attempting to determine the expected version of NiFi API", e);
+            }
+        }
+
+        return null;
+    }
+
+    private Artifact getProvidedArtifact(final String groupId, final String artifactId, final String version) throws MojoExecutionException {
+        final ArtifactHandler handler = artifactHandlerManager.getArtifactHandler("jar");
+
+        final VersionRange versionRange;
+        try {
+            versionRange = VersionRange.createFromVersionSpec(version);
+        } catch (final Exception e) {
+            throw new MojoExecutionException("Could not determine appropriate version for Provided Artifact " + groupId + ":" + artifactId, e);
+        }
+
+        final Artifact artifact = new DefaultArtifact(groupId, artifactId, versionRange, null, "jar", null, handler);
+
+        final ArtifactResolutionRequest request = new ArtifactResolutionRequest();
+        request.setLocalRepository(localRepo);
+        request.setRemoteRepositories(remoteRepos);
+        request.setArtifact(artifact);
+
+        final ArtifactResolutionResult result = artifactResolver.resolve(request);
+        if (!result.isSuccess()) {
+            final List<Exception> exceptions = result.getExceptions();
+
+            final MojoExecutionException exception = new MojoExecutionException("Could not resolve local dependency " + artifact);
+            if (exceptions != null) {
+                for (final Exception e : exceptions) {
+                    exception.addSuppressed(e);
+                }
+            }
+
+            throw exception;
+        }
+
+        final Set<Artifact> artifacts = result.getArtifacts();
+        if (artifacts.isEmpty()) {
+            throw new MojoExecutionException("Could not resolve any artifacts for dependency " + artifact);
+        }
+
+        final List<Artifact> sorted = new ArrayList<>(artifacts);
+        Collections.sort(sorted);
+
+        return sorted.get(0);
+    }
+
+    private ClassLoader createProvidedEntitiesClassLoader(final VersionLookup versionLookup) throws MojoExecutionException {
+        final String nifiApiVersion = versionLookup.getVersion("org.apache.nifi", "nifi-api");
+        if (nifiApiVersion == null) {
+            throw new MojoExecutionException("Could not find any dependency, provided or otherwise, on [org.apache.nifi:nifi-api]");
+        } else {
+            getLog().info("Found a dependency on version " + nifiApiVersion + " of NiFi API");
+        }
+
+        final String slf4jApiVersion = versionLookup.getVersion("org.slf4j", "slf4j-api");
+
+        final Artifact nifiApiArtifact = getProvidedArtifact("org.apache.nifi", "nifi-api", nifiApiVersion);
+        final Artifact nifiFrameworkApiArtifact = getProvidedArtifact("org.apache.nifi", "nifi-framework-api", nifiApiArtifact.getVersion());
+
+        final Artifact slf4jArtifact = getProvidedArtifact("org.slf4j", "slf4j-api", slf4jApiVersion);
+
+        final Set<Artifact> providedArtifacts = new HashSet<>();
+        providedArtifacts.add(nifiApiArtifact);
+        providedArtifacts.add(nifiFrameworkApiArtifact);
+        providedArtifacts.add(slf4jArtifact);
+
+        getLog().debug("Creating Provided Entities Class Loader with artifacts: " + providedArtifacts);
+        return createUrlClassLoader(providedArtifacts, null);
+    }
+
+    private ClassLoader createUrlClassLoader(final Set<Artifact> artifacts, final ClassLoader parent) throws MojoExecutionException {
+        final Set<URL> urls = new HashSet<>();
+        for (final Artifact artifact : artifacts) {
+            final Set<URL> artifactUrls = toURLs(artifact);
+            urls.addAll(artifactUrls);
+        }
+
+        getLog().debug("Creating class loader with following dependencies: " + urls);
+
+        final URL[] urlArray = urls.toArray(new URL[0]);
+        return new URLClassLoader(urlArray, parent);
+    }
+
+    private ExtensionClassLoader createClassLoader(final Set<Artifact> artifacts, final ClassLoader parent, final Artifact narArtifact) throws MojoExecutionException {
+        final Set<URL> urls = new HashSet<>();
+        for (final Artifact artifact : artifacts) {
+            final Set<URL> artifactUrls = toURLs(artifact);
+            urls.addAll(artifactUrls);
+        }
+
+        getLog().debug("Creating class loader with following dependencies: " + urls);
+
+        final URL[] urlArray = urls.toArray(new URL[0]);
+        if (parent == null) {
+            return new ExtensionClassLoader(urlArray, narArtifact, artifacts);
+        } else {
+            return new ExtensionClassLoader(urlArray, parent, narArtifact, artifacts);
+        }
+    }
+
+
+    private void gatherArtifacts(final MavenProject mavenProject, final Set<Artifact> artifacts) throws MojoExecutionException {
+        final DependencyNodeVisitor nodeVisitor = new DependencyNodeVisitor() {
+            @Override
+            public boolean visit(final DependencyNode dependencyNode) {
+                final Artifact artifact = dependencyNode.getArtifact();
+                artifacts.add(artifact);
+                return true;
+            }
+
+            @Override
+            public boolean endVisit(final DependencyNode dependencyNode) {
+                return true;
+            }
+        };
+
+        try {
+            final DependencyNode depNode = dependencyTreeBuilder.buildDependencyTree(mavenProject, localRepo, null);
+            depNode.accept(nodeVisitor);
+        } catch (DependencyTreeBuilderException e) {
+            throw new MojoExecutionException("Failed to build dependency tree", e);
+        }
+    }
+
+
+
+    private Set<URL> toURLs(final Artifact artifact) throws MojoExecutionException {
+        final Set<URL> urls = new HashSet<>();
+
+        final File artifactFile = artifact.getFile();
+        if (artifactFile == null) {
+            getLog().debug("Attempting to resolve Artifact " + artifact + " because it has no File associated with it");
+
+            final ArtifactResolutionRequest request = new ArtifactResolutionRequest();
+            request.setLocalRepository(localRepo);
+            request.setRemoteRepositories(remoteRepos);
+            request.setArtifact(artifact);
+
+            final ArtifactResolutionResult result = artifactResolver.resolve(request);
+            if (!result.isSuccess()) {
+                throw new MojoExecutionException("Could not resolve local dependency " + artifact);
+            }
+
+            getLog().info("Resolved Artifact " + artifact + " to " + result.getArtifacts());
+
+            for (final Artifact resolved : result.getArtifacts()) {
+                urls.addAll(toURLs(resolved));
+            }
+        } else {
+            try {
+                final URL url = artifact.getFile().toURI().toURL();
+                getLog().debug("Adding URL " + url + " to ClassLoader");
+                urls.add(url);
+            } catch (final MalformedURLException mue) {
+                throw new MojoExecutionException("Failed to convert File " + artifact.getFile() + " into URL", mue);
+            }
+        }
+
+        return urls;
+    }
+
+
+
+    public static class Builder {
+        private Log log;
+        private MavenProject project;
+        private ArtifactRepository localRepo;
+        private List<ArtifactRepository> remoteRepos;
+        private DependencyTreeBuilder dependencyTreeBuilder;
+        private ArtifactResolver artifactResolver;
+        private ProjectBuilder projectBuilder;
+        private RepositorySystemSession repositorySession;
+        private ArtifactHandlerManager artifactHandlerManager;
+
+        public Builder log(final Log log) {
+            this.log = log;
+            return this;
+        }
+
+        public Builder projectBuilder(final ProjectBuilder projectBuilder) {
+            this.projectBuilder = projectBuilder;
+            return this;
+        }
+
+        public Builder project(final MavenProject project) {
+            this.project = project;
+            return this;
+        }
+
+        public Builder localRepository(final ArtifactRepository localRepo) {
+            this.localRepo = localRepo;
+            return this;
+        }
+
+        public Builder remoteRepositories(final List<ArtifactRepository> remoteRepos) {
+            this.remoteRepos = remoteRepos;
+            return this;
+        }
+
+        public Builder dependencyTreeBuilder(final DependencyTreeBuilder dependencyTreeBuilder) {
+            this.dependencyTreeBuilder = dependencyTreeBuilder;
+            return this;
+        }
+
+        public Builder artifactResolver(final ArtifactResolver resolver) {
+            this.artifactResolver = resolver;
+            return this;
+        }
+
+        public Builder repositorySession(final RepositorySystemSession repositorySession) {
+            this.repositorySession = repositorySession;
+            return this;
+        }
+
+        public Builder artifactHandlerManager(final ArtifactHandlerManager artifactHandlerManager) {
+            this.artifactHandlerManager = artifactHandlerManager;
+            return this;
+        }
+
+        public ExtensionClassLoaderFactory build() {
+            return new ExtensionClassLoaderFactory(this);
+        }
+    }
+
+
+    private interface VersionLookup {
+        String getVersion(String groupId, String artifactId);
+    }
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionDefinitionFactory.java b/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionDefinitionFactory.java
new file mode 100644
index 0000000..d0aee68
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/ExtensionDefinitionFactory.java
@@ -0,0 +1,247 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.maven.artifact.Artifact;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.nifi.extension.definition.ExtensionDefinition;
+import org.apache.nifi.extension.definition.ExtensionType;
+import org.apache.nifi.extension.definition.Restriction;
+import org.apache.nifi.extension.definition.Restrictions;
+import org.apache.nifi.extension.definition.ServiceAPIDefinition;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class ExtensionDefinitionFactory {
+    private static final String SERVICES_DIRECTORY = "META-INF/services/";
+
+    private static final Map<ExtensionType, String> INTERFACE_NAMES = new HashMap<>();
+    static {
+        INTERFACE_NAMES.put(ExtensionType.PROCESSOR, "org.apache.nifi.processor.Processor");
+        INTERFACE_NAMES.put(ExtensionType.CONTROLLER_SERVICE, "org.apache.nifi.controller.ControllerService");
+        INTERFACE_NAMES.put(ExtensionType.REPORTING_TASK, "org.apache.nifi.reporting.ReportingTask");
+    }
+
+    private final ClassLoader extensionClassLoader;
+
+    public ExtensionDefinitionFactory(final ClassLoader classLoader) {
+        this.extensionClassLoader = classLoader;
+    }
+
+    public Set<ExtensionDefinition> discoverExtensions(final ExtensionType extensionType) throws IOException {
+        final String interfaceName = INTERFACE_NAMES.get(extensionType);
+        final Set<String> classNames = discoverClassNames(interfaceName);
+
+        if (classNames.isEmpty()) {
+            return Collections.emptySet();
+        }
+
+        final Set<ExtensionDefinition> definitions = new HashSet<>();
+        for (final String className : classNames) {
+            try {
+                definitions.add(createExtensionDefinition(extensionType, className));
+            } catch (final Exception e) {
+                throw new IOException("Failed to create Extension Definition for " + extensionType + " " + className, e);
+            }
+        }
+
+        return definitions;
+    }
+
+    private ExtensionDefinition createExtensionDefinition(final ExtensionType extensionType, final String className) throws ClassNotFoundException,
+                IllegalAccessException, NoSuchMethodException, InvocationTargetException {
+
+        final Class<?> extensionClass = Class.forName(className, false, extensionClassLoader);
+
+        final String capabilityDescription = getCapabilityDescription(extensionClass);
+        final Set<String> tags = getTags(extensionClass);
+        final Restrictions restrictions = getRestrictions(extensionClass);
+        final Set<ServiceAPIDefinition> serviceApis = getProvidedServiceAPIs(extensionType, extensionClass);
+
+        return new StandardExtensionDefinition(extensionType, className, capabilityDescription, tags, restrictions, serviceApis);
+    }
+
+    private Set<ServiceAPIDefinition> getProvidedServiceAPIs(final ExtensionType extensionType, final Class<?> extensionClass) throws ClassNotFoundException {
+        if (extensionType != ExtensionType.CONTROLLER_SERVICE) {
+            return Collections.emptySet();
+        }
+
+        final Set<ServiceAPIDefinition> serviceApis = new HashSet<>();
+        final Class<?> controllerServiceClass = Class.forName("org.apache.nifi.controller.ControllerService", false, extensionClassLoader);
+
+        for (final Class<?> implementedInterface : extensionClass.getInterfaces()) {
+            if (controllerServiceClass.isAssignableFrom(implementedInterface)) {
+                final ClassLoader interfaceClassLoader = implementedInterface.getClassLoader();
+                if (interfaceClassLoader instanceof ExtensionClassLoader) {
+                    final Artifact interfaceNarArtifact = ((ExtensionClassLoader) interfaceClassLoader).getNarArtifact();
+
+                    final ServiceAPIDefinition serviceDefinition = new StandardServiceAPIDefinition(implementedInterface.getName(),
+                        interfaceNarArtifact.getGroupId(), interfaceNarArtifact.getArtifactId(), interfaceNarArtifact.getVersion());
+
+                    serviceApis.add(serviceDefinition);
+                }
+            }
+        }
+
+        return serviceApis;
+    }
+
+    private Restrictions getRestrictions(final Class<?> extensionClass) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException {
+        final String restrictedDescription = getRestrictedDescription(extensionClass);
+        final Map<String, String> specificRestrictions = getSpecificRestrictions(extensionClass);
+
+        final boolean hasRestriction = restrictedDescription != null || !specificRestrictions.isEmpty();
+        final Restrictions restrictions;
+        if (!hasRestriction) {
+            return null;
+        }
+
+        final Set<Restriction> restrictionSet = new HashSet<>();
+        for (final Map.Entry<String, String> specificRestriction : specificRestrictions.entrySet()) {
+            restrictionSet.add(new StandardRestriction(specificRestriction.getKey(), specificRestriction.getValue()));
+        }
+
+        return new StandardRestrictions(restrictedDescription, restrictionSet);
+    }
+
+
+    private String getCapabilityDescription(final Class<?> extensionClass) throws InvocationTargetException,
+                IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
+
+        final Class capabilityDescriptionClass = Class.forName("org.apache.nifi.annotation.documentation.CapabilityDescription", false, extensionClass.getClassLoader());
+        final Method valueMethod = capabilityDescriptionClass.getMethod("value");
+
+        final Annotation capabilityDescriptionAnnotation = extensionClass.getAnnotation(capabilityDescriptionClass);
+        if (capabilityDescriptionAnnotation == null) {
+            return null;
+        }
+
+        final String capabilityDescriptionText = (String) valueMethod.invoke(capabilityDescriptionAnnotation);
+        return capabilityDescriptionText;
+    }
+
+
+    private Set<String> getTags(final Class<?> extensionClass) throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
+        final Class tagsClass = Class.forName("org.apache.nifi.annotation.documentation.Tags", false, extensionClass.getClassLoader());
+        final Method valueMethod = tagsClass.getMethod("value");
+
+        final Annotation tagsAnnotation = extensionClass.getAnnotation(tagsClass);
+        if (tagsAnnotation == null) {
+            return Collections.emptySet();
+        }
+
+        final String[] tags = (String[]) valueMethod.invoke(tagsAnnotation);
+        return Stream.of(tags).collect(Collectors.<String>toSet());
+    }
+
+
+    private Map<String, String> getSpecificRestrictions(final Class<?> extensionClass) throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
+        final Class restrictedClass = Class.forName("org.apache.nifi.annotation.behavior.Restricted", false, extensionClass.getClassLoader());
+        final Class restrictionClass = Class.forName("org.apache.nifi.annotation.behavior.Restriction", false, extensionClass.getClassLoader());
+        final Class requiredPermissionClass = Class.forName("org.apache.nifi.components.RequiredPermission", false, extensionClass.getClassLoader());
+
+        final Method restrictionsMethod = restrictedClass.getMethod("restrictions");
+        final Method explanationMethod = restrictionClass.getMethod("explanation");
+        final Method requiredPermissionMethod = restrictionClass.getMethod("requiredPermission");
+        final Method getPermissionIdentifierMethod = requiredPermissionClass.getMethod("getPermissionIdentifier");
+
+        final Annotation restrictionAnnotation = restrictedClass.getAnnotation(restrictedClass);
+        if (restrictionAnnotation == null) {
+            return edu.emory.mathcs.backport.java.util.Collections.emptyMap();
+        }
+
+        final Object[] restrictionsArray = (Object[]) restrictionsMethod.invoke(restrictionAnnotation);
+
+        final Map<String, String> restrictions = new HashMap<>();
+        for (final Object restriction : restrictionsArray) {
+            final String explanation = (String) explanationMethod.invoke(restriction);
+
+            final Object requiredPermission = requiredPermissionMethod.invoke(restriction);
+            final String requiredPermissionId = (String) getPermissionIdentifierMethod.invoke(requiredPermission);
+
+            restrictions.put(requiredPermissionId, explanation);
+        }
+
+        return restrictions;
+    }
+
+    private String getRestrictedDescription(final Class<?> extensionClass) throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
+        final Class restrictedClass = Class.forName("org.apache.nifi.annotation.behavior.Restricted", false, extensionClass.getClassLoader());
+        final Method valueMethod = restrictedClass.getMethod("value");
+
+        final Annotation restrictedAnnotation = extensionClass.getAnnotation(restrictedClass);
+        if (restrictedAnnotation == null) {
+            return null;
+        }
+
+        return (String) valueMethod.invoke(restrictedAnnotation);
+    }
+
+
+
+    private Set<String> discoverClassNames(final String extensionType) throws IOException {
+        final Set<String> classNames = new HashSet<>();
+
+        final Enumeration<URL> resources = extensionClassLoader.getResources(SERVICES_DIRECTORY + extensionType);
+
+        while (resources.hasMoreElements()) {
+            final URL resourceUrl = resources.nextElement();
+            classNames.addAll(discoverClassNames(extensionClassLoader, resourceUrl));
+        }
+
+        return classNames;
+    }
+
+    private Set<String> discoverClassNames(final ClassLoader classLoader, final URL serviceUrl) throws IOException {
+        final Set<String> classNames = new HashSet<>();
+
+        try (final InputStream in = serviceUrl.openStream();
+             final Reader rawReader = new InputStreamReader(in);
+             final BufferedReader reader = new BufferedReader(rawReader)) {
+
+            String line;
+            while ((line = reader.readLine()) != null) {
+                line = line.trim();
+
+                if (line.isEmpty() || line.startsWith("#")) {
+                    continue;
+                }
+
+                classNames.add(line);
+            }
+        }
+
+        return classNames;
+    }
+
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/StandardExtensionDefinition.java b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardExtensionDefinition.java
new file mode 100644
index 0000000..dadb747
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardExtensionDefinition.java
@@ -0,0 +1,78 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.nifi.extension.definition.ExtensionDefinition;
+import org.apache.nifi.extension.definition.ExtensionType;
+import org.apache.nifi.extension.definition.Restrictions;
+import org.apache.nifi.extension.definition.ServiceAPIDefinition;
+
+import java.util.Set;
+
+public class StandardExtensionDefinition implements ExtensionDefinition {
+    private final String capabilityDescription;
+    private final Set<String> tags;
+    private final Restrictions restrictions;
+    private final ExtensionType extensionType;
+    private final String extensionName;
+    private final Set<ServiceAPIDefinition> providedServiceApis;
+
+    public StandardExtensionDefinition(final ExtensionType extensionType, final String extensionName, final String capabilityDescription, final Set<String> tags, final Restrictions restrictions,
+                                       final Set<ServiceAPIDefinition> providedServiceApis) {
+        this.extensionType = extensionType;
+        this.extensionName = extensionName;
+        this.capabilityDescription = capabilityDescription;
+        this.tags = tags;
+        this.restrictions = restrictions;
+        this.providedServiceApis = providedServiceApis;
+    }
+
+    @Override
+    public String getCapabilityDescription() {
+        return capabilityDescription;
+    }
+
+    @Override
+    public Set<String> getTags() {
+        return tags;
+    }
+
+    @Override
+    public Restrictions getRestrictions() {
+        return restrictions;
+    }
+
+    @Override
+    public ExtensionType getExtensionType() {
+        return extensionType;
+    }
+
+    @Override
+    public Set<ServiceAPIDefinition> getProvidedServiceAPIs() {
+        return providedServiceApis;
+    }
+
+    @Override
+    public String getExtensionName() {
+        return extensionName;
+    }
+
+    @Override
+    public String toString() {
+        return "ExtensionDefinition[type=" + getExtensionType() + ", name=" + getExtensionName() + "]";
+    }
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/StandardRestriction.java b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardRestriction.java
new file mode 100644
index 0000000..6825d5b
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardRestriction.java
@@ -0,0 +1,39 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.nifi.extension.definition.Restriction;
+
+public class StandardRestriction implements Restriction {
+    private final String identifier;
+    private final String explanation;
+
+    public StandardRestriction(final String identifier, final String explanation) {
+        this.identifier = identifier;
+        this.explanation = explanation;
+    }
+
+    @Override
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    @Override
+    public String getExplanation() {
+        return explanation;
+    }
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/StandardRestrictions.java b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardRestrictions.java
new file mode 100644
index 0000000..97fa9ba
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardRestrictions.java
@@ -0,0 +1,42 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.nifi.extension.definition.Restriction;
+import org.apache.nifi.extension.definition.Restrictions;
+
+import java.util.Set;
+
+public class StandardRestrictions implements Restrictions {
+    private final String generalRestrictionExplanation;
+    private final Set<Restriction> restrictions;
+
+    public StandardRestrictions(final String generalRestrictionExplanation, final Set<Restriction> restrictions) {
+        this.generalRestrictionExplanation = generalRestrictionExplanation;
+        this.restrictions = restrictions;
+    }
+
+    @Override
+    public String getGeneralRestrictionExplanation() {
+        return generalRestrictionExplanation;
+    }
+
+    @Override
+    public Set<Restriction> getRestrictions() {
+        return restrictions;
+    }
+}
diff --git a/src/main/java/org/apache/nifi/extension/definition/extraction/StandardServiceAPIDefinition.java b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardServiceAPIDefinition.java
new file mode 100644
index 0000000..eb11a2d
--- /dev/null
+++ b/src/main/java/org/apache/nifi/extension/definition/extraction/StandardServiceAPIDefinition.java
@@ -0,0 +1,53 @@
+/*
+ * 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.extension.definition.extraction;
+
+import org.apache.nifi.extension.definition.ServiceAPIDefinition;
+
+public class StandardServiceAPIDefinition implements ServiceAPIDefinition {
+    private final String serviceAPIClassName;
+    private final String serviceGroupId;
+    private final String serviceArtifactId;
+    private final String serviceVersion;
+
+    public StandardServiceAPIDefinition(final String serviceAPIClassName, final String serviceGroupId, final String serviceArtifactId, final String serviceVersion) {
+        this.serviceAPIClassName = serviceAPIClassName;
+        this.serviceGroupId = serviceGroupId;
+        this.serviceArtifactId = serviceArtifactId;
+        this.serviceVersion = serviceVersion;
+    }
+
+    @Override
+    public String getServiceAPIClassName() {
+        return serviceAPIClassName;
+    }
+
+    @Override
+    public String getServiceGroupId() {
+        return serviceGroupId;
+    }
+
+    @Override
+    public String getServiceArtifactId() {
+        return serviceArtifactId;
+    }
+
+    @Override
+    public String getServiceVersion() {
+        return serviceVersion;
+    }
+}