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/03/13 19:34:12 UTC

[nifi-registry] branch master updated: NIFIREG-213 Implementing parser for extension-docs.xml file and integrating into the NarBundleExtractor

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-registry.git


The following commit(s) were added to refs/heads/master by this push:
     new e851401  NIFIREG-213 Implementing parser for extension-docs.xml file and integrating into the NarBundleExtractor
e851401 is described below

commit e851401bde16a03992a2457fe09cc6a54e4e665c
Author: Bryan Bende <bb...@apache.org>
AuthorDate: Wed Dec 12 15:43:02 2018 -0500

    NIFIREG-213 Implementing parser for extension-docs.xml file and integrating into the NarBundleExtractor
    
    - Incorporating BuildInfo into DB and appropriate entity/model classes
    - Added ProvidedServiceAPI to DB schema, entities, and metadata service
    - Added Restrictions to DB, entity classes, and metadata service
    - Refactored DataModelMapper into separate classes
    - Updated StandardExtensionService to persist Extensions to DB
    - Added link to extension-metadata from extension repo API
    - Updated NarBundleExtractor to extract additionalDetails and added additionalDetails to DB schema and entities
    - Added REST end-points to access extension metadata by bundle version
    - Refactored NarBundleExtractor to unmarshall extension-manifest.xml using Jackson XML
    - Refactored how extensions are stored in the DB to store a serialized version of the content
    - Removed extension prefix from bundle and bundle version class and tables
    - Added ExtensionResource to get all extensions
    - Creating links from ExtensionMetadata to Extension
    - Added swagger and validation annotations to extension model objects
    - Added filter params for retrieving extensions
    - Added ExtensionClient and JerseyExtensionClient
    - Added REST end-point to search by provided service API
    - Added display name to extensions
    - Added versioned serializer for the Extension entity
    - Added comment to default providers.xml about ordering of providers
    
    This closes #160.
    
    Signed-off-by: Kevin Doran <kd...@apache.org>
---
 .../nifi-registry-bundle-utils/pom.xml             |   11 +-
 .../registry/bundle/extract/BundleException.java}  |   10 +-
 .../extract}/BundleExtractor.java                  |    4 +-
 .../minificpp/MiNiFiCppBundleExtractor.java        |    6 +-
 .../bundle/extract/nar/NarBundleExtractor.java     |  234 ++
 .../extract}/nar/NarManifestEntry.java             |    2 +-
 .../extract/nar/docs/ExtensionManifestParser.java} |   14 +-
 .../nar/docs/JacksonExtensionManifestParser.java   |   50 +
 .../nifi/registry/bundle/model/BundleDetails.java  |  145 +
 .../model/BundleIdentifier.java}                   |   40 +-
 .../nifi/registry/bundle/util/BundleUtils.java}    |   27 +-
 .../nifi/registry/extension/BundleDetails.java     |   72 -
 .../registry/extension/nar/NarBundleExtractor.java |   77 -
 .../bundle/extract/nar/TestNarBundleExtractor.java |  154 +
 .../docs/TestJacksonExtensionManifestParser.java   |  161 +
 .../registry/extension/TestNarBundleExtractor.java |   93 -
 .../descriptors/extension-manifest-ambari-nar.xml  |   84 +
 .../extension-manifest-dbcp-service-nar.xml        |  325 ++
 .../descriptors/extension-manifest-hadoop-nar.xml  | 3274 ++++++++++++++++++++
 .../extension-manifest-missing-sys-api.xml         |    4 +
 .../extension-manifest-test-components.xml         |   56 +
 .../nifi-foo-nar-missing-extension-descriptor.nar  |  Bin 0 -> 574 bytes
 .../src/test/resources/nars/nifi-foo-nar.nar       |  Bin 428 -> 981 bytes
 .../src/test/resources/nars/nifi-framework-nar.nar |  Bin 577 -> 984 bytes
 .../src/test/resources/nars/nifi-hadoop-nar.nar    |  Bin 0 -> 17236 bytes
 ...xtensionBundleClient.java => BundleClient.java} |   16 +-
 ...VersionClient.java => BundleVersionClient.java} |   63 +-
 .../nifi/registry/client/ExtensionClient.java      |   62 +
 .../nifi/registry/client/ExtensionRepoClient.java  |   50 +
 .../nifi/registry/client/NiFiRegistryClient.java   |   18 +-
 .../client/impl/BucketItemDeserializer.java        |    6 +-
 .../nifi/registry/client/impl/ClientUtils.java     |   47 +
 ...onBundleClient.java => JerseyBundleClient.java} |   39 +-
 ...nClient.java => JerseyBundleVersionClient.java} |  119 +-
 .../client/impl/JerseyExtensionClient.java         |  111 +
 .../client/impl/JerseyExtensionRepoClient.java     |   77 +-
 .../client/impl/JerseyNiFiRegistryClient.java      |   32 +-
 .../nifi-registry-data-model/pom.xml               |    4 -
 .../nifi/registry/extension/bundle/BuildInfo.java  |  102 +
 .../{ExtensionBundle.java => bundle/Bundle.java}   |   16 +-
 .../BundleFilterParams.java}                       |   38 +-
 .../BundleInfo.java}                               |   81 +-
 .../BundleType.java}                               |   21 +-
 .../BundleTypeAdapter.java}                        |   10 +-
 .../extension/bundle/BundleTypeValues.java}        |   12 +-
 .../BundleVersion.java}                            |   33 +-
 .../BundleVersionDependency.java}                  |    6 +-
 .../BundleVersionFilterParams.java}                |   21 +-
 .../BundleVersionMetadata.java}                    |   44 +-
 .../extension/component/ExtensionFilterParams.java |  101 +
 .../extension/component/ExtensionMetadata.java     |  144 +
 .../component/ExtensionMetadataContainer.java      |   57 +
 .../registry/extension/component/TagCount.java     |   66 +
 .../component/manifest/AllowableValue.java         |   59 +
 .../extension/component/manifest/Attribute.java}   |   39 +-
 .../manifest/ControllerServiceDefinition.java}     |   55 +-
 .../component/manifest/DeprecationNotice.java      |   55 +
 .../component/manifest/DynamicProperty.java        |   79 +
 .../component/manifest/DynamicRelationship.java    |   50 +
 .../manifest/ExpressionLanguageScope.java}         |   22 +-
 .../extension/component/manifest/Extension.java    |  258 ++
 .../component/manifest/ExtensionManifest.java      |   64 +
 .../component/manifest/ExtensionType.java}         |    7 +-
 .../component/manifest/InputRequirement.java       |   48 +
 .../extension/component/manifest/Property.java     |  159 +
 .../manifest/ProvidedServiceAPI.java}              |   51 +-
 .../extension/component/manifest/Relationship.java |   59 +
 .../extension/component/manifest/Restricted.java   |   58 +
 .../extension/component/manifest/Restriction.java  |   67 +
 .../extension/component/manifest/Scope.java}       |   14 +-
 .../extension/component/manifest/Stateful.java     |   56 +
 .../manifest/SystemResourceConsideration.java      |   50 +
 .../extension/repo/ExtensionRepoArtifact.java      |    2 -
 .../repo/ExtensionRepoExtensionMetadata.java       |   51 +
 .../extension/repo/ExtensionRepoGroup.java         |    1 -
 .../extension/repo/ExtensionRepoVersion.java       |   14 +-
 .../repo/ExtensionRepoVersionSummary.java          |   27 +-
 .../org/apache/nifi/registry/flow/Position.java    |   26 +-
 .../registry/flow/VersionedRemoteProcessGroup.java |   13 +-
 .../nifi/registry/db/DatabaseKeyService.java       |    8 +-
 .../nifi/registry/db/DatabaseMetadataService.java  |  531 ++--
 ...xtensionBundleEntity.java => BundleEntity.java} |   14 +-
 ...ity.java => BundleVersionDependencyEntity.java} |    2 +-
 ...VersionEntity.java => BundleVersionEntity.java} |   91 +-
 .../nifi/registry/db/entity/ExtensionEntity.java   |  158 +-
 ...java => ExtensionProvidedServiceApiEntity.java} |   47 +-
 .../db/entity/ExtensionRestrictionEntity.java      |   78 +
 ...sionEntityCategory.java => TagCountEntity.java} |   21 +-
 .../db/mapper/BucketItemEntityRowMapper.java       |    8 +-
 ...tyRowMapper.java => BundleEntityRowMapper.java} |   15 +-
 ...=> BundleVersionDependencyEntityRowMapper.java} |   10 +-
 ...pper.java => BundleVersionEntityRowMapper.java} |   19 +-
 .../db/mapper/ExtensionEntityRowMapper.java        |   26 +-
 ...ityRowMapper.java => TagCountEntityMapper.java} |   11 +-
 .../apache/nifi/registry/event/EventFactory.java   |   16 +-
 .../nifi/registry/extension/ExtensionManager.java  |    2 +-
 .../nifi/registry/provider/ProviderFactory.java    |    6 +-
 .../registry/provider/StandardProviderFactory.java |   22 +-
 ...va => FileSystemBundlePersistenceProvider.java} |   38 +-
 ...ndleContext.java => StandardBundleContext.java} |   10 +-
 ...er.java => AbstractMultiVersionSerializer.java} |   63 +-
 .../serialization/ExtensionSerializer.java         |   54 +
 .../VersionedProcessGroupSerializer.java           |   94 +-
 .../jackson/JacksonExtensionSerializer.java}       |   20 +-
 .../jackson/ObjectMapperProvider.java              |   13 +-
 .../nifi/registry/service/DataModelMapper.java     |  293 --
 .../nifi/registry/service/MetadataService.java     |  100 +-
 .../nifi/registry/service/RegistryService.java     |  190 +-
 .../BundleMetadataExtractors.java}                 |   20 +-
 .../extension/ExtensionBundleCoordinate.java       |   57 -
 .../service/extension/ExtensionService.java        |  106 +-
 .../extension/StandardExtensionService.java        |  630 ++--
 .../registry/service/mapper/BucketMappings.java    |   49 +
 .../registry/service/mapper/ExtensionMappings.java |  304 ++
 .../nifi/registry/service/mapper/FlowMappings.java |  127 +
 .../KeyMappings.java}                              |   33 +-
 ...i.registry.extension.BundlePersistenceProvider} |    2 +-
 .../resources/db/migration/V3__AddExtensions.sql   |   68 +-
 .../registry/GenerateExtensionManifestSchema.java  |   53 +
 .../registry/db/TestDatabaseMetadataService.java   |  411 ++-
 .../nifi/registry/event/TestEventFactory.java      |   56 +-
 ...der.java => MockBundlePersistenceProvider.java} |   18 +-
 .../provider/TestStandardProviderFactory.java      |    6 +-
 ...> TestFileSystemBundlePersistenceProvider.java} |   92 +-
 .../serialization/TestExtensionSerializer.java     |   53 +
 .../TestVersionedProcessGroupSerializer.java       |    6 +-
 .../db/migration/V999999.1__test-setup.sql         |   58 +-
 .../provider/hook/bad-script-provider.xml          |    2 +-
 .../provider/providers-class-not-found.xml         |    2 +-
 .../src/test/resources/provider/providers-good.xml |    2 +-
 ...ensionBundleContext.java => BundleContext.java} |    4 +-
 ...eption.java => BundlePersistenceException.java} |    8 +-
 ...rovider.java => BundlePersistenceProvider.java} |   20 +-
 .../src/main/resources/conf/providers.xml          |    8 +-
 .../registry/web/NiFiRegistryResourceConfig.java   |   10 +-
 ...sionResource.java => BucketBundleResource.java} |   47 +-
 ...{ExtensionResource.java => BundleResource.java} |  245 +-
 ...oryResource.java => ExtensionRepoResource.java} |  164 +-
 .../nifi/registry/web/api/ExtensionResource.java   |  378 +--
 .../apache/nifi/registry/web/link/LinkService.java |   92 +-
 .../mapper/ConstraintViolationExceptionMapper.java |   11 +-
 ...nMapper.java => QueryParamExceptionMapper.java} |   34 +-
 .../web/api/UnsecuredNiFiRegistryClientIT.java     |  559 ++--
 .../nifi/registry/web/link/TestLinkService.java    |  114 +-
 .../src/test/resources/conf/providers.xml          |    2 +-
 .../extensions/nars/nifi-foo-nar-1.0.0.nar         |  Bin 571 -> 1092 bytes
 .../nars/nifi-foo-nar-2.0.0-SNAPSHOT-BUILD1.nar    |  Bin 781 -> 1302 bytes
 .../nars/nifi-foo-nar-2.0.0-SNAPSHOT-BUILD2.nar    |  Bin 781 -> 1302 bytes
 .../nars/nifi-foo-nar-2.0.0-SNAPSHOT-BUILD3.nar    |  Bin 781 -> 1302 bytes
 .../extensions/nars/nifi-test-nar-1.0.0.nar        |  Bin 568 -> 1334 bytes
 .../nars/nifi-test-nar-2.0.0-bad-manifest.nar      |  Bin 506 -> 1272 bytes
 .../nars/nifi-test-nar-2.0.0-diff-checksum.nar     |  Bin 735 -> 1501 bytes
 ...ifi-test-nar-2.0.0-missing-docs-descriptor.nar} |  Bin
 .../nars/nifi-test-nar-2.0.0-missing-manifest.nar  |  Bin 168 -> 934 bytes
 .../nars/nifi-test-nar-2.0.0-no-dependency.nar     |  Bin 545 -> 1311 bytes
 .../extensions/nars/nifi-test-nar-2.0.0.nar        |  Bin 571 -> 1337 bytes
 .../src/main/assembly/extension.xml                |    2 +-
 .../FlowPersistenceProviderMigrator.java           |    9 +-
 .../FlowPersistenceProviderMigratorTest.java       |    9 +-
 pom.xml                                            |   15 +-
 160 files changed, 10602 insertions(+), 2867 deletions(-)

diff --git a/nifi-registry-core/nifi-registry-bundle-utils/pom.xml b/nifi-registry-core/nifi-registry-bundle-utils/pom.xml
index 670331e..f0daf53 100644
--- a/nifi-registry-core/nifi-registry-bundle-utils/pom.xml
+++ b/nifi-registry-core/nifi-registry-bundle-utils/pom.xml
@@ -21,7 +21,16 @@
     <packaging>jar</packaging>
 
     <dependencies>
-
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-data-model</artifactId>
+            <version>0.4.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.dataformat</groupId>
+            <artifactId>jackson-dataformat-xml</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
     </dependencies>
 
 </project>
diff --git a/nifi-registry-core/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/extension/ExtensionBundlePersistenceException.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/BundleException.java
similarity index 70%
copy from nifi-registry-core/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/extension/ExtensionBundlePersistenceException.java
copy to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/BundleException.java
index 4722604..3d58906 100644
--- a/nifi-registry-core/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/extension/ExtensionBundlePersistenceException.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/BundleException.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.bundle.extract;
 
 /**
- * An Exception for errors encountered when a ExtensionBundlePersistenceProvider saves or retrieves a bundle.
+ * Exception to be thrown from a BundleExtractor when an issue occurs during extraction.
  */
-public class ExtensionBundlePersistenceException extends RuntimeException {
+public class BundleException extends RuntimeException {
 
-    public ExtensionBundlePersistenceException(String message) {
+    public BundleException(String message) {
         super(message);
     }
 
-    public ExtensionBundlePersistenceException(String message, Throwable cause) {
+    public BundleException(String message, Throwable cause) {
         super(message, cause);
     }
 
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleExtractor.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/BundleExtractor.java
similarity index 91%
rename from nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleExtractor.java
rename to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/BundleExtractor.java
index 771c632..4dc042b 100644
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleExtractor.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/BundleExtractor.java
@@ -14,7 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.bundle.extract;
+
+import org.apache.nifi.registry.bundle.model.BundleDetails;
 
 import java.io.IOException;
 import java.io.InputStream;
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/minificpp/MiNiFiCppBundleExtractor.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/minificpp/MiNiFiCppBundleExtractor.java
similarity index 86%
rename from nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/minificpp/MiNiFiCppBundleExtractor.java
rename to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/minificpp/MiNiFiCppBundleExtractor.java
index ba0eb68..d4bc22c 100644
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/minificpp/MiNiFiCppBundleExtractor.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/minificpp/MiNiFiCppBundleExtractor.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension.minificpp;
+package org.apache.nifi.registry.bundle.extract.minificpp;
 
-import org.apache.nifi.registry.extension.BundleDetails;
-import org.apache.nifi.registry.extension.BundleExtractor;
+import org.apache.nifi.registry.bundle.model.BundleDetails;
+import org.apache.nifi.registry.bundle.extract.BundleExtractor;
 
 import java.io.IOException;
 import java.io.InputStream;
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/NarBundleExtractor.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/NarBundleExtractor.java
new file mode 100644
index 0000000..35d11cd
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/NarBundleExtractor.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.bundle.extract.nar;
+
+import org.apache.nifi.registry.bundle.extract.BundleException;
+import org.apache.nifi.registry.bundle.extract.BundleExtractor;
+import org.apache.nifi.registry.bundle.extract.nar.docs.ExtensionManifestParser;
+import org.apache.nifi.registry.bundle.extract.nar.docs.JacksonExtensionManifestParser;
+import org.apache.nifi.registry.bundle.model.BundleIdentifier;
+import org.apache.nifi.registry.bundle.model.BundleDetails;
+import org.apache.nifi.registry.extension.bundle.BuildInfo;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionManifest;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.jar.Attributes;
+import java.util.jar.JarEntry;
+import java.util.jar.JarInputStream;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Implementation of ExtensionBundleExtractor for NAR bundles.
+ */
+public class NarBundleExtractor implements BundleExtractor {
+
+    /**
+     * The name of the JarEntry that contains the extension-docs.xml file.
+     */
+    private static String EXTENSION_DESCRIPTOR_ENTRY = "META-INF/docs/extension-manifest.xml";
+
+    /**
+     * The pattern of a JarEntry for additionalDetails.html entries.
+     */
+    private static Pattern ADDITIONAL_DETAILS_ENTRY_PATTERN =
+            Pattern.compile("META-INF\\/docs\\/additional-details\\/(.+)\\/additionalDetails.html");
+
+    /**
+     * The format of the date string in the NAR MANIFEST for Built-Timestamp.
+     */
+    private static String BUILT_TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss'Z'";
+
+    /**
+     * Used in place of any build info that is not present.
+     */
+    static String NA = "N/A";
+
+
+    @Override
+    public BundleDetails extract(final InputStream inputStream) throws IOException {
+        try (final JarInputStream jarInputStream = new JarInputStream(inputStream)) {
+            final Manifest manifest = jarInputStream.getManifest();
+            if (manifest == null) {
+                throw new BundleException("NAR bundles must contain a valid MANIFEST");
+            }
+
+            final Attributes attributes = manifest.getMainAttributes();
+            final BundleIdentifier bundleIdentifier = getBundleCoordinate(attributes);
+            final BundleIdentifier dependencyCoordinate = getDependencyBundleCoordinate(attributes);
+            final BuildInfo buildInfo = getBuildInfo(attributes);
+
+            final BundleDetails.Builder builder = new BundleDetails.Builder()
+                    .coordinate(bundleIdentifier)
+                    .addDependencyCoordinate(dependencyCoordinate)
+                    .buildInfo(buildInfo);
+
+            parseExtensionDocs(jarInputStream, builder);
+
+            return builder.build();
+        }
+    }
+
+    private BundleIdentifier getBundleCoordinate(final Attributes attributes) {
+        try {
+            final String groupId = attributes.getValue(NarManifestEntry.NAR_GROUP.getManifestName());
+            final String artifactId = attributes.getValue(NarManifestEntry.NAR_ID.getManifestName());
+            final String version = attributes.getValue(NarManifestEntry.NAR_VERSION.getManifestName());
+
+            return new BundleIdentifier(groupId, artifactId, version);
+        } catch (Exception e) {
+            throw new BundleException("Unable to obtain bundle coordinate due to: " + e.getMessage(), e);
+        }
+    }
+
+    private BundleIdentifier getDependencyBundleCoordinate(final Attributes attributes) {
+        try {
+            final String dependencyGroupId = attributes.getValue(NarManifestEntry.NAR_DEPENDENCY_GROUP.getManifestName());
+            final String dependencyArtifactId = attributes.getValue(NarManifestEntry.NAR_DEPENDENCY_ID.getManifestName());
+            final String dependencyVersion = attributes.getValue(NarManifestEntry.NAR_DEPENDENCY_VERSION.getManifestName());
+
+            final BundleIdentifier dependencyCoordinate;
+            if (dependencyArtifactId != null) {
+                dependencyCoordinate = new BundleIdentifier(dependencyGroupId, dependencyArtifactId, dependencyVersion);
+            } else {
+                dependencyCoordinate = null;
+            }
+            return dependencyCoordinate;
+        } catch (Exception e) {
+            throw new BundleException("Unable to obtain bundle coordinate for dependency due to: " + e.getMessage(), e);
+        }
+    }
+
+    private BuildInfo getBuildInfo(final Attributes attributes) {
+        final String buildBranch = attributes.getValue(NarManifestEntry.BUILD_BRANCH.getManifestName());
+        final String buildTag = attributes.getValue(NarManifestEntry.BUILD_TAG.getManifestName());
+        final String buildRevision = attributes.getValue(NarManifestEntry.BUILD_REVISION.getManifestName());
+        final String buildTimestamp = attributes.getValue(NarManifestEntry.BUILD_TIMESTAMP.getManifestName());
+        final String buildJdk = attributes.getValue(NarManifestEntry.BUILD_JDK.getManifestName());
+        final String builtBy = attributes.getValue(NarManifestEntry.BUILT_BY.getManifestName());
+
+        final SimpleDateFormat simpleDateFormat = new SimpleDateFormat(BUILT_TIMESTAMP_FORMAT);
+        try {
+            final Date buildDate = simpleDateFormat.parse(buildTimestamp);
+
+            final BuildInfo buildInfo = new BuildInfo();
+            buildInfo.setBuildTool(isBlank(buildJdk) ? NA : buildJdk);
+            buildInfo.setBuildBranch(isBlank(buildBranch) ? NA : buildBranch);
+            buildInfo.setBuildTag(isBlank(buildTag) ? NA : buildTag);
+            buildInfo.setBuildRevision(isBlank(buildRevision) ? NA : buildRevision);
+            buildInfo.setBuilt(buildDate.getTime());
+            buildInfo.setBuiltBy(isBlank(builtBy) ? NA : builtBy);
+            buildInfo.setBuildFlags(NA);
+            return buildInfo;
+
+        } catch (ParseException e) {
+            throw new BundleException("Unable to parse " + NarManifestEntry.BUILD_TIMESTAMP.getManifestName(), e);
+        } catch (Exception e) {
+            throw new BundleException("Unable to create build info for bundle due to: " + e.getMessage(), e);
+        }
+    }
+
+    public boolean isBlank(String value) {
+        return (value == null || value.trim().isEmpty());
+    }
+
+    private void parseExtensionDocs(final JarInputStream jarInputStream, final BundleDetails.Builder builder) throws IOException {
+        JarEntry jarEntry;
+        boolean foundExtensionDocs = false;
+        while((jarEntry = jarInputStream.getNextJarEntry()) != null) {
+            final String jarEntryName = jarEntry.getName();
+            if (EXTENSION_DESCRIPTOR_ENTRY.equals(jarEntryName)) {
+                try {
+                    final byte[] rawDocsContent = toByteArray(jarInputStream);
+                    final ExtensionManifestParser docsParser = new JacksonExtensionManifestParser();
+                    final InputStream inputStream = new NonCloseableInputStream(new ByteArrayInputStream(rawDocsContent));
+
+                    final ExtensionManifest extensionManifest = docsParser.parse(inputStream);
+                    builder.addExtensions(extensionManifest.getExtensions());
+                    builder.systemApiVersion(extensionManifest.getSystemApiVersion());
+
+                    foundExtensionDocs = true;
+                } catch (Exception e) {
+                    throw new BundleException("Unable to obtain extension info for bundle due to: " + e.getMessage(), e);
+                }
+            } else {
+                final Matcher matcher = ADDITIONAL_DETAILS_ENTRY_PATTERN.matcher(jarEntryName);
+                if (matcher.matches()) {
+                    final String extensionName = matcher.group(1);
+                    final String additionalDetailsContent = new String(toByteArray(jarInputStream), StandardCharsets.UTF_8);
+                    builder.addAdditionalDetails(extensionName, additionalDetailsContent);
+                }
+            }
+        }
+
+        if (!foundExtensionDocs) {
+            throw new BundleException("Unable to find descriptor at '" + EXTENSION_DESCRIPTOR_ENTRY + "'. " +
+                    "This NAR may need to be rebuilt with the latest version of the NiFi NAR Maven Plugin.");
+        }
+    }
+
+    private byte[] toByteArray(final InputStream input) throws IOException {
+        final ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+
+        int nRead;
+        byte[] data = new byte[16384];
+        while ((nRead = input.read(data, 0, data.length)) != -1) {
+            buffer.write(data, 0, nRead);
+        }
+
+        return buffer.toByteArray();
+    }
+
+    private static class NonCloseableInputStream extends FilterInputStream {
+
+        private final InputStream toWrap;
+
+        public NonCloseableInputStream(final InputStream toWrap) {
+            super(toWrap);
+            this.toWrap = toWrap;
+        }
+
+        @Override
+        public int read() throws IOException {
+            return toWrap.read();
+        }
+
+        @Override
+        public int read(byte[] b) throws IOException {
+            return toWrap.read(b);
+        }
+
+        @Override
+        public int read(byte[] b, int off, int len) throws IOException {
+            return toWrap.read(b, off, len);
+        }
+
+        @Override
+        public void close() throws IOException {
+            // do nothing
+        }
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/nar/NarManifestEntry.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/NarManifestEntry.java
similarity index 96%
rename from nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/nar/NarManifestEntry.java
rename to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/NarManifestEntry.java
index 0c75ed3..5650477 100644
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/nar/NarManifestEntry.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/NarManifestEntry.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension.nar;
+package org.apache.nifi.registry.bundle.extract.nar;
 
 /**
  * Enumeration of entries that will be in a NAR MANIFEST file.
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionBundleEntityType.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/docs/ExtensionManifestParser.java
similarity index 68%
copy from nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionBundleEntityType.java
copy to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/docs/ExtensionManifestParser.java
index 0f4950c..a4260a9 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionBundleEntityType.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/docs/ExtensionManifestParser.java
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.db.entity;
+package org.apache.nifi.registry.bundle.extract.nar.docs;
+
+import org.apache.nifi.registry.extension.component.manifest.ExtensionManifest;
+
+import java.io.InputStream;
 
 /**
- * The possible types of extension bundles.
+ * Parses an InputStream that is expected to contain the content of META-INF/docs/extensions-manifest.xml from a NAR.
  */
-public enum ExtensionBundleEntityType {
-
-    NIFI_NAR,
+public interface ExtensionManifestParser {
 
-    MINIFI_CPP;
+    ExtensionManifest parse(InputStream inputStream);
 
 }
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/docs/JacksonExtensionManifestParser.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/docs/JacksonExtensionManifestParser.java
new file mode 100644
index 0000000..a785bae
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/extract/nar/docs/JacksonExtensionManifestParser.java
@@ -0,0 +1,50 @@
+/*
+ * 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.registry.bundle.extract.nar.docs;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.xml.XmlMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationModule;
+import org.apache.nifi.registry.bundle.extract.BundleException;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionManifest;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Implementation of {@link ExtensionManifestParser} that uses Jackson XML to unmarshall the extension-manifest.xml content.
+ */
+public class JacksonExtensionManifestParser implements ExtensionManifestParser {
+
+    private final ObjectMapper mapper;
+
+    public JacksonExtensionManifestParser() {
+        this.mapper = new XmlMapper();
+        this.mapper.registerModule(new JaxbAnnotationModule());
+        this.mapper.configure(DeserializationFeature.ACCEPT_SINGLE_VALUE_AS_ARRAY, true);
+    }
+
+    @Override
+    public ExtensionManifest parse(InputStream inputStream) {
+        try {
+            return mapper.readValue(inputStream, ExtensionManifest.class);
+        } catch (IOException e) {
+            throw new BundleException("Unable to parse extension manifest due to: " + e.getMessage(), e);
+        }
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/model/BundleDetails.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/model/BundleDetails.java
new file mode 100644
index 0000000..7a083ac
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/model/BundleDetails.java
@@ -0,0 +1,145 @@
+/*
+ * 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.registry.bundle.model;
+
+
+import org.apache.nifi.registry.bundle.extract.BundleExtractor;
+import org.apache.nifi.registry.extension.bundle.BuildInfo;
+import org.apache.nifi.registry.extension.component.manifest.Extension;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.registry.bundle.util.BundleUtils.validateNotNull;
+
+/**
+ * Details for a given bundle which are obtained from a given {@link BundleExtractor}.
+ */
+public class BundleDetails {
+
+    private final BundleIdentifier bundleIdentifier;
+    private final Set<BundleIdentifier> dependencies;
+
+    private final String systemApiVersion;
+
+    private final Set<Extension> extensions;
+    private final Map<String,String> additionalDetails;
+
+    private final BuildInfo buildInfo;
+
+    private BundleDetails(final Builder builder) {
+        this.bundleIdentifier = builder.bundleIdentifier;
+        this.dependencies = Collections.unmodifiableSet(new HashSet<>(builder.dependencies));
+        this.extensions = Collections.unmodifiableSet(new HashSet<>(builder.extensions));
+        this.additionalDetails = Collections.unmodifiableMap(new HashMap<>(builder.additionalDetails));
+        this.systemApiVersion = builder.systemApiVersion;
+        this.buildInfo = builder.buildInfo;
+
+        validateNotNull("Bundle Coordinate", this.bundleIdentifier);
+        validateNotNull("Dependency Coordinates", this.dependencies);
+        validateNotNull("Extension Details", this.extensions);
+        validateNotNull("System API Version", this.systemApiVersion);
+        validateNotNull("Build Details", this.buildInfo);
+    }
+
+    public BundleIdentifier getBundleIdentifier() {
+        return bundleIdentifier;
+    }
+
+    public Set<BundleIdentifier> getDependencies() {
+        return dependencies;
+    }
+
+    public String getSystemApiVersion() {
+        return systemApiVersion;
+    }
+
+    public Set<Extension> getExtensions() {
+        return extensions;
+    }
+
+    public Map<String, String> getAdditionalDetails() {
+        return additionalDetails;
+    }
+
+    public BuildInfo getBuildInfo() {
+        return buildInfo;
+    }
+
+    /**
+     * Builder for creating instances of BundleDetails.
+     */
+    public static class Builder {
+
+        private BundleIdentifier bundleIdentifier;
+        private Set<BundleIdentifier> dependencies = new HashSet<>();
+        private Set<Extension> extensions = new HashSet<>();
+        private Map<String,String> additionalDetails = new HashMap<>();
+        private BuildInfo buildInfo;
+        private String systemApiVersion;
+
+        public Builder coordinate(final BundleIdentifier bundleIdentifier) {
+            this.bundleIdentifier = bundleIdentifier;
+            return this;
+        }
+
+        public Builder addDependencyCoordinate(final BundleIdentifier dependencyCoordinate) {
+            if (dependencyCoordinate != null) {
+                this.dependencies.add(dependencyCoordinate);
+            }
+            return this;
+        }
+
+        public Builder systemApiVersion(final String systemApiVersion) {
+            this.systemApiVersion = systemApiVersion;
+            return this;
+        }
+
+        public Builder addExtension(final Extension extension) {
+            if (extension != null) {
+                this.extensions.add(extension);
+            }
+            return this;
+        }
+
+        public Builder addExtensions(final List<Extension> extensions) {
+            if (extensions != null) {
+                this.extensions.addAll(extensions);
+            }
+            return this;
+        }
+
+        public Builder addAdditionalDetails(final String extensionName, final String additionalDetails) {
+            this.additionalDetails.put(extensionName, additionalDetails);
+            return this;
+        }
+
+        public Builder buildInfo(final BuildInfo buildInfo) {
+            this.buildInfo = buildInfo;
+            return this;
+        }
+
+        public BundleDetails build() {
+            return new BundleDetails(this);
+        }
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleCoordinate.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/model/BundleIdentifier.java
similarity index 62%
rename from nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleCoordinate.java
rename to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/model/BundleIdentifier.java
index 96ce5ea..21d8ca6 100644
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleCoordinate.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/model/BundleIdentifier.java
@@ -14,34 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.bundle.model;
+
+import static org.apache.nifi.registry.bundle.util.BundleUtils.validateNotBlank;
 
 /**
- * The coordinate of an extension bundle (i.e group + artifact + version).
+ * The identifier of an extension bundle (i.e group + artifact + version).
  */
-public class BundleCoordinate {
+public class BundleIdentifier {
 
     private final String groupId;
     private final String artifactId;
     private final String version;
 
-    private final String coordinate;
-
+    private final String identifier;
 
-    public BundleCoordinate(final String groupId, final String artifactId, final String version) {
+    public BundleIdentifier(final String groupId, final String artifactId, final String version) {
         this.groupId = groupId;
         this.artifactId = artifactId;
         this.version = version;
+        validateNotBlank("Group Id", this.groupId);
+        validateNotBlank("Artifact Id", this.artifactId);
+        validateNotBlank("Version", this.version);
 
-        if (isBlank(this.groupId) || isBlank(this.artifactId) || isBlank(this.version)) {
-            throw new IllegalStateException("Group, Id, and Version are required for BundleCoordinate");
-        }
-
-        this.coordinate = this.groupId + ":" + this.artifactId + ":" + this.version;
-    }
-
-    private boolean isBlank(String str) {
-        return str == null || str.trim().length() == 0;
+        this.identifier = this.groupId + ":" + this.artifactId + ":" + this.version;
     }
 
     public String getGroupId() {
@@ -56,13 +52,13 @@ public class BundleCoordinate {
         return version;
     }
 
-    public final String getCoordinate() {
-        return coordinate;
+    public final String getIdentifier() {
+        return identifier;
     }
 
     @Override
     public String toString() {
-        return coordinate;
+        return identifier;
     }
 
     @Override
@@ -71,17 +67,17 @@ public class BundleCoordinate {
             return false;
         }
 
-        if (!(obj instanceof BundleCoordinate)) {
+        if (!(obj instanceof BundleIdentifier)) {
             return false;
         }
 
-        final BundleCoordinate other = (BundleCoordinate) obj;
-        return getCoordinate().equals(other.getCoordinate());
+        final BundleIdentifier other = (BundleIdentifier) obj;
+        return getIdentifier().equals(other.getIdentifier());
     }
 
     @Override
     public int hashCode() {
-        return 37 * this.coordinate.hashCode();
+        return 37 * this.identifier.hashCode();
     }
 
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleTypeAdapter.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/util/BundleUtils.java
similarity index 60%
copy from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleTypeAdapter.java
copy to nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/util/BundleUtils.java
index 1a993cf..2640b23 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleTypeAdapter.java
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/bundle/util/BundleUtils.java
@@ -14,27 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.bundle.util;
 
-import javax.xml.bind.annotation.adapters.XmlAdapter;
+public class BundleUtils {
 
-public class ExtensionBundleTypeAdapter extends XmlAdapter<String,ExtensionBundleType> {
+    public static boolean isBlank(final String value) {
+        return (value == null || value.trim().isEmpty());
+    }
 
-    @Override
-    public ExtensionBundleType unmarshal(String v) throws Exception {
-        if (v == null) {
-            return null;
+    public static void validateNotNull(String fieldName, Object value) {
+        if (value == null) {
+            throw new IllegalArgumentException(fieldName + " is required");
         }
-
-        return ExtensionBundleType.fromString(v);
     }
 
-    @Override
-    public String marshal(final ExtensionBundleType v) throws Exception {
-        if (v == null) {
-            return null;
+    public static void validateNotBlank(String fieldName, String value) {
+        if (isBlank(value)) {
+            throw new IllegalArgumentException(fieldName + " is required");
         }
-
-        return v.toString();
     }
+
 }
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleDetails.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleDetails.java
deleted file mode 100644
index 88ec469..0000000
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/BundleDetails.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.extension;
-
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-public class BundleDetails {
-
-    private final BundleCoordinate bundleCoordinate;
-
-    // Can be null when there is no dependent bundle
-    private final Set<BundleCoordinate> dependencyBundleCoordinates;
-
-    private BundleDetails(final Builder builder) {
-        this.bundleCoordinate = builder.bundleCoordinate;
-        this.dependencyBundleCoordinates = Collections.unmodifiableSet(new HashSet<>(builder.dependencyBundleCoordinates));
-        if (this.bundleCoordinate == null) {
-            throw new IllegalStateException("A bundle coordinate is required");
-        }
-    }
-
-    public BundleCoordinate getBundleCoordinate() {
-        return bundleCoordinate;
-    }
-
-    public Set<BundleCoordinate> getDependencyBundleCoordinates() {
-        return dependencyBundleCoordinates;
-    }
-
-    /**
-     * Builder for creating instances of BundleDetails.
-     */
-    public static class Builder {
-
-        private BundleCoordinate bundleCoordinate;
-        private Set<BundleCoordinate> dependencyBundleCoordinates = new HashSet<>();
-
-        public Builder coordinate(final BundleCoordinate bundleCoordinate) {
-            this.bundleCoordinate = bundleCoordinate;
-            return this;
-        }
-
-        public Builder dependencyCoordinate(final BundleCoordinate dependencyCoordinate) {
-            if (dependencyCoordinate != null) {
-                this.dependencyBundleCoordinates.add(dependencyCoordinate);
-            }
-            return this;
-        }
-
-        public BundleDetails build() {
-            return new BundleDetails(this);
-        }
-    }
-
-}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/nar/NarBundleExtractor.java b/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/nar/NarBundleExtractor.java
deleted file mode 100644
index d9cfe71..0000000
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/main/java/org/apache/nifi/registry/extension/nar/NarBundleExtractor.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.extension.nar;
-
-import org.apache.nifi.registry.extension.BundleCoordinate;
-import org.apache.nifi.registry.extension.BundleDetails;
-import org.apache.nifi.registry.extension.BundleExtractor;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.jar.Attributes;
-import java.util.jar.JarInputStream;
-import java.util.jar.Manifest;
-
-/**
- * Implementation of ExtensionBundleExtractor for NAR bundles.
- */
-public class NarBundleExtractor implements BundleExtractor {
-
-    @Override
-    public BundleDetails extract(final InputStream inputStream) throws IOException {
-        try (final JarInputStream jarInputStream = new JarInputStream(inputStream)) {
-            final Manifest manifest = jarInputStream.getManifest();
-            if (manifest == null) {
-                throw new IllegalArgumentException("NAR bundles must contain a valid MANIFEST");
-            }
-
-            final Attributes attributes = manifest.getMainAttributes();
-
-            final String groupId = attributes.getValue(NarManifestEntry.NAR_GROUP.getManifestName());
-            final String artifactId = attributes.getValue(NarManifestEntry.NAR_ID.getManifestName());
-            final String version = attributes.getValue(NarManifestEntry.NAR_VERSION.getManifestName());
-
-            final BundleCoordinate bundleCoordinate = new BundleCoordinate(groupId, artifactId, version);
-
-            final String dependencyGroupId = attributes.getValue(NarManifestEntry.NAR_DEPENDENCY_GROUP.getManifestName());
-            final String dependencyArtifactId = attributes.getValue(NarManifestEntry.NAR_DEPENDENCY_ID.getManifestName());
-            final String dependencyVersion = attributes.getValue(NarManifestEntry.NAR_DEPENDENCY_VERSION.getManifestName());
-
-            final BundleCoordinate dependencyCoordinate;
-            if (dependencyArtifactId != null) {
-                dependencyCoordinate = new BundleCoordinate(dependencyGroupId, dependencyArtifactId, dependencyVersion);
-            } else {
-                dependencyCoordinate = null;
-            }
-
-            // TODO figure out what to do with build info
-            final String buildBranch = attributes.getValue(NarManifestEntry.BUILD_BRANCH.getManifestName());
-            final String buildTag = attributes.getValue(NarManifestEntry.BUILD_TAG.getManifestName());
-            final String buildRevision = attributes.getValue(NarManifestEntry.BUILD_REVISION.getManifestName());
-            final String buildTimestamp = attributes.getValue(NarManifestEntry.BUILD_TIMESTAMP.getManifestName());
-            final String buildJdk = attributes.getValue(NarManifestEntry.BUILD_JDK.getManifestName());
-            final String builtBy = attributes.getValue(NarManifestEntry.BUILT_BY.getManifestName());
-
-            final BundleDetails.Builder builder = new BundleDetails.Builder()
-                    .coordinate(bundleCoordinate)
-                    .dependencyCoordinate(dependencyCoordinate);
-
-            return builder.build();
-        }
-    }
-
-}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/bundle/extract/nar/TestNarBundleExtractor.java b/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/bundle/extract/nar/TestNarBundleExtractor.java
new file mode 100644
index 0000000..afa8a1a
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/bundle/extract/nar/TestNarBundleExtractor.java
@@ -0,0 +1,154 @@
+/*
+ * 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.registry.bundle.extract.nar;
+
+import org.apache.nifi.registry.bundle.extract.BundleException;
+import org.apache.nifi.registry.bundle.extract.BundleExtractor;
+import org.apache.nifi.registry.bundle.model.BundleIdentifier;
+import org.apache.nifi.registry.bundle.model.BundleDetails;
+import org.apache.nifi.registry.extension.bundle.BuildInfo;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestNarBundleExtractor {
+
+    private BundleExtractor extractor;
+
+    @Before
+    public void setup() {
+        this.extractor = new NarBundleExtractor();
+    }
+
+    @Test
+    public void testExtractFromGoodNarNoDependencies() throws IOException {
+        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-framework-nar.nar")) {
+            final BundleDetails bundleDetails = extractor.extract(in);
+            assertNotNull(bundleDetails);
+            assertNotNull(bundleDetails.getBundleIdentifier());
+            assertNotNull(bundleDetails.getDependencies());
+            assertEquals(0, bundleDetails.getDependencies().size());
+
+            final BundleIdentifier bundleIdentifier = bundleDetails.getBundleIdentifier();
+            assertEquals("org.apache.nifi", bundleIdentifier.getGroupId());
+            assertEquals("nifi-framework-nar", bundleIdentifier.getArtifactId());
+            assertEquals("1.8.0", bundleIdentifier.getVersion());
+
+            assertNotNull(bundleDetails.getExtensions());
+            assertEquals(0, bundleDetails.getExtensions().size());
+            assertEquals("1.8.0", bundleDetails.getSystemApiVersion());
+        }
+    }
+
+    @Test
+    public void testExtractFromGoodNarWithDependencies() throws IOException {
+        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-foo-nar.nar")) {
+            final BundleDetails bundleDetails = extractor.extract(in);
+            assertNotNull(bundleDetails);
+            assertNotNull(bundleDetails.getBundleIdentifier());
+            assertNotNull(bundleDetails.getDependencies());
+            assertEquals(1, bundleDetails.getDependencies().size());
+
+            final BundleIdentifier bundleIdentifier = bundleDetails.getBundleIdentifier();
+            assertEquals("org.apache.nifi", bundleIdentifier.getGroupId());
+            assertEquals("nifi-foo-nar", bundleIdentifier.getArtifactId());
+            assertEquals("1.8.0", bundleIdentifier.getVersion());
+
+            final BundleIdentifier dependencyCoordinate = bundleDetails.getDependencies().stream().findFirst().get();
+            assertEquals("org.apache.nifi", dependencyCoordinate.getGroupId());
+            assertEquals("nifi-bar-nar", dependencyCoordinate.getArtifactId());
+            assertEquals("2.0.0", dependencyCoordinate.getVersion());
+
+            final Map<String,String> additionalDetails = bundleDetails.getAdditionalDetails();
+            assertNotNull(additionalDetails);
+            assertEquals(0, additionalDetails.size());
+        }
+    }
+
+    @Test(expected = BundleException.class)
+    public void testExtractFromNarMissingRequiredManifestEntries() throws IOException {
+        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-missing-manifest-entries.nar")) {
+            extractor.extract(in);
+            fail("Should have thrown exception");
+        }
+    }
+
+    @Test(expected = BundleException.class)
+    public void testExtractFromNarMissingManifest() throws IOException {
+        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-missing-manifest.nar")) {
+            extractor.extract(in);
+            fail("Should have thrown exception");
+        }
+    }
+
+    @Test(expected = BundleException.class)
+    public void testExtractFromNarMissingExtensionDescriptor() throws IOException {
+        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-foo-nar-missing-extension-descriptor.nar")) {
+            extractor.extract(in);
+            fail("Should have thrown exception");
+        }
+    }
+
+    @Test
+    public void testExtractFromNarWithDescriptorAndAdditionalDetails() throws IOException {
+        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-hadoop-nar.nar")) {
+            final BundleDetails bundleDetails = extractor.extract(in);
+            assertNotNull(bundleDetails);
+            assertNotNull(bundleDetails.getBundleIdentifier());
+            assertNotNull(bundleDetails.getDependencies());
+            assertEquals(1, bundleDetails.getDependencies().size());
+
+            final BundleIdentifier bundleIdentifier = bundleDetails.getBundleIdentifier();
+            assertEquals("org.apache.nifi", bundleIdentifier.getGroupId());
+            assertEquals("nifi-hadoop-nar", bundleIdentifier.getArtifactId());
+            assertEquals("1.9.0-SNAPSHOT", bundleIdentifier.getVersion());
+
+            final BuildInfo buildDetails = bundleDetails.getBuildInfo();
+            assertNotNull(buildDetails);
+            assertEquals("1.8.0_162", buildDetails.getBuildTool());
+            assertEquals(NarBundleExtractor.NA, buildDetails.getBuildFlags());
+            assertEquals("master", buildDetails.getBuildBranch());
+            assertEquals("HEAD", buildDetails.getBuildTag());
+            assertEquals("1a937b6", buildDetails.getBuildRevision());
+            assertEquals("jsmith", buildDetails.getBuiltBy());
+            assertNotNull(buildDetails.getBuilt());
+
+            assertEquals("1.10.0-SNAPSHOT", bundleDetails.getSystemApiVersion());
+            assertNotNull(bundleDetails.getExtensions());
+            assertEquals(10, bundleDetails.getExtensions().size());
+
+            final Map<String,String> additionalDetails = bundleDetails.getAdditionalDetails();
+            assertNotNull(additionalDetails);
+            assertEquals(3, additionalDetails.size());
+
+            final String listHdfsKey = "org.apache.nifi.processors.hadoop.ListHDFS";
+            assertTrue(additionalDetails.containsKey(listHdfsKey));
+            assertTrue(additionalDetails.get(listHdfsKey).startsWith("<!DOCTYPE html>"));
+            assertTrue(additionalDetails.get(listHdfsKey).trim().endsWith("</html>"));
+        }
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/bundle/extract/nar/docs/TestJacksonExtensionManifestParser.java b/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/bundle/extract/nar/docs/TestJacksonExtensionManifestParser.java
new file mode 100644
index 0000000..6f57ea9
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/bundle/extract/nar/docs/TestJacksonExtensionManifestParser.java
@@ -0,0 +1,161 @@
+/*
+ * 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.registry.bundle.extract.nar.docs;
+
+import org.apache.nifi.registry.extension.component.manifest.Extension;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionManifest;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionType;
+import org.apache.nifi.registry.extension.component.manifest.ProvidedServiceAPI;
+import org.apache.nifi.registry.extension.component.manifest.Restriction;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestJacksonExtensionManifestParser {
+
+    private ExtensionManifestParser parser;
+
+    @Before
+    public void setup() {
+        parser = new JacksonExtensionManifestParser();
+    }
+
+    @Test
+    public void testDocsWithProcessors() throws IOException {
+        final ExtensionManifest extensionManifest = parse("src/test/resources/descriptors/extension-manifest-hadoop-nar.xml");
+        assertNotNull(extensionManifest);
+        assertEquals("1.10.0-SNAPSHOT", extensionManifest.getSystemApiVersion());
+
+        final List<Extension> extensionDetails = extensionManifest.getExtensions();
+        assertEquals(10, extensionDetails.size());
+
+        final Extension putHdfsExtension = extensionDetails.stream()
+                .filter(e -> e.getName().equals("org.apache.nifi.processors.hadoop.PutHDFS"))
+                .findFirst()
+                .orElse(null);
+
+        assertNotNull(putHdfsExtension);
+        assertEquals(ExtensionType.PROCESSOR, putHdfsExtension.getType());
+        assertEquals("Write FlowFile data to Hadoop Distributed File System (HDFS)", putHdfsExtension.getDescription());
+        assertEquals(5, putHdfsExtension.getTags().size());
+        assertTrue(putHdfsExtension.getTags().contains("hadoop"));
+        assertTrue(putHdfsExtension.getTags().contains("HDFS"));
+        assertTrue(putHdfsExtension.getTags().contains("put"));
+        assertTrue(putHdfsExtension.getTags().contains("copy"));
+        assertTrue(putHdfsExtension.getTags().contains("filesystem"));
+        assertNull(putHdfsExtension.getProvidedServiceAPIs());
+
+        assertNotNull(putHdfsExtension.getProperties());
+        assertEquals(15, putHdfsExtension.getProperties().size());
+
+        assertNull(putHdfsExtension.getRestricted().getGeneralRestrictionExplanation());
+
+        final List<Restriction> restrictions = putHdfsExtension.getRestricted().getRestrictions();
+        assertNotNull(restrictions);
+        assertEquals(1, restrictions.size());
+
+        final Restriction restriction = restrictions.stream().findFirst().orElse(null);
+        assertEquals("write filesystem", restriction.getRequiredPermission());
+        assertEquals("Provides operator the ability to delete any file that NiFi has access to in HDFS or\n" +
+                "                            the local filesystem.", restriction.getExplanation().trim());
+    }
+
+    @Test
+    public void testDocsWithControllerService() throws IOException {
+        final ExtensionManifest extensionManifest = parse("src/test/resources/descriptors/extension-manifest-dbcp-service-nar.xml");
+        assertNotNull(extensionManifest);
+        assertEquals("1.10.0-SNAPSHOT", extensionManifest.getSystemApiVersion());
+
+        final List<Extension> extensions = extensionManifest.getExtensions();
+        assertEquals(2, extensions.size());
+
+        final Extension dbcpPoolExtension = extensions.stream()
+                .filter(e -> e.getName().equals("org.apache.nifi.dbcp.DBCPConnectionPool"))
+                .findFirst()
+                .orElse(null);
+
+        assertNotNull(dbcpPoolExtension);
+        assertEquals(ExtensionType.CONTROLLER_SERVICE, dbcpPoolExtension.getType());
+        assertEquals("Provides Database Connection Pooling Service. Connections can be asked from pool and returned\n" +
+                "                after usage.", dbcpPoolExtension.getDescription().trim());
+        assertEquals(6, dbcpPoolExtension.getTags().size());
+        assertEquals(1, dbcpPoolExtension.getProvidedServiceAPIs().size());
+
+        final ProvidedServiceAPI providedServiceApi = dbcpPoolExtension.getProvidedServiceAPIs().iterator().next();
+        assertNotNull(providedServiceApi);
+        assertEquals("org.apache.nifi.dbcp.DBCPService", providedServiceApi.getClassName());
+        assertEquals("org.apache.nifi", providedServiceApi.getGroupId());
+        assertEquals("nifi-standard-services-api-nar", providedServiceApi.getArtifactId());
+        assertEquals("1.10.0-SNAPSHOT", providedServiceApi.getVersion());
+    }
+
+    @Test
+    public void testDocsWithReportingTask() throws IOException {
+        final ExtensionManifest extensionManifest = parse("src/test/resources/descriptors/extension-manifest-ambari-nar.xml");
+        assertNotNull(extensionManifest);
+        assertEquals("1.10.0-SNAPSHOT", extensionManifest.getSystemApiVersion());
+
+        final List<Extension> extensions = extensionManifest.getExtensions();
+        assertEquals(1, extensions.size());
+
+        final Extension reportingTask = extensions.stream()
+                .filter(e -> e.getName().equals("org.apache.nifi.reporting.ambari.AmbariReportingTask"))
+                .findFirst()
+                .orElse(null);
+
+        assertNotNull(reportingTask);
+        assertEquals(ExtensionType.REPORTING_TASK, reportingTask.getType());
+        assertNotNull(reportingTask.getDescription());
+        assertEquals(3, reportingTask.getTags().size());
+        assertTrue(reportingTask.getTags().contains("reporting"));
+        assertTrue(reportingTask.getTags().contains("metrics"));
+        assertTrue(reportingTask.getTags().contains("ambari"));
+        assertNull(reportingTask.getProvidedServiceAPIs());
+    }
+
+    @Test
+    public void testDocsForTestComponents() throws IOException {
+        final ExtensionManifest extensionManifest = parse("src/test/resources/descriptors/extension-manifest-test-components.xml");
+        assertNotNull(extensionManifest);
+        assertEquals("1.8.0", extensionManifest.getSystemApiVersion());
+
+        final List<Extension> extensionDetails = extensionManifest.getExtensions();
+        assertEquals(4, extensionDetails.size());
+
+    }
+
+    @Test
+    public void testDocsForMissingSystemApi() throws IOException {
+        final ExtensionManifest extensionManifest = parse("src/test/resources/descriptors/extension-manifest-missing-sys-api.xml");
+        assertNotNull(extensionManifest);
+    }
+
+    private ExtensionManifest parse(final String file) throws IOException {
+        try (final InputStream inputStream = new FileInputStream(file)) {
+            return parser.parse(inputStream);
+        }
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/extension/TestNarBundleExtractor.java b/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/extension/TestNarBundleExtractor.java
deleted file mode 100644
index 306f1d1..0000000
--- a/nifi-registry-core/nifi-registry-bundle-utils/src/test/java/org/apache/nifi/registry/extension/TestNarBundleExtractor.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.extension;
-
-import org.apache.nifi.registry.extension.nar.NarBundleExtractor;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-
-public class TestNarBundleExtractor {
-
-    private BundleExtractor extractor;
-
-    @Before
-    public void setup() {
-        this.extractor = new NarBundleExtractor();
-    }
-
-    @Test
-    public void testExtractFromGoodNarNoDependencies() throws IOException {
-        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-framework-nar.nar")) {
-            final BundleDetails bundleDetails = extractor.extract(in);
-            assertNotNull(bundleDetails);
-            assertNotNull(bundleDetails.getBundleCoordinate());
-            assertNotNull(bundleDetails.getDependencyBundleCoordinates());
-            assertEquals(0, bundleDetails.getDependencyBundleCoordinates().size());
-
-            final BundleCoordinate bundleCoordinate = bundleDetails.getBundleCoordinate();
-            assertEquals("org.apache.nifi", bundleCoordinate.getGroupId());
-            assertEquals("nifi-framework-nar", bundleCoordinate.getArtifactId());
-            assertEquals("1.8.0", bundleCoordinate.getVersion());
-        }
-    }
-
-    @Test
-    public void testExtractFromGoodNarWithDependencies() throws IOException {
-        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-foo-nar.nar")) {
-            final BundleDetails bundleDetails = extractor.extract(in);
-            assertNotNull(bundleDetails);
-            assertNotNull(bundleDetails.getBundleCoordinate());
-            assertNotNull(bundleDetails.getDependencyBundleCoordinates());
-            assertEquals(1, bundleDetails.getDependencyBundleCoordinates().size());
-
-            final BundleCoordinate bundleCoordinate = bundleDetails.getBundleCoordinate();
-            assertEquals("org.apache.nifi", bundleCoordinate.getGroupId());
-            assertEquals("nifi-foo-nar", bundleCoordinate.getArtifactId());
-            assertEquals("1.8.0", bundleCoordinate.getVersion());
-
-            final BundleCoordinate dependencyCoordinate = bundleDetails.getDependencyBundleCoordinates().stream().findFirst().get();
-            assertEquals("org.apache.nifi", dependencyCoordinate.getGroupId());
-            assertEquals("nifi-bar-nar", dependencyCoordinate.getArtifactId());
-            assertEquals("2.0.0", dependencyCoordinate.getVersion());
-        }
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void testExtractFromNarMissingRequiredManifestEntries() throws IOException {
-        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-missing-manifest-entries.nar")) {
-            extractor.extract(in);
-            fail("Should have thrown exception");
-        }
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testExtractFromNarMissingManifest() throws IOException {
-        try (final InputStream in = new FileInputStream("src/test/resources/nars/nifi-missing-manifest.nar")) {
-            extractor.extract(in);
-            fail("Should have thrown exception");
-        }
-    }
-
-}
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-ambari-nar.xml b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-ambari-nar.xml
new file mode 100644
index 0000000..4cfe3ea
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-ambari-nar.xml
@@ -0,0 +1,84 @@
+<extensionManifest>
+    <systemApiVersion>1.10.0-SNAPSHOT</systemApiVersion>
+    <extensions>
+        <extension>
+            <name>org.apache.nifi.reporting.ambari.AmbariReportingTask</name>
+            <type>REPORTING_TASK</type>
+            <deprecationNotice/>
+            <description>Publishes metrics from NiFi to Ambari Metrics Service (AMS). Due to how the Ambari Metrics
+                Service works, this reporting task should be scheduled to run every 60 seconds. Each iteration it will
+                send the metrics from the previous iteration, and calculate the current metrics to be sent on next
+                iteration. Scheduling this reporting task at a frequency other than 60 seconds may produce unexpected
+                results.
+            </description>
+            <tags>
+                <tag>reporting</tag>
+                <tag>ambari</tag>
+                <tag>metrics</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Metrics Collector URL</name>
+                    <displayName>Metrics Collector URL</displayName>
+                    <description>The URL of the Ambari Metrics Collector Service</description>
+                    <defaultValue>http://localhost:6188/ws/v1/timeline/metrics</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Application ID</name>
+                    <displayName>Application ID</displayName>
+                    <description>The Application ID to be included in the metrics sent to Ambari</description>
+                    <defaultValue>nifi</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Hostname</name>
+                    <displayName>Hostname</displayName>
+                    <description>The Hostname of this NiFi instance to be included in the metrics sent to Ambari
+                    </description>
+                    <defaultValue>${hostname(true)}</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Process Group ID</name>
+                    <displayName>Process Group ID</displayName>
+                    <description>If specified, the reporting task will send metrics about this process group only. If
+                        not, the root process group is used and global metrics are sent.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <stateful></stateful>
+            <restricted></restricted>
+            <inputRequirement></inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso/>
+        </extension>
+    </extensions>
+</extensionManifest>
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-dbcp-service-nar.xml b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-dbcp-service-nar.xml
new file mode 100644
index 0000000..672ee75
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-dbcp-service-nar.xml
@@ -0,0 +1,325 @@
+<extensionManifest>
+    <systemApiVersion>1.10.0-SNAPSHOT</systemApiVersion>
+    <extensions>
+        <extension>
+            <name>org.apache.nifi.dbcp.DBCPConnectionPoolLookup</name>
+            <type>CONTROLLER_SERVICE</type>
+            <deprecationNotice/>
+            <description>Provides a DBCPService that can be used to dynamically select another DBCPService. This service
+                requires an attribute named 'database.name' to be passed in when asking for a connection, and will throw
+                an exception if the attribute is missing. The value of 'database.name' will be used to select the
+                DBCPService that has been registered with that name. This will allow multiple DBCPServices to be defined
+                and registered, and then selected dynamically at runtime by tagging flow files with the appropriate
+                'database.name' attribute.
+            </description>
+            <tags>
+                <tag>dbcp</tag>
+                <tag>jdbc</tag>
+                <tag>database</tag>
+                <tag>connection</tag>
+                <tag>pooling</tag>
+                <tag>store</tag>
+            </tags>
+            <properties></properties>
+            <dynamicProperties>
+                <dynamicProperty>
+                    <name>The</name>
+                    <value>JDBC property value</value>
+                    <description></description>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                </dynamicProperty>
+            </dynamicProperties>
+            <stateful></stateful>
+            <restricted></restricted>
+            <inputRequirement></inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso/>
+            <providedServiceAPIs>
+                <providedServiceAPI>
+                    <className>org.apache.nifi.dbcp.DBCPService</className>
+                    <groupId>org.apache.nifi</groupId>
+                    <artifactId>nifi-standard-services-api-nar</artifactId>
+                    <version>1.10.0-SNAPSHOT</version>
+                </providedServiceAPI>
+            </providedServiceAPIs>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.dbcp.DBCPConnectionPool</name>
+            <type>CONTROLLER_SERVICE</type>
+            <deprecationNotice/>
+            <description>Provides Database Connection Pooling Service. Connections can be asked from pool and returned
+                after usage.
+            </description>
+            <tags>
+                <tag>dbcp</tag>
+                <tag>jdbc</tag>
+                <tag>database</tag>
+                <tag>connection</tag>
+                <tag>pooling</tag>
+                <tag>store</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Database Connection URL</name>
+                    <displayName>Database Connection URL</displayName>
+                    <description>A database connection URL used to connect to a database. May contain database system
+                        name, host, port, database name and some parameters. The exact syntax of a database connection
+                        URL is specified by your DBMS.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Database Driver Class Name</name>
+                    <displayName>Database Driver Class Name</displayName>
+                    <description>Database driver class name</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>database-driver-locations</name>
+                    <displayName>Database Driver Location(s)</displayName>
+                    <description>Comma-separated list of files/folders and/or URLs containing the driver JAR and its
+                        dependencies (if any). For example '/var/tmp/mariadb-java-client-1.1.7.jar'
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Database User</name>
+                    <displayName>Database User</displayName>
+                    <description>Database user name</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Password</name>
+                    <displayName>Password</displayName>
+                    <description>The password for the database user</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>true</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Max Wait Time</name>
+                    <displayName>Max Wait Time</displayName>
+                    <description>The maximum amount of time that the pool will wait (when there are no available
+                        connections) for a connection to be returned before failing, or -1 to wait indefinitely.
+                    </description>
+                    <defaultValue>500 millis</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Max Total Connections</name>
+                    <displayName>Max Total Connections</displayName>
+                    <description>The maximum number of active connections that can be allocated from this pool at the
+                        same time, or negative for no limit.
+                    </description>
+                    <defaultValue>8</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Validation-query</name>
+                    <displayName>Validation query</displayName>
+                    <description>Validation query used to validate connections before returning them. When connection is
+                        invalid, it get's dropped and new valid connection will be returned. Note!! Using validation
+                        might have some performance penalty.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>dbcp-min-idle-conns</name>
+                    <displayName>Minimum Idle Connections</displayName>
+                    <description>The minimum number of connections that can remain idle in the pool, without extra ones
+                        being created, or zero to create none.
+                    </description>
+                    <defaultValue>0</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>dbcp-max-idle-conns</name>
+                    <displayName>Max Idle Connections</displayName>
+                    <description>The maximum number of connections that can remain idle in the pool, without extra ones
+                        being released, or negative for no limit.
+                    </description>
+                    <defaultValue>8</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>dbcp-max-conn-lifetime</name>
+                    <displayName>Max Connection Lifetime</displayName>
+                    <description>The maximum lifetime in milliseconds of a connection. After this time is exceeded the
+                        connection will fail the next activation, passivation or validation test. A value of zero or
+                        less means the connection has an infinite lifetime.
+                    </description>
+                    <defaultValue>-1</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>dbcp-time-between-eviction-runs</name>
+                    <displayName>Time Between Eviction Runs</displayName>
+                    <description>The number of milliseconds to sleep between runs of the idle connection evictor thread.
+                        When non-positive, no idle connection evictor thread will be run.
+                    </description>
+                    <defaultValue>-1</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>dbcp-min-evictable-idle-time</name>
+                    <displayName>Minimum Evictable Idle Time</displayName>
+                    <description>The minimum amount of time a connection may sit idle in the pool before it is eligible
+                        for eviction.
+                    </description>
+                    <defaultValue>30 mins</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>dbcp-soft-min-evictable-idle-time</name>
+                    <displayName>Soft Minimum Evictable Idle Time</displayName>
+                    <description>The minimum amount of time a connection may sit idle in the pool before it is eligible
+                        for eviction by the idle connection evictor, with the extra condition that at least a minimum
+                        number of idle connections remain in the pool. When the not-soft version of this option is set
+                        to a positive value, it is examined first by the idle connection evictor: when idle connections
+                        are visited by the evictor, idle time is first compared against it (without considering the
+                        number of idle connections in the pool) and then against this soft option, including the minimum
+                        idle connections constraint.
+                    </description>
+                    <defaultValue>-1</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties>
+                <dynamicProperty>
+                    <name>JDBC property name</name>
+                    <value>JDBC property value</value>
+                    <description>Specifies a property name and value to be set on the JDBC connection(s). If Expression
+                        Language is used, evaluation will be performed upon the controller service being enabled. Note
+                        that no flow file input (attributes, e.g.) is available for use in Expression Language
+                        constructs for these properties.
+                    </description>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                </dynamicProperty>
+            </dynamicProperties>
+            <stateful></stateful>
+            <restricted></restricted>
+            <inputRequirement></inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso/>
+            <providedServiceAPIs>
+                <providedServiceAPI>
+                    <className>org.apache.nifi.dbcp.DBCPService</className>
+                    <groupId>org.apache.nifi</groupId>
+                    <artifactId>nifi-standard-services-api-nar</artifactId>
+                    <version>1.10.0-SNAPSHOT</version>
+                </providedServiceAPI>
+            </providedServiceAPIs>
+        </extension>
+    </extensions>
+</extensionManifest>
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-hadoop-nar.xml b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-hadoop-nar.xml
new file mode 100644
index 0000000..d4a91bc
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-hadoop-nar.xml
@@ -0,0 +1,3274 @@
+<extensionManifest>
+    <systemApiVersion>1.10.0-SNAPSHOT</systemApiVersion>
+    <extensions>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.MoveHDFS</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Rename existing files or a directory of files (non-recursive) on Hadoop Distributed File System
+                (HDFS).
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>put</tag>
+                <tag>move</tag>
+                <tag>filesystem</tag>
+                <tag>moveHDFS</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Conflict Resolution Strategy</name>
+                    <displayName>Conflict Resolution Strategy</displayName>
+                    <description>Indicates what should happen when a file with the same name already exists in the
+                        output directory
+                    </description>
+                    <defaultValue>fail</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>replace</displayName>
+                            <value>replace</value>
+                            <description>Replaces the existing file if any.</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>ignore</displayName>
+                            <value>ignore</value>
+                            <description>Failed rename operation stops processing and routes to success.</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>fail</displayName>
+                            <value>fail</value>
+                            <description>Failing to rename a file routes to failure.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Input Directory or File</name>
+                    <displayName>Input Directory or File</displayName>
+                    <description>The HDFS directory from which files should be read, or a single file to read.
+                    </description>
+                    <defaultValue>${path}</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Output Directory</name>
+                    <displayName>Output Directory</displayName>
+                    <description>The HDFS directory where the files will be moved to</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>HDFS Operation</name>
+                    <displayName>HDFS Operation</displayName>
+                    <description>The operation that will be performed on the source file</description>
+                    <defaultValue>move</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>move</displayName>
+                            <value>move</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>copy</displayName>
+                            <value>copy</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>File Filter Regex</name>
+                    <displayName>File Filter Regex</displayName>
+                    <description>A Java Regular Expression for filtering Filenames; if a filter is supplied then only
+                        files whose names match that Regular Expression will be fetched, otherwise all files will be
+                        fetched
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Ignore Dotted Files</name>
+                    <displayName>Ignore Dotted Files</displayName>
+                    <description>If true, files whose names begin with a dot (".") will be ignored</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Remote Owner</name>
+                    <displayName>Remote Owner</displayName>
+                    <description>Changes the owner of the HDFS file to this value after it is written. This only works
+                        if NiFi is running as a user that has HDFS super user privilege to change owner
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Remote Group</name>
+                    <displayName>Remote Group</displayName>
+                    <description>Changes the group of the HDFS file to this value after it is written. This only works
+                        if NiFi is running as a user that has HDFS super user privilege to change group
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>Files that have been successfully renamed on HDFS are transferred to this
+                        relationship
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>failure</name>
+                    <description>Files that could not be renamed on HDFS are transferred to this relationship
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes>
+                <readsAttribute>
+                    <name>filename</name>
+                    <description>The name of the file written to HDFS comes from the value of this attribute.
+                    </description>
+                </readsAttribute>
+            </readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>filename</name>
+                    <description>The name of the file written to HDFS is stored in this attribute.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>absolute.hdfs.path</name>
+                    <description>The absolute path to the file on HDFS is stored in this attribute.</description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>read filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to retrieve any file that NiFi has access to in HDFS
+                            or the local filesystem.
+                        </explanation>
+                    </restriction>
+                    <restriction>
+                        <requiredPermission>write filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to delete any file that NiFi has access to in HDFS or
+                            the local filesystem.
+                        </explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+            <inputRequirement>INPUT_ALLOWED</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.GetHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.GetHDFS</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles. This Processor will
+                delete the file from HDFS after fetching it.
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>get</tag>
+                <tag>fetch</tag>
+                <tag>ingest</tag>
+                <tag>source</tag>
+                <tag>filesystem</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Directory</name>
+                    <displayName>Directory</displayName>
+                    <description>The HDFS directory from which files should be read</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Recurse Subdirectories</name>
+                    <displayName>Recurse Subdirectories</displayName>
+                    <description>Indicates whether to pull files from subdirectories of the HDFS directory</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Keep Source File</name>
+                    <displayName>Keep Source File</displayName>
+                    <description>Determines whether to delete the file from HDFS after it has been successfully
+                        transferred. If true, the file will be fetched repeatedly. This is intended for testing only.
+                    </description>
+                    <defaultValue>false</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>File Filter Regex</name>
+                    <displayName>File Filter Regex</displayName>
+                    <description>A Java Regular Expression for filtering Filenames; if a filter is supplied then only
+                        files whose names match that Regular Expression will be fetched, otherwise all files will be
+                        fetched
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Filter Match Name Only</name>
+                    <displayName>Filter Match Name Only</displayName>
+                    <description>If true then File Filter Regex will match on just the filename, otherwise subdirectory
+                        names will be included with filename in the regex comparison
+                    </description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Ignore Dotted Files</name>
+                    <displayName>Ignore Dotted Files</displayName>
+                    <description>If true, files whose names begin with a dot (".") will be ignored</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Minimum File Age</name>
+                    <displayName>Minimum File Age</displayName>
+                    <description>The minimum age that a file must be in order to be pulled; any file younger than this
+                        amount of time (based on last modification date) will be ignored
+                    </description>
+                    <defaultValue>0 sec</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Maximum File Age</name>
+                    <displayName>Maximum File Age</displayName>
+                    <description>The maximum age that a file must be in order to be pulled; any file older than this
+                        amount of time (based on last modification date) will be ignored
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Polling Interval</name>
+                    <displayName>Polling Interval</displayName>
+                    <description>Indicates how long to wait between performing directory listings</description>
+                    <defaultValue>0 sec</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Batch Size</name>
+                    <displayName>Batch Size</displayName>
+                    <description>The maximum number of files to pull in each iteration, based on run schedule.
+                    </description>
+                    <defaultValue>100</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>IO Buffer Size</name>
+                    <displayName>IO Buffer Size</displayName>
+                    <description>Amount of memory to use to buffer file contents during IO. This overrides the Hadoop
+                        Configuration
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Compression codec</name>
+                    <displayName>Compression codec</displayName>
+                    <description></description>
+                    <defaultValue>NONE</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>NONE</displayName>
+                            <value>NONE</value>
+                            <description>No compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>DEFAULT</displayName>
+                            <value>DEFAULT</value>
+                            <description>Default ZLIB compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>BZIP</displayName>
+                            <value>BZIP</value>
+                            <description>BZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>GZIP</displayName>
+                            <value>GZIP</value>
+                            <description>GZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZ4</displayName>
+                            <value>LZ4</value>
+                            <description>LZ4 compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZO</displayName>
+                            <value>LZO</value>
+                            <description>LZO compression - it assumes LD_LIBRARY_PATH has been set and jar is
+                                available
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>SNAPPY</displayName>
+                            <value>SNAPPY</value>
+                            <description>Snappy compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>AUTOMATIC</displayName>
+                            <value>AUTOMATIC</value>
+                            <description>Will attempt to automatically detect the compression codec.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>All files retrieved from HDFS are transferred to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>filename</name>
+                    <description>The name of the file that was read from HDFS.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>path</name>
+                    <description>The path is set to the relative path of the file's directory on HDFS. For example, if
+                        the Directory property is set to /tmp, then files picked up from /tmp will have the path
+                        attribute set to "./". If the Recurse Subdirectories property is set to true and a file is
+                        picked up from /tmp/abc/1/2/3, then the path attribute will be set to "abc/1/2/3".
+                    </description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>read filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to retrieve any file that NiFi has access to in HDFS
+                            or the local filesystem.
+                        </explanation>
+                    </restriction>
+                    <restriction>
+                        <requiredPermission>write filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to delete any file that NiFi has access to in HDFS or
+                            the local filesystem.
+                        </explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+            <inputRequirement>INPUT_FORBIDDEN</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.ListHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.PutHDFS</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Write FlowFile data to Hadoop Distributed File System (HDFS)</description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>put</tag>
+                <tag>copy</tag>
+                <tag>filesystem</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Directory</name>
+                    <displayName>Directory</displayName>
+                    <description>The parent HDFS directory to which files should be written. The directory will be
+                        created if it doesn't exist.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Conflict Resolution Strategy</name>
+                    <displayName>Conflict Resolution Strategy</displayName>
+                    <description>Indicates what should happen when a file with the same name already exists in the
+                        output directory
+                    </description>
+                    <defaultValue>fail</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>replace</displayName>
+                            <value>replace</value>
+                            <description>Replaces the existing file if any.</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>ignore</displayName>
+                            <value>ignore</value>
+                            <description>Ignores the flow file and routes it to success.</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>fail</displayName>
+                            <value>fail</value>
+                            <description>Penalizes the flow file and routes it to failure.</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>append</displayName>
+                            <value>append</value>
+                            <description>Appends to the existing file if any, creates a new file otherwise.
+                            </description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Block Size</name>
+                    <displayName>Block Size</displayName>
+                    <description>Size of each block as written to HDFS. This overrides the Hadoop Configuration
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>IO Buffer Size</name>
+                    <displayName>IO Buffer Size</displayName>
+                    <description>Amount of memory to use to buffer file contents during IO. This overrides the Hadoop
+                        Configuration
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Replication</name>
+                    <displayName>Replication</displayName>
+                    <description>Number of times that HDFS will replicate each file. This overrides the Hadoop
+                        Configuration
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Permissions umask</name>
+                    <displayName>Permissions umask</displayName>
+                    <description>A umask represented as an octal number which determines the permissions of files
+                        written to HDFS. This overrides the Hadoop property "fs.permission.umask-mode". If this property
+                        and "fs.permission.umask-mode" are undefined, the Hadoop default "022" will be used.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Remote Owner</name>
+                    <displayName>Remote Owner</displayName>
+                    <description>Changes the owner of the HDFS file to this value after it is written. This only works
+                        if NiFi is running as a user that has HDFS super user privilege to change owner
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Remote Group</name>
+                    <displayName>Remote Group</displayName>
+                    <description>Changes the group of the HDFS file to this value after it is written. This only works
+                        if NiFi is running as a user that has HDFS super user privilege to change group
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Compression codec</name>
+                    <displayName>Compression codec</displayName>
+                    <description></description>
+                    <defaultValue>NONE</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>NONE</displayName>
+                            <value>NONE</value>
+                            <description>No compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>DEFAULT</displayName>
+                            <value>DEFAULT</value>
+                            <description>Default ZLIB compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>BZIP</displayName>
+                            <value>BZIP</value>
+                            <description>BZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>GZIP</displayName>
+                            <value>GZIP</value>
+                            <description>GZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZ4</displayName>
+                            <value>LZ4</value>
+                            <description>LZ4 compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZO</displayName>
+                            <value>LZO</value>
+                            <description>LZO compression - it assumes LD_LIBRARY_PATH has been set and jar is
+                                available
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>SNAPPY</displayName>
+                            <value>SNAPPY</value>
+                            <description>Snappy compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>AUTOMATIC</displayName>
+                            <value>AUTOMATIC</value>
+                            <description>Will attempt to automatically detect the compression codec.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>Files that have been successfully written to HDFS are transferred to this
+                        relationship
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>failure</name>
+                    <description>Files that could not be written to HDFS for some reason are transferred to this
+                        relationship
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes>
+                <readsAttribute>
+                    <name>filename</name>
+                    <description>The name of the file written to HDFS comes from the value of this attribute.
+                    </description>
+                </readsAttribute>
+            </readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>filename</name>
+                    <description>The name of the file written to HDFS is stored in this attribute.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>absolute.hdfs.path</name>
+                    <description>The absolute path to the file on HDFS is stored in this attribute.</description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>write filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to delete any file that NiFi has access to in HDFS or
+                            the local filesystem.
+                        </explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+            <inputRequirement>INPUT_REQUIRED</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.GetHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.inotify.GetHDFSEvents</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>This processor polls the notification events provided by the HdfsAdmin API. Since this uses the
+                HdfsAdmin APIs it is required to run as an HDFS super user. Currently there are six types of events
+                (append, close, create, metadata, rename, and unlink). Please see org.apache.hadoop.hdfs.inotify.Event
+                documentation for full explanations of each event. This processor will poll for new events based on a
+                defined duration. For each event received a new flow file will be created with the expected attributes
+                and the event itself serialized to JSON and written to the flow file's content. For example, if
+                event.type is APPEND then the content of the flow file will contain a JSON file containing the
+                information about the append event. If successful the flow files are sent to the 'success' relationship.
+                Be careful of where the generated flow files are stored. If the flow files are stored in one of
+                processor's watch directories there will be a never ending flow of events. It is also important to be
+                aware that this processor must consume all events. The filtering must happen within the processor. This
+                is because the HDFS admin's event notifications API does not have filtering.
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>events</tag>
+                <tag>inotify</tag>
+                <tag>notifications</tag>
+                <tag>filesystem</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Poll Duration</name>
+                    <displayName>Poll Duration</displayName>
+                    <description>The time before the polling method returns with the next batch of events if they exist.
+                        It may exceed this amount of time by up to the time required for an RPC to the NameNode.
+                    </description>
+                    <defaultValue>1 second</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>HDFS Path to Watch</name>
+                    <displayName>HDFS Path to Watch</displayName>
+                    <description>The HDFS path to get event notifications for. This property accepts both expression
+                        language and regular expressions. This will be evaluated during the OnScheduled phase.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Ignore Hidden Files</name>
+                    <displayName>Ignore Hidden Files</displayName>
+                    <description>If true and the final component of the path associated with a given event starts with a
+                        '.' then that event will not be processed.
+                    </description>
+                    <defaultValue>false</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Event Types to Filter On</name>
+                    <displayName>Event Types to Filter On</displayName>
+                    <description>A comma-separated list of event types to process. Valid event types are: append, close,
+                        create, metadata, rename, and unlink. Case does not matter.
+                    </description>
+                    <defaultValue>append, close, create, metadata, rename, unlink</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>IOException Retries During Event Polling</name>
+                    <displayName>IOException Retries During Event Polling</displayName>
+                    <description>According to the HDFS admin API for event polling it is good to retry at least a few
+                        times. This number defines how many times the poll will be retried if it throws an IOException.
+                    </description>
+                    <defaultValue>3</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>A flow file with updated information about a specific event will be sent to this
+                        relationship.
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>mime.type</name>
+                    <description>This is always application/json.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.inotify.event.type</name>
+                    <description>This will specify the specific HDFS notification event type. Currently there are six
+                        types of events (append, close, create, metadata, rename, and unlink).
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.inotify.event.path</name>
+                    <description>The specific path that the event is tied to.</description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful>
+                <description>The last used transaction id is stored. This is used</description>
+                <scopes>
+                    <scope>CLUSTER</scope>
+                </scopes>
+            </stateful>
+            <restricted></restricted>
+            <inputRequirement>INPUT_FORBIDDEN</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.GetHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.FetchHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.ListHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Creates Hadoop Sequence Files from incoming flow files</description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>sequence file</tag>
+                <tag>create</tag>
+                <tag>sequencefile</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>compression type</name>
+                    <displayName>Compression type</displayName>
+                    <description>Type of compression to use when creating Sequence File</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>NONE</displayName>
+                            <value>NONE</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>RECORD</displayName>
+                            <value>RECORD</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>BLOCK</displayName>
+                            <value>BLOCK</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Compression codec</name>
+                    <displayName>Compression codec</displayName>
+                    <description></description>
+                    <defaultValue>NONE</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>NONE</displayName>
+                            <value>NONE</value>
+                            <description>No compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>DEFAULT</displayName>
+                            <value>DEFAULT</value>
+                            <description>Default ZLIB compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>BZIP</displayName>
+                            <value>BZIP</value>
+                            <description>BZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>GZIP</displayName>
+                            <value>GZIP</value>
+                            <description>GZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZ4</displayName>
+                            <value>LZ4</value>
+                            <description>LZ4 compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZO</displayName>
+                            <value>LZO</value>
+                            <description>LZO compression - it assumes LD_LIBRARY_PATH has been set and jar is
+                                available
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>SNAPPY</displayName>
+                            <value>SNAPPY</value>
+                            <description>Snappy compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>AUTOMATIC</displayName>
+                            <value>AUTOMATIC</value>
+                            <description>Will attempt to automatically detect the compression codec.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>Generated Sequence Files are sent to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>failure</name>
+                    <description>Incoming files that failed to generate a Sequence File are sent to this relationship
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes></writesAttributes>
+            <stateful></stateful>
+            <restricted></restricted>
+            <inputRequirement>INPUT_REQUIRED</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.FetchHDFS</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Retrieves a file from HDFS. The content of the incoming FlowFile is replaced by the content of
+                the file in HDFS. The file in HDFS is left intact without any changes being made to it.
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>hdfs</tag>
+                <tag>get</tag>
+                <tag>ingest</tag>
+                <tag>fetch</tag>
+                <tag>source</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>HDFS Filename</name>
+                    <displayName>HDFS Filename</displayName>
+                    <description>The name of the HDFS file to retrieve</description>
+                    <defaultValue>${path}/${filename}</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Compression codec</name>
+                    <displayName>Compression codec</displayName>
+                    <description></description>
+                    <defaultValue>NONE</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>NONE</displayName>
+                            <value>NONE</value>
+                            <description>No compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>DEFAULT</displayName>
+                            <value>DEFAULT</value>
+                            <description>Default ZLIB compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>BZIP</displayName>
+                            <value>BZIP</value>
+                            <description>BZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>GZIP</displayName>
+                            <value>GZIP</value>
+                            <description>GZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZ4</displayName>
+                            <value>LZ4</value>
+                            <description>LZ4 compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZO</displayName>
+                            <value>LZO</value>
+                            <description>LZO compression - it assumes LD_LIBRARY_PATH has been set and jar is
+                                available
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>SNAPPY</displayName>
+                            <value>SNAPPY</value>
+                            <description>Snappy compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>AUTOMATIC</displayName>
+                            <value>AUTOMATIC</value>
+                            <description>Will attempt to automatically detect the compression codec.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>FlowFiles will be routed to this relationship once they have been updated with the
+                        content of the HDFS file
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>comms.failure</name>
+                    <description>FlowFiles will be routed to this relationship if the content of the HDFS file cannot be
+                        retrieve due to a communications failure. This generally indicates that the Fetch should be
+                        tried again.
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>failure</name>
+                    <description>FlowFiles will be routed to this relationship if the content of the HDFS file cannot be
+                        retrieved and trying again will likely not be helpful. This would occur, for instance, if the
+                        file is not found or if there is a permissions issue
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>hdfs.failure.reason</name>
+                    <description>When a FlowFile is routed to 'failure', this attribute is added indicating why the file
+                        could not be fetched from HDFS
+                    </description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>read filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to retrieve any file that NiFi has access to in HDFS
+                            or the local filesystem.
+                        </explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+            <inputRequirement>INPUT_REQUIRED</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.ListHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.GetHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.DeleteHDFS</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Deletes one or more files or directories from HDFS. The path can be provided as an attribute
+                from an incoming FlowFile, or a statically set path that is periodically removed. If this processor has
+                an incoming connection, itwill ignore running on a periodic basis and instead rely on incoming FlowFiles
+                to trigger a delete. Note that you may use a wildcard character to match multiple files or directories.
+                If there are no incoming connections no flowfiles will be transfered to any output relationships. If
+                there is an incoming flowfile then provided there are no detected failures it will be transferred to
+                success otherwise it will be sent to false. If knowledge of globbed files deleted is necessary use
+                ListHDFS first to produce a specific list of files to delete.
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>delete</tag>
+                <tag>remove</tag>
+                <tag>filesystem</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>file_or_directory</name>
+                    <displayName>Path</displayName>
+                    <description>The HDFS file or directory to delete. A wildcard expression may be used to only delete
+                        certain files
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>recursive</name>
+                    <displayName>Recursive</displayName>
+                    <description>Remove contents of a non-empty directory recursively</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>When an incoming flowfile is used then if there are no errors invoking delete the
+                        flowfile will route here.
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>failure</name>
+                    <description>When an incoming flowfile is used and there is a failure while deleting then the
+                        flowfile will route here.
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>hdfs.filename</name>
+                    <description>HDFS file to be deleted. If multiple files are deleted, then only the last filename is
+                        set.
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.path</name>
+                    <description>HDFS Path specified in the delete request. If multiple paths are deleted, then only the
+                        last path is set.
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.error.message</name>
+                    <description>HDFS error message related to the hdfs.error.code</description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>write filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to delete any file that NiFi has access to in HDFS or
+                            the local filesystem.
+                        </explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+            <inputRequirement>INPUT_ALLOWED</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.ListHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.GetHDFSSequenceFile</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Fetch sequence files from Hadoop Distributed File System (HDFS) into FlowFiles</description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>get</tag>
+                <tag>fetch</tag>
+                <tag>ingest</tag>
+                <tag>source</tag>
+                <tag>sequence file</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Directory</name>
+                    <displayName>Directory</displayName>
+                    <description>The HDFS directory from which files should be read</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Recurse Subdirectories</name>
+                    <displayName>Recurse Subdirectories</displayName>
+                    <description>Indicates whether to pull files from subdirectories of the HDFS directory</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Keep Source File</name>
+                    <displayName>Keep Source File</displayName>
+                    <description>Determines whether to delete the file from HDFS after it has been successfully
+                        transferred. If true, the file will be fetched repeatedly. This is intended for testing only.
+                    </description>
+                    <defaultValue>false</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>File Filter Regex</name>
+                    <displayName>File Filter Regex</displayName>
+                    <description>A Java Regular Expression for filtering Filenames; if a filter is supplied then only
+                        files whose names match that Regular Expression will be fetched, otherwise all files will be
+                        fetched
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Filter Match Name Only</name>
+                    <displayName>Filter Match Name Only</displayName>
+                    <description>If true then File Filter Regex will match on just the filename, otherwise subdirectory
+                        names will be included with filename in the regex comparison
+                    </description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Ignore Dotted Files</name>
+                    <displayName>Ignore Dotted Files</displayName>
+                    <description>If true, files whose names begin with a dot (".") will be ignored</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Minimum File Age</name>
+                    <displayName>Minimum File Age</displayName>
+                    <description>The minimum age that a file must be in order to be pulled; any file younger than this
+                        amount of time (based on last modification date) will be ignored
+                    </description>
+                    <defaultValue>0 sec</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Maximum File Age</name>
+                    <displayName>Maximum File Age</displayName>
+                    <description>The maximum age that a file must be in order to be pulled; any file older than this
+                        amount of time (based on last modification date) will be ignored
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Polling Interval</name>
+                    <displayName>Polling Interval</displayName>
+                    <description>Indicates how long to wait between performing directory listings</description>
+                    <defaultValue>0 sec</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Batch Size</name>
+                    <displayName>Batch Size</displayName>
+                    <description>The maximum number of files to pull in each iteration, based on run schedule.
+                    </description>
+                    <defaultValue>100</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>IO Buffer Size</name>
+                    <displayName>IO Buffer Size</displayName>
+                    <description>Amount of memory to use to buffer file contents during IO. This overrides the Hadoop
+                        Configuration
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Compression codec</name>
+                    <displayName>Compression codec</displayName>
+                    <description></description>
+                    <defaultValue>NONE</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>NONE</displayName>
+                            <value>NONE</value>
+                            <description>No compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>DEFAULT</displayName>
+                            <value>DEFAULT</value>
+                            <description>Default ZLIB compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>BZIP</displayName>
+                            <value>BZIP</value>
+                            <description>BZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>GZIP</displayName>
+                            <value>GZIP</value>
+                            <description>GZIP compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZ4</displayName>
+                            <value>LZ4</value>
+                            <description>LZ4 compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>LZO</displayName>
+                            <value>LZO</value>
+                            <description>LZO compression - it assumes LD_LIBRARY_PATH has been set and jar is
+                                available
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>SNAPPY</displayName>
+                            <value>SNAPPY</value>
+                            <description>Snappy compression</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>AUTOMATIC</displayName>
+                            <value>AUTOMATIC</value>
+                            <description>Will attempt to automatically detect the compression codec.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>FlowFile Content</name>
+                    <displayName>FlowFile Content</displayName>
+                    <description>Indicate if the content is to be both the key and value of the Sequence File, or just
+                        the value.
+                    </description>
+                    <defaultValue>VALUE ONLY</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>VALUE ONLY</displayName>
+                            <value>VALUE ONLY</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>KEY VALUE PAIR</displayName>
+                            <value>KEY VALUE PAIR</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>All files retrieved from HDFS are transferred to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>filename</name>
+                    <description>The name of the file that was read from HDFS.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>path</name>
+                    <description>The path is set to the relative path of the file's directory on HDFS. For example, if
+                        the Directory property is set to /tmp, then files picked up from /tmp will have the path
+                        attribute set to "./". If the Recurse Subdirectories property is set to true and a file is
+                        picked up from /tmp/abc/1/2/3, then the path attribute will be set to "abc/1/2/3".
+                    </description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>read filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to retrieve any file that NiFi has access to in HDFS
+                            or the local filesystem.
+                        </explanation>
+                    </restriction>
+                    <restriction>
+                        <requiredPermission>write filesystem</requiredPermission>
+                        <explanation>Provides operator the ability to delete any file that NiFi has access to in HDFS or
+                            the local filesystem.
+                        </explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+            <inputRequirement>INPUT_FORBIDDEN</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.GetHDFSFileInfo</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Retrieves a listing of files and directories from HDFS. This processor creates a FlowFile(s)
+                that represents the HDFS file/dir with relevant information. Main purpose of this processor to provide
+                functionality similar to HDFS Client, i.e. count, du, ls, test, etc. Unlike ListHDFS, this processor is
+                stateless, supports incoming connections and provides information on a dir level.
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>get</tag>
+                <tag>list</tag>
+                <tag>ingest</tag>
+                <tag>source</tag>
+                <tag>filesystem</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-full-path</name>
+                    <displayName>Full path</displayName>
+                    <description>A directory to start listing from, or a file's full path.</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-recurse-subdirs</name>
+                    <displayName>Recurse Subdirectories</displayName>
+                    <description>Indicates whether to list files from subdirectories of the HDFS directory</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-dir-filter</name>
+                    <displayName>Directory Filter</displayName>
+                    <description>Regex. Only directories whose names match the given regular expression will be picked
+                        up. If not provided, any filter would be apply (performance considerations).
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-file-filter</name>
+                    <displayName>File Filter</displayName>
+                    <description>Regex. Only files whose names match the given regular expression will be picked up. If
+                        not provided, any filter would be apply (performance considerations).
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-file-exclude-filter</name>
+                    <displayName>Exclude Files</displayName>
+                    <description>Regex. Files whose names match the given regular expression will not be picked up. If
+                        not provided, any filter won't be apply (performance considerations).
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-ignore-dotted-dirs</name>
+                    <displayName>Ignore Dotted Directories</displayName>
+                    <description>If true, directories whose names begin with a dot (".") will be ignored</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-ignore-dotted-files</name>
+                    <displayName>Ignore Dotted Files</displayName>
+                    <description>If true, files whose names begin with a dot (".") will be ignored</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-group</name>
+                    <displayName>Group Results</displayName>
+                    <description>Groups HDFS objects</description>
+                    <defaultValue>gethdfsfileinfo-group-all</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>All</displayName>
+                            <value>gethdfsfileinfo-group-all</value>
+                            <description>Group all results into a single flowfile.</description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>Parent Directory</displayName>
+                            <value>gethdfsfileinfo-group-parent-dir</value>
+                            <description>Group HDFS objects by their parent directories only. Processor will generate
+                                flowfile for each directory (if recursive). If 'Recurse Subdirectories' property set to
+                                'false', then will have the same effect as 'All'
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>None</displayName>
+                            <value>gethdfsfileinfo-group-none</value>
+                            <description>Don't group results. Generate flowfile per each HDFS object.</description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>gethdfsfileinfo-destination</name>
+                    <displayName>Destination</displayName>
+                    <description>Sets the destination for the resutls. When set to 'Content', attributes of flowfile
+                        won't be used for storing results.
+                    </description>
+                    <defaultValue>gethdfsfileinfo-dest-content</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>Attributes</displayName>
+                            <value>gethdfsfileinfo-dest-attr</value>
+                            <description>Details of given HDFS object will be stored in attributes of flowfile. WARNING:
+                                In case when scan finds thousands or millions of objects, having huge values in
+                                attribute could impact flow file repo and GC/heap usage. Use content destination for
+                                such cases.
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>Content</displayName>
+                            <value>gethdfsfileinfo-dest-content</value>
+                            <description>Details of given HDFS object will be stored in a content in JSON format
+                            </description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>All successfully generated FlowFiles are transferred to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>not found</name>
+                    <description>If no objects are found, original FlowFile are transferred to this relationship
+                    </description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>failure</name>
+                    <description>All failed attempts to access HDFS will be routed to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+                <relationship>
+                    <name>original</name>
+                    <description>Original FlowFiles are transferred to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>hdfs.objectName</name>
+                    <description>The name of the file/dir found on HDFS.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.path</name>
+                    <description>The path is set to the absolute path of the object's parent directory on HDFS. For
+                        example, if an object is a directory 'foo', under directory '/bar' then 'hdfs.objectName' will
+                        have value 'foo', and 'hdfs.path' will be '/bar'
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.type</name>
+                    <description>The type of an object. Possible values: directory, file, link</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.owner</name>
+                    <description>The user that owns the object in HDFS</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.group</name>
+                    <description>The group that owns the object in HDFS</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.lastModified</name>
+                    <description>The timestamp of when the object in HDFS was last modified, as milliseconds since
+                        midnight Jan 1, 1970 UTC
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.length</name>
+                    <description>In case of files: The number of bytes in the file in HDFS. In case of dirs: Retuns
+                        storage space consumed by directory.
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.count.files</name>
+                    <description>In case of type='directory' will represent total count of files under this dir. Won't
+                        be populated to other types of HDFS objects.
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.count.dirs</name>
+                    <description>In case of type='directory' will represent total count of directories under this dir
+                        (including itself). Won't be populated to other types of HDFS objects.
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.replication</name>
+                    <description>The number of HDFS replicas for the file</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.permissions</name>
+                    <description>The permissions for the object in HDFS. This is formatted as 3 characters for the
+                        owner, 3 for the group, and 3 for other users. For example rw-rw-r--
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.status</name>
+                    <description>The status contains comma separated list of file/dir paths, which couldn't be
+                        listed/accessed. Status won't be set if no errors occured.
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.full.tree</name>
+                    <description>When destination is 'attribute', will be populated with full tree of HDFS directory in
+                        JSON format.WARNING: In case when scan finds thousands or millions of objects, having huge
+                        values in attribute could impact flow file repo and GC/heap usage. Use content destination for
+                        such cases
+                    </description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful></stateful>
+            <restricted></restricted>
+            <inputRequirement>INPUT_ALLOWED</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.ListHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.GetHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.FetchHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+            </seeAlso>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.hadoop.ListHDFS</name>
+            <type>PROCESSOR</type>
+            <deprecationNotice/>
+            <description>Retrieves a listing of files from HDFS. Each time a listing is performed, the files with the
+                latest timestamp will be excluded and picked up during the next execution of the processor. This is done
+                to ensure that we do not miss any files, or produce duplicates, in the cases where files with the same
+                timestamp are written immediately before and after a single execution of the processor. For each file
+                that is listed in HDFS, this processor creates a FlowFile that represents the HDFS file to be fetched in
+                conjunction with FetchHDFS. This Processor is designed to run on Primary Node only in a cluster. If the
+                primary node changes, the new Primary Node will pick up where the previous node left off without
+                duplicating all of the data. Unlike GetHDFS, this Processor does not delete any data from HDFS.
+            </description>
+            <tags>
+                <tag>hadoop</tag>
+                <tag>HDFS</tag>
+                <tag>get</tag>
+                <tag>list</tag>
+                <tag>ingest</tag>
+                <tag>source</tag>
+                <tag>filesystem</tag>
+            </tags>
+            <properties>
+                <property>
+                    <name>Hadoop Configuration Resources</name>
+                    <displayName>Hadoop Configuration Resources</displayName>
+                    <description>A file or comma separated list of files which contains the Hadoop file system
+                        configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and
+                        'hdfs-site.xml' file or will revert to a default configuration. To use swebhdfs, see 'Additional
+                        Details' section of PutHDFS's documentation.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>kerberos-credentials-service</name>
+                    <displayName>Kerberos Credentials Service</displayName>
+                    <description>Specifies the Kerberos Credentials Controller Service that should be used for
+                        authenticating with Kerberos
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.kerberos.KerberosCredentialsService</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Principal</name>
+                    <displayName>Kerberos Principal</displayName>
+                    <description>Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in
+                        your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Keytab</name>
+                    <displayName>Kerberos Keytab</displayName>
+                    <description>Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be
+                        set in your nifi.properties
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Kerberos Relogin Period</name>
+                    <displayName>Kerberos Relogin Period</displayName>
+                    <description>Period of time which should pass before attempting a kerberos relogin.
+
+                        This property has been deprecated, and has no effect on processing. Relogins now occur
+                        automatically.
+                    </description>
+                    <defaultValue>4 hours</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>VARIABLE_REGISTRY</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Additional Classpath Resources</name>
+                    <displayName>Additional Classpath Resources</displayName>
+                    <description>A comma-separated list of paths to files and/or directories that will be added to the
+                        classpath. When specifying a directory, all files with in the directory will be added to the
+                        classpath, but further sub-directories will not be included.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>true</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Distributed Cache Service</name>
+                    <displayName>Distributed Cache Service</displayName>
+                    <description>This property is ignored. State will be stored in the LOCAL or CLUSTER scope by the
+                        State Manager based on NiFi's configuration.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <controllerServiceDefinition>
+                        <className>org.apache.nifi.distributed.cache.client.DistributedMapCacheClient</className>
+                        <groupId>org.apache.nifi</groupId>
+                        <artifactId>org.apache.nifi:nifi-standard-services-api-nar:nar:1.10.0-SNAPSHOT</artifactId>
+                        <version>1.10.0-SNAPSHOT</version>
+                    </controllerServiceDefinition>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Directory</name>
+                    <displayName>Directory</displayName>
+                    <description>The HDFS directory from which files should be read</description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>true</expressionLanguageSupported>
+                    <expressionLanguageScope>FLOWFILE_ATTRIBUTES</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>Recurse Subdirectories</name>
+                    <displayName>Recurse Subdirectories</displayName>
+                    <description>Indicates whether to list files from subdirectories of the HDFS directory</description>
+                    <defaultValue>true</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>true</displayName>
+                            <value>true</value>
+                            <description></description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>false</displayName>
+                            <value>false</value>
+                            <description></description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>File Filter</name>
+                    <displayName>File Filter</displayName>
+                    <description>Only files whose names match the given regular expression will be picked up
+                    </description>
+                    <defaultValue>[^\.].*</defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>file-filter-mode</name>
+                    <displayName>File Filter Mode</displayName>
+                    <description>Determines how the regular expression in File Filter will be used when retrieving
+                        listings.
+                    </description>
+                    <defaultValue>filter-mode-directories-and-files</defaultValue>
+                    <allowableValues>
+                        <allowableValue>
+                            <displayName>Directories and Files</displayName>
+                            <value>filter-mode-directories-and-files</value>
+                            <description>Filtering will be applied to the names of directories and files. If Recurse
+                                Subdirectories is set to true, only subdirectories with a matching name will be searched
+                                for files that match the regular expression defined in File Filter.
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>Files Only</displayName>
+                            <value>filter-mode-files-only</value>
+                            <description>Filtering will only be applied to the names of files. If Recurse Subdirectories
+                                is set to true, the entire subdirectory tree will be searched for files that match the
+                                regular expression defined in File Filter.
+                            </description>
+                        </allowableValue>
+                        <allowableValue>
+                            <displayName>Full Path</displayName>
+                            <value>filter-mode-full-path</value>
+                            <description>Filtering will be applied to the full path of files. If Recurse Subdirectories
+                                is set to true, the entire subdirectory tree will be searched for files in which the
+                                full path of the file matches the regular expression defined in File Filter.
+                            </description>
+                        </allowableValue>
+                    </allowableValues>
+                    <required>true</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>minimum-file-age</name>
+                    <displayName>Minimum File Age</displayName>
+                    <description>The minimum age that a file must be in order to be pulled; any file younger than this
+                        amount of time (based on last modification date) will be ignored
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+                <property>
+                    <name>maximum-file-age</name>
+                    <displayName>Maximum File Age</displayName>
+                    <description>The maximum age that a file must be in order to be pulled; any file older than this
+                        amount of time (based on last modification date) will be ignored. Minimum value is 100ms.
+                    </description>
+                    <defaultValue></defaultValue>
+                    <allowableValues></allowableValues>
+                    <required>false</required>
+                    <sensitive>false</sensitive>
+                    <expressionLanguageSupported>false</expressionLanguageSupported>
+                    <expressionLanguageScope>NONE</expressionLanguageScope>
+                    <dynamicallyModifiesClasspath>false</dynamicallyModifiesClasspath>
+                    <dynamic>false</dynamic>
+                </property>
+            </properties>
+            <dynamicProperties></dynamicProperties>
+            <relationships>
+                <relationship>
+                    <name>success</name>
+                    <description>All FlowFiles are transferred to this relationship</description>
+                    <autoTerminated>false</autoTerminated>
+                </relationship>
+            </relationships>
+            <dynamicRelationship></dynamicRelationship>
+            <readsAttributes></readsAttributes>
+            <writesAttributes>
+                <writesAttribute>
+                    <name>filename</name>
+                    <description>The name of the file that was read from HDFS.</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>path</name>
+                    <description>The path is set to the absolute path of the file's directory on HDFS. For example, if
+                        the Directory property is set to /tmp, then files picked up from /tmp will have the path
+                        attribute set to "./". If the Recurse Subdirectories property is set to true and a file is
+                        picked up from /tmp/abc/1/2/3, then the path attribute will be set to "/tmp/abc/1/2/3".
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.owner</name>
+                    <description>The user that owns the file in HDFS</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.group</name>
+                    <description>The group that owns the file in HDFS</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.lastModified</name>
+                    <description>The timestamp of when the file in HDFS was last modified, as milliseconds since
+                        midnight Jan 1, 1970 UTC
+                    </description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.length</name>
+                    <description>The number of bytes in the file in HDFS</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.replication</name>
+                    <description>The number of HDFS replicas for hte file</description>
+                </writesAttribute>
+                <writesAttribute>
+                    <name>hdfs.permissions</name>
+                    <description>The permissions for the file in HDFS. This is formatted as 3 characters for the owner,
+                        3 for the group, and 3 for other users. For example rw-rw-r--
+                    </description>
+                </writesAttribute>
+            </writesAttributes>
+            <stateful>
+                <description>After performing a listing of HDFS files, the latest timestamp of all the files listed and
+                    the latest timestamp of all the files transferred are both stored. This allows the Processor to list
+                    only files that have been added or modified after this date the next time that the Processor is run,
+                    without having to store all of the actual filenames/paths which could lead to performance problems.
+                    State is stored across the cluster so that this Processor can be run on Primary Node only and if a
+                    new Primary Node is selected, the new node can pick up where the previous node left off, without
+                    duplicating the data.
+                </description>
+                <scopes>
+                    <scope>CLUSTER</scope>
+                </scopes>
+            </stateful>
+            <restricted></restricted>
+            <inputRequirement>INPUT_FORBIDDEN</inputRequirement>
+            <systemResourceConsiderations></systemResourceConsiderations>
+            <seeAlso>
+                <see>org.apache.nifi.processors.hadoop.GetHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.FetchHDFS</see>
+                <see>org.apache.nifi.processors.hadoop.PutHDFS</see>
+            </seeAlso>
+        </extension>
+    </extensions>
+</extensionManifest>
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-missing-sys-api.xml b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-missing-sys-api.xml
new file mode 100644
index 0000000..d490690
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-missing-sys-api.xml
@@ -0,0 +1,4 @@
+<extensionManifest>
+    <systemApiVersion></systemApiVersion>
+    <extensions></extensions>
+</extensionManifest>
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-test-components.xml b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-test-components.xml
new file mode 100644
index 0000000..7783e1d
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/descriptors/extension-manifest-test-components.xml
@@ -0,0 +1,56 @@
+<extensionManifest>
+    <systemApiVersion>1.8.0</systemApiVersion>
+    <extensions>
+        <extension>
+            <name>org.apache.nifi.processors.TestProcessor1</name>
+            <type>PROCESSOR</type>
+            <description>Test processor 1.</description>
+            <tags>
+                <tag>test</tag>
+                <tag>processor</tag>
+            </tags>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.TestProcessor2</name>
+            <type>PROCESSOR</type>
+            <description>Test processor 2.</description>
+            <tags>
+                <tag>test</tag>
+                <tag>processor</tag>
+            </tags>
+            <restricted>
+                <restrictions>
+                    <restriction>
+                        <requiredPermission>write filesystem</requiredPermission>
+                        <explanation>Test explanation.</explanation>
+                    </restriction>
+                </restrictions>
+            </restricted>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.processors.TestProcessor3</name>
+            <type>PROCESSOR</type>
+            <description/>
+            <tags>
+            </tags>
+        </extension>
+        <extension>
+            <name>org.apache.nifi.service.TestServiceImpl</name>
+            <type>CONTROLLER_SERVICE</type>
+            <deprecationNotice/>
+            <description>Test service.</description>
+            <tags>
+                <tag>test</tag>
+                <tag>service</tag>
+            </tags>
+            <providedServiceAPIs>
+                <providedServiceAPI>
+                    <className>org.apache.nifi.service.TestService</className>
+                    <groupId>org.apache.nifi</groupId>
+                    <artifactId>nifi-test-service-api-nar</artifactId>
+                    <version>1.0.0</version>
+                </providedServiceAPI>
+            </providedServiceAPIs>
+        </extension>
+    </extensions>
+</extensionManifest>
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar-missing-extension-descriptor.nar b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar-missing-extension-descriptor.nar
new file mode 100644
index 0000000..def9f8a
Binary files /dev/null and b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar-missing-extension-descriptor.nar differ
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar.nar b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar.nar
index e33ee88..f61fca6 100644
Binary files a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar.nar and b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-foo-nar.nar differ
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-framework-nar.nar b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-framework-nar.nar
index 0d0319b..3089458 100644
Binary files a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-framework-nar.nar and b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-framework-nar.nar differ
diff --git a/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-hadoop-nar.nar b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-hadoop-nar.nar
new file mode 100644
index 0000000..bf0020d
Binary files /dev/null and b/nifi-registry-core/nifi-registry-bundle-utils/src/test/resources/nars/nifi-hadoop-nar.nar differ
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionBundleClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/BundleClient.java
similarity index 79%
rename from nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionBundleClient.java
rename to nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/BundleClient.java
index b2724ae..b71b22a 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionBundleClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/BundleClient.java
@@ -16,8 +16,8 @@
  */
 package org.apache.nifi.registry.client;
 
-import org.apache.nifi.registry.extension.ExtensionBundle;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleFilterParams;
+import org.apache.nifi.registry.extension.bundle.Bundle;
+import org.apache.nifi.registry.extension.bundle.BundleFilterParams;
 
 import java.io.IOException;
 import java.util.List;
@@ -25,7 +25,7 @@ import java.util.List;
 /**
  * Client for interacting with extension bundles.
  */
-public interface ExtensionBundleClient {
+public interface BundleClient {
 
     /**
      * Retrieves all extension bundles located in buckets the current user is authorized for.
@@ -35,7 +35,7 @@ public interface ExtensionBundleClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    List<ExtensionBundle> getAll() throws IOException, NiFiRegistryException;
+    List<Bundle> getAll() throws IOException, NiFiRegistryException;
 
     /**
      * Retrieves all extension bundles matching the specified filters, located in buckets the current user is authorized for.
@@ -46,7 +46,7 @@ public interface ExtensionBundleClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    List<ExtensionBundle> getAll(ExtensionBundleFilterParams filterParams) throws IOException, NiFiRegistryException;
+    List<Bundle> getAll(BundleFilterParams filterParams) throws IOException, NiFiRegistryException;
 
     /**
      * Retrieves the extension bundles located in the given bucket.
@@ -57,7 +57,7 @@ public interface ExtensionBundleClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    List<ExtensionBundle> getByBucket(String bucketId) throws IOException, NiFiRegistryException;
+    List<Bundle> getByBucket(String bucketId) throws IOException, NiFiRegistryException;
 
     /**
      * Retrieves the extension bundle with the given id.
@@ -68,7 +68,7 @@ public interface ExtensionBundleClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundle get(String bundleId) throws IOException, NiFiRegistryException;
+    Bundle get(String bundleId) throws IOException, NiFiRegistryException;
 
     /**
      * Deletes the extension bundle with the given id, and all of its versions.
@@ -79,6 +79,6 @@ public interface ExtensionBundleClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundle delete(String bundleId) throws IOException, NiFiRegistryException;
+    Bundle delete(String bundleId) throws IOException, NiFiRegistryException;
 
 }
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionBundleVersionClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/BundleVersionClient.java
similarity index 70%
rename from nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionBundleVersionClient.java
rename to nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/BundleVersionClient.java
index dd07df3..5a6ab97 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionBundleVersionClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/BundleVersionClient.java
@@ -16,10 +16,12 @@
  */
 package org.apache.nifi.registry.client;
 
-import org.apache.nifi.registry.extension.ExtensionBundleType;
-import org.apache.nifi.registry.extension.ExtensionBundleVersion;
-import org.apache.nifi.registry.extension.ExtensionBundleVersionMetadata;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleVersionFilterParams;
+import org.apache.nifi.registry.extension.bundle.BundleType;
+import org.apache.nifi.registry.extension.bundle.BundleVersion;
+import org.apache.nifi.registry.extension.bundle.BundleVersionFilterParams;
+import org.apache.nifi.registry.extension.bundle.BundleVersionMetadata;
+import org.apache.nifi.registry.extension.component.manifest.Extension;
+import org.apache.nifi.registry.extension.component.ExtensionMetadata;
 
 import java.io.File;
 import java.io.IOException;
@@ -29,7 +31,7 @@ import java.util.List;
 /**
  * Client for interacting with extension bundle versions.
  */
-public interface ExtensionBundleVersionClient {
+public interface BundleVersionClient {
 
     /**
      * Uploads a version of an extension bundle to NiFi Registry where the bundle content comes from an InputStream.
@@ -37,12 +39,12 @@ public interface ExtensionBundleVersionClient {
      * @param bucketId the bucket where the extension bundle will leave
      * @param bundleType the type of bundle being uploaded
      * @param bundleContentStream the input stream with the binary content of the bundle
-     * @return the ExtensionBundleVersion entity
+     * @return the BundleVersion entity
      *
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundleVersion create(String bucketId, ExtensionBundleType bundleType, InputStream bundleContentStream)
+    BundleVersion create(String bucketId, BundleType bundleType, InputStream bundleContentStream)
             throws IOException, NiFiRegistryException;
 
     /**
@@ -52,12 +54,12 @@ public interface ExtensionBundleVersionClient {
      * @param bundleType the type of bundle being uploaded
      * @param bundleContentStream the input stream with the binary content of the bundle
      * @param sha256 the optional SHA-256 in hex form
-     * @return the ExtensionBundleVersion entity
+     * @return the BundleVersion entity
      *
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundleVersion create(String bucketId, ExtensionBundleType bundleType, InputStream bundleContentStream, String sha256)
+    BundleVersion create(String bucketId, BundleType bundleType, InputStream bundleContentStream, String sha256)
             throws IOException, NiFiRegistryException;
 
     /**
@@ -66,12 +68,12 @@ public interface ExtensionBundleVersionClient {
      * @param bucketId the bucket where the extension bundle will leave
      * @param bundleType the type of bundle being uploaded
      * @param bundleFile the file with the binary content of the bundle
-     * @return the ExtensionBundleVersion entity
+     * @return the BundleVersion entity
      *
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundleVersion create(String bucketId, ExtensionBundleType bundleType, File bundleFile)
+    BundleVersion create(String bucketId, BundleType bundleType, File bundleFile)
             throws IOException, NiFiRegistryException;
 
     /**
@@ -81,12 +83,12 @@ public interface ExtensionBundleVersionClient {
      * @param bundleType the type of bundle being uploaded
      * @param bundleFile the file with the binary content of the bundle
      * @param sha256 the optional SHA-256 in hex form
-     * @return the ExtensionBundleVersion entity
+     * @return the BundleVersion entity
      *
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundleVersion create(String bucketId, ExtensionBundleType bundleType, File bundleFile, String sha256)
+    BundleVersion create(String bucketId, BundleType bundleType, File bundleFile, String sha256)
             throws IOException, NiFiRegistryException;
 
     /**
@@ -99,7 +101,7 @@ public interface ExtensionBundleVersionClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    List<ExtensionBundleVersionMetadata> getBundleVersions(ExtensionBundleVersionFilterParams filterParams)
+    List<BundleVersionMetadata> getBundleVersions(BundleVersionFilterParams filterParams)
             throws IOException, NiFiRegistryException;
 
     /**
@@ -111,7 +113,7 @@ public interface ExtensionBundleVersionClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    List<ExtensionBundleVersionMetadata> getBundleVersions(String bundleId) throws IOException, NiFiRegistryException;
+    List<BundleVersionMetadata> getBundleVersions(String bundleId) throws IOException, NiFiRegistryException;
 
     /**
      * Retrieves bundle version entity for the given bundle id and version string.
@@ -122,12 +124,37 @@ public interface ExtensionBundleVersionClient {
      *
      * @param bundleId the bundle id
      * @param version the bundle version
-     * @return the ExtensionBundleVersion entity
+     * @return the BundleVersion entity
      *
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundleVersion getBundleVersion(String bundleId, String version) throws IOException, NiFiRegistryException;
+    BundleVersion getBundleVersion(String bundleId, String version) throws IOException, NiFiRegistryException;
+
+    /**
+     * Retrieves the metadata about the extensions in the given bundle version.
+     *
+     * @param bundleId the bundle id
+     * @param version the bundle version
+     * @return the list of metadata about the extensions
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    List<ExtensionMetadata> getExtensions(String bundleId, String version) throws IOException, NiFiRegistryException;
+
+    /**
+     * Retrieves the full extension info for the extension with the given name in the given bundle version.
+     *
+     * @param bundleId the bundle id
+     * @param version the version of the bundle
+     * @param name the name of the extension
+     * @return the extension info
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    Extension getExtension(String bundleId, String version, String name) throws IOException, NiFiRegistryException;
 
     /**
      * Obtains an InputStream for the binary content for the version of the given bundle.
@@ -164,6 +191,6 @@ public interface ExtensionBundleVersionClient {
      * @throws IOException if an I/O error occurs
      * @throws NiFiRegistryException if an non I/O error occurs
      */
-    ExtensionBundleVersion delete(String bundleId, String version) throws IOException, NiFiRegistryException;
+    BundleVersion delete(String bundleId, String version) throws IOException, NiFiRegistryException;
 
 }
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionClient.java
new file mode 100644
index 0000000..b0e3e0e
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionClient.java
@@ -0,0 +1,62 @@
+/*
+ * 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.registry.client;
+
+import org.apache.nifi.registry.extension.component.ExtensionFilterParams;
+import org.apache.nifi.registry.extension.component.ExtensionMetadataContainer;
+import org.apache.nifi.registry.extension.component.TagCount;
+import org.apache.nifi.registry.extension.component.manifest.ProvidedServiceAPI;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Client for obtaining information about extensions.
+ */
+public interface ExtensionClient {
+
+    /**
+     * Retrieves extensions according to the given filter params.
+     *
+     * @param filterParams the filter params
+     * @return the metadata for the extensions matching the filter params
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    ExtensionMetadataContainer findExtensions(ExtensionFilterParams filterParams) throws IOException, NiFiRegistryException;
+
+    /**
+     * Retrieves extensions that provide the given service API.
+     *
+     * @param providedServiceAPI the service API
+     * @return the metadata for extensions that provided the service API
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    ExtensionMetadataContainer findExtensions(ProvidedServiceAPI providedServiceAPI) throws IOException, NiFiRegistryException;
+
+    /**
+     * @return all of the tags known the registry with their corresponding counts
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    List<TagCount> getTagCounts() throws IOException, NiFiRegistryException;
+
+}
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionRepoClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionRepoClient.java
index bcb1b30..3650d63 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionRepoClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/ExtensionRepoClient.java
@@ -16,12 +16,15 @@
  */
 package org.apache.nifi.registry.client;
 
+import org.apache.nifi.registry.extension.component.manifest.Extension;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoArtifact;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoBucket;
+import org.apache.nifi.registry.extension.repo.ExtensionRepoExtensionMetadata;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoGroup;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoVersion;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
@@ -95,6 +98,37 @@ public interface ExtensionRepoClient {
             throws IOException, NiFiRegistryException;
 
     /**
+     * Gets the metadata about the extensions for the given bucket, group, artifact, and version.
+     *
+     * @param bucketName the bucket name
+     * @param groupId the group id
+     * @param artifactId the artifact id
+     * @param version the version
+     * @return the list of extension metadata
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    List<ExtensionRepoExtensionMetadata> getVersionExtensions(String bucketName, String groupId, String artifactId, String version)
+            throws IOException, NiFiRegistryException;
+
+    /**
+     * Gets the metadata about the extension with the given name in the given bucket, group, artifact, and version.
+     *
+     * @param bucketName the bucket name
+     * @param groupId the group id
+     * @param artifactId the artifact id
+     * @param version the version
+     * @param extensionName the extension name
+     * @return the extension info
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    Extension getVersionExtension(String bucketName, String groupId, String artifactId, String version, String extensionName)
+            throws IOException, NiFiRegistryException;
+
+    /**
      * Gets an InputStream for the binary content of the specified version.
      *
      * @param bucketName the bucket name
@@ -110,6 +144,22 @@ public interface ExtensionRepoClient {
             throws IOException, NiFiRegistryException;
 
     /**
+     * Writes the binary content for the version of the given the bundle to the specified directory.
+     *
+     * @param bucketName the bucket name
+     * @param groupId the group id
+     * @param artifactId the artifact id
+     * @param version the version
+     * @param directory the directory to write to
+     * @return the File object for the bundle that was written
+     *
+     * @throws IOException if an I/O error occurs
+     * @throws NiFiRegistryException if an non I/O error occurs
+     */
+    File writeBundleVersionContent(String bucketName, String groupId, String artifactId, String version, File directory)
+            throws IOException, NiFiRegistryException;
+
+    /**
      * Gets the hex representation of the SHA-256 hash of the binary content for the given version.
      *
      * @param bucketName the bucket name
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
index c5c4df4..e141136 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
@@ -76,22 +76,22 @@ public interface NiFiRegistryClient extends Closeable {
     /**
      * @return the client for interacting with extension bundles
      */
-    ExtensionBundleClient getExtensionBundleClient();
+    BundleClient getBundleClient();
 
     /**
      * @return the client for interacting with extension bundles on behalf of the given proxied entities
      */
-    ExtensionBundleClient getExtensionBundleClient(String ... proxiedEntity);
+    BundleClient getBundleClient(String ... proxiedEntity);
 
     /**
      * @return the client for interacting with extension bundle versions
      */
-    ExtensionBundleVersionClient getExtensionBundleVersionClient();
+    BundleVersionClient getBundleVersionClient();
 
     /**
      * @return the client for interacting with extension bundle versions on behalf of the given proxied entities
      */
-    ExtensionBundleVersionClient getExtensionBundleVersionClient(String ... proxiedEntity);
+    BundleVersionClient getBundleVersionClient(String ... proxiedEntity);
 
     /**
      * @return the client for interacting with the extension repository
@@ -104,6 +104,16 @@ public interface NiFiRegistryClient extends Closeable {
     ExtensionRepoClient getExtensionRepoClient(String ... proxiedEntity);
 
     /**
+     * @return the client for interacting with extensions
+     */
+    ExtensionClient getExtensionClient();
+
+    /**
+     * @return the client for interacting with extensions on behalf of the given proxied entities
+     */
+    ExtensionClient getExtensionClient(String ... proxiedEntity);
+
+    /**
      * The builder interface that implementations should provide for obtaining the client.
      */
     interface Builder {
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java
index 3bd7359..efd7e61 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java
@@ -24,7 +24,7 @@ import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.bucket.BucketItemType;
-import org.apache.nifi.registry.extension.ExtensionBundle;
+import org.apache.nifi.registry.extension.bundle.Bundle;
 import org.apache.nifi.registry.flow.VersionedFlow;
 
 import java.io.IOException;
@@ -67,8 +67,8 @@ public class BucketItemDeserializer extends StdDeserializer<BucketItem[]> {
                     bucketItems.add(versionedFlow);
                     break;
                 case Extension_Bundle:
-                    final ExtensionBundle extensionBundle = jsonParser.getCodec().treeToValue(node, ExtensionBundle.class);
-                    bucketItems.add(extensionBundle);
+                    final Bundle bundle = jsonParser.getCodec().treeToValue(node, Bundle.class);
+                    bucketItems.add(bundle);
                     break;
                 default:
                     throw new IllegalStateException("Unknown type for BucketItem: " + bucketItemType);
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/ClientUtils.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/ClientUtils.java
new file mode 100644
index 0000000..98614fb
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/ClientUtils.java
@@ -0,0 +1,47 @@
+/*
+ * 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.registry.client.impl;
+
+import org.apache.commons.lang3.StringUtils;
+
+import javax.ws.rs.core.Response;
+import java.io.File;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+
+public class ClientUtils {
+
+    public static File getExtensionBundleVersionContent(final Response response, final File outputDirectory) {
+        final String contentDispositionHeader = response.getHeaderString("Content-Disposition");
+        if (StringUtils.isBlank(contentDispositionHeader)) {
+            throw new IllegalStateException("Content-Disposition header was blank or missing");
+        }
+
+        final int equalsIndex = contentDispositionHeader.lastIndexOf("=");
+        final String filename = contentDispositionHeader.substring(equalsIndex + 1).trim();
+        final File bundleFile = new File(outputDirectory, filename);
+
+        try (final InputStream responseInputStream = response.readEntity(InputStream.class)) {
+            Files.copy(responseInputStream, bundleFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            return bundleFile;
+        } catch (Exception e) {
+            throw new IllegalStateException("Unable to write bundle content due to: " + e.getMessage(), e);
+        }
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionBundleClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBundleClient.java
similarity index 68%
rename from nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionBundleClient.java
rename to nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBundleClient.java
index 9f748d5..5425531 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionBundleClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBundleClient.java
@@ -17,10 +17,10 @@
 package org.apache.nifi.registry.client.impl;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.client.ExtensionBundleClient;
+import org.apache.nifi.registry.client.BundleClient;
 import org.apache.nifi.registry.client.NiFiRegistryException;
-import org.apache.nifi.registry.extension.ExtensionBundle;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleFilterParams;
+import org.apache.nifi.registry.extension.bundle.Bundle;
+import org.apache.nifi.registry.extension.bundle.BundleFilterParams;
 
 import javax.ws.rs.client.WebTarget;
 import java.io.IOException;
@@ -30,34 +30,37 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Jersey implementation of ExtensionBundleClient.
+ * Jersey implementation of BundleClient.
  */
-public class JerseyExtensionBundleClient extends AbstractJerseyClient implements ExtensionBundleClient {
+public class JerseyBundleClient extends AbstractJerseyClient implements BundleClient {
 
     private final WebTarget bucketExtensionBundlesTarget;
     private final WebTarget extensionBundlesTarget;
 
-    public JerseyExtensionBundleClient(final WebTarget baseTarget) {
+    public JerseyBundleClient(final WebTarget baseTarget) {
         this(baseTarget, Collections.emptyMap());
     }
 
-    public JerseyExtensionBundleClient(final WebTarget baseTarget, final Map<String, String> headers) {
+    public JerseyBundleClient(final WebTarget baseTarget, final Map<String, String> headers) {
         super(headers);
-        this.bucketExtensionBundlesTarget = baseTarget.path("buckets/{bucketId}/extensions/bundles");
-        this.extensionBundlesTarget = baseTarget.path("extensions/bundles");
+        this.bucketExtensionBundlesTarget = baseTarget.path("buckets/{bucketId}/bundles");
+        this.extensionBundlesTarget = baseTarget.path("bundles");
     }
 
     @Override
-    public List<ExtensionBundle> getAll() throws IOException, NiFiRegistryException {
+    public List<Bundle> getAll() throws IOException, NiFiRegistryException {
         return getAll(null);
     }
 
     @Override
-    public List<ExtensionBundle> getAll(final ExtensionBundleFilterParams filterParams) throws IOException, NiFiRegistryException {
+    public List<Bundle> getAll(final BundleFilterParams filterParams) throws IOException, NiFiRegistryException {
         return executeAction("Error getting extension bundles", () -> {
             WebTarget target = extensionBundlesTarget;
 
             if (filterParams != null) {
+                if (!StringUtils.isBlank(filterParams.getBucketName())) {
+                    target = target.queryParam("bucketName", filterParams.getBucketName());
+                }
                 if (!StringUtils.isBlank(filterParams.getGroupId())) {
                     target = target.queryParam("groupId", filterParams.getGroupId());
                 }
@@ -66,13 +69,13 @@ public class JerseyExtensionBundleClient extends AbstractJerseyClient implements
                 }
             }
 
-            final ExtensionBundle[] bundles = getRequestBuilder(target).get(ExtensionBundle[].class);
+            final Bundle[] bundles = getRequestBuilder(target).get(Bundle[].class);
             return  bundles == null ? Collections.emptyList() : Arrays.asList(bundles);
         });
     }
 
     @Override
-    public List<ExtensionBundle> getByBucket(final String bucketId) throws IOException, NiFiRegistryException {
+    public List<Bundle> getByBucket(final String bucketId) throws IOException, NiFiRegistryException {
         if (StringUtils.isBlank(bucketId)) {
             throw new IllegalArgumentException("Bucket id cannot be null or blank");
         }
@@ -80,13 +83,13 @@ public class JerseyExtensionBundleClient extends AbstractJerseyClient implements
         return executeAction("Error getting extension bundles for bucket", () -> {
             WebTarget target = bucketExtensionBundlesTarget.resolveTemplate("bucketId", bucketId);
 
-            final ExtensionBundle[] bundles = getRequestBuilder(target).get(ExtensionBundle[].class);
+            final Bundle[] bundles = getRequestBuilder(target).get(Bundle[].class);
             return  bundles == null ? Collections.emptyList() : Arrays.asList(bundles);
         });
     }
 
     @Override
-    public ExtensionBundle get(final String bundleId) throws IOException, NiFiRegistryException {
+    public Bundle get(final String bundleId) throws IOException, NiFiRegistryException {
         if (StringUtils.isBlank(bundleId)) {
             throw new IllegalArgumentException("Bundle id cannot be null or blank");
         }
@@ -96,12 +99,12 @@ public class JerseyExtensionBundleClient extends AbstractJerseyClient implements
                     .path("{bundleId}")
                     .resolveTemplate("bundleId", bundleId);
 
-            return getRequestBuilder(target).get(ExtensionBundle.class);
+            return getRequestBuilder(target).get(Bundle.class);
         });
     }
 
     @Override
-    public ExtensionBundle delete(final String bundleId) throws IOException, NiFiRegistryException {
+    public Bundle delete(final String bundleId) throws IOException, NiFiRegistryException {
         if (StringUtils.isBlank(bundleId)) {
             throw new IllegalArgumentException("Bundle id cannot be null or blank");
         }
@@ -111,7 +114,7 @@ public class JerseyExtensionBundleClient extends AbstractJerseyClient implements
                     .path("{bundleId}")
                     .resolveTemplate("bundleId", bundleId);
 
-            return getRequestBuilder(target).delete(ExtensionBundle.class);
+            return getRequestBuilder(target).delete(Bundle.class);
         });
     }
 
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionBundleVersionClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBundleVersionClient.java
similarity index 68%
rename from nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionBundleVersionClient.java
rename to nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBundleVersionClient.java
index 0332412..32b8bb2 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionBundleVersionClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBundleVersionClient.java
@@ -17,12 +17,14 @@
 package org.apache.nifi.registry.client.impl;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.client.ExtensionBundleVersionClient;
+import org.apache.nifi.registry.client.BundleVersionClient;
 import org.apache.nifi.registry.client.NiFiRegistryException;
-import org.apache.nifi.registry.extension.ExtensionBundleType;
-import org.apache.nifi.registry.extension.ExtensionBundleVersion;
-import org.apache.nifi.registry.extension.ExtensionBundleVersionMetadata;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleVersionFilterParams;
+import org.apache.nifi.registry.extension.bundle.BundleType;
+import org.apache.nifi.registry.extension.bundle.BundleVersion;
+import org.apache.nifi.registry.extension.bundle.BundleVersionFilterParams;
+import org.apache.nifi.registry.extension.bundle.BundleVersionMetadata;
+import org.apache.nifi.registry.extension.component.manifest.Extension;
+import org.apache.nifi.registry.extension.component.ExtensionMetadata;
 import org.glassfish.jersey.media.multipart.FormDataMultiPart;
 import org.glassfish.jersey.media.multipart.file.FileDataBodyPart;
 import org.glassfish.jersey.media.multipart.file.StreamDataBodyPart;
@@ -34,39 +36,37 @@ import javax.ws.rs.core.Response;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
 /**
- * Jersey implementation of ExtensionBundleVersionClient.
+ * Jersey implementation of BundleVersionClient.
  */
-public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient implements ExtensionBundleVersionClient {
+public class JerseyBundleVersionClient extends AbstractJerseyClient implements BundleVersionClient {
 
     private final WebTarget bucketExtensionBundlesTarget;
     private final WebTarget extensionBundlesTarget;
 
-    public JerseyExtensionBundleVersionClient(final WebTarget baseTarget) {
+    public JerseyBundleVersionClient(final WebTarget baseTarget) {
         this(baseTarget, Collections.emptyMap());
     }
 
-    public JerseyExtensionBundleVersionClient(final WebTarget baseTarget, final Map<String, String> headers) {
+    public JerseyBundleVersionClient(final WebTarget baseTarget, final Map<String, String> headers) {
         super(headers);
-        this.bucketExtensionBundlesTarget = baseTarget.path("buckets/{bucketId}/extensions/bundles");
-        this.extensionBundlesTarget = baseTarget.path("extensions/bundles");
+        this.bucketExtensionBundlesTarget = baseTarget.path("buckets/{bucketId}/bundles");
+        this.extensionBundlesTarget = baseTarget.path("bundles");
     }
 
     @Override
-    public ExtensionBundleVersion create(final String bucketId, final ExtensionBundleType bundleType, final InputStream bundleContentStream)
+    public BundleVersion create(final String bucketId, final BundleType bundleType, final InputStream bundleContentStream)
             throws IOException, NiFiRegistryException {
         return create(bucketId, bundleType, bundleContentStream, null);
     }
 
         @Override
-    public ExtensionBundleVersion create(final String bucketId, final ExtensionBundleType bundleType, final InputStream bundleContentStream, final String sha256)
+    public BundleVersion create(final String bucketId, final BundleType bundleType, final InputStream bundleContentStream, final String sha256)
             throws IOException, NiFiRegistryException {
 
         if (StringUtils.isBlank(bucketId)) {
@@ -99,19 +99,19 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
             return getRequestBuilder(target)
                     .post(
                             Entity.entity(multipart, multipart.getMediaType()),
-                            ExtensionBundleVersion.class
+                            BundleVersion.class
                     );
         });
     }
 
     @Override
-    public ExtensionBundleVersion create(final String bucketId, final ExtensionBundleType bundleType, final File bundleFile)
+    public BundleVersion create(final String bucketId, final BundleType bundleType, final File bundleFile)
             throws IOException, NiFiRegistryException {
         return create(bucketId, bundleType, bundleFile, null);
     }
 
     @Override
-    public ExtensionBundleVersion create(final String bucketId, final ExtensionBundleType bundleType, final File bundleFile, final String sha256)
+    public BundleVersion create(final String bucketId, final BundleType bundleType, final File bundleFile, final String sha256)
             throws IOException, NiFiRegistryException {
 
         if (StringUtils.isBlank(bucketId)) {
@@ -144,13 +144,13 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
             return getRequestBuilder(target)
                     .post(
                             Entity.entity(multipart, multipart.getMediaType()),
-                            ExtensionBundleVersion.class
+                            BundleVersion.class
                     );
         });
     }
 
     @Override
-    public List<ExtensionBundleVersionMetadata> getBundleVersions(final ExtensionBundleVersionFilterParams filterParams)
+    public List<BundleVersionMetadata> getBundleVersions(final BundleVersionFilterParams filterParams)
             throws IOException, NiFiRegistryException {
 
         return executeAction("Error getting extension bundle versions", () -> {
@@ -170,13 +170,13 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
                 }
             }
 
-            final ExtensionBundleVersionMetadata[] bundleVersions = getRequestBuilder(target).get(ExtensionBundleVersionMetadata[].class);
+            final BundleVersionMetadata[] bundleVersions = getRequestBuilder(target).get(BundleVersionMetadata[].class);
             return  bundleVersions == null ? Collections.emptyList() : Arrays.asList(bundleVersions);
         });
     }
 
     @Override
-    public List<ExtensionBundleVersionMetadata> getBundleVersions(final String bundleId)
+    public List<BundleVersionMetadata> getBundleVersions(final String bundleId)
             throws IOException, NiFiRegistryException {
 
         if (StringUtils.isBlank(bundleId)) {
@@ -188,13 +188,13 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
                     .path("{bundleId}/versions")
                     .resolveTemplate("bundleId", bundleId);
 
-            final ExtensionBundleVersionMetadata[] bundleVersions = getRequestBuilder(target).get(ExtensionBundleVersionMetadata[].class);
+            final BundleVersionMetadata[] bundleVersions = getRequestBuilder(target).get(BundleVersionMetadata[].class);
             return  bundleVersions == null ? Collections.emptyList() : Arrays.asList(bundleVersions);
         });
     }
 
     @Override
-    public ExtensionBundleVersion getBundleVersion(final String bundleId, final String version)
+    public BundleVersion getBundleVersion(final String bundleId, final String version)
             throws IOException, NiFiRegistryException {
 
         if (StringUtils.isBlank(bundleId)) {
@@ -211,11 +211,60 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
                     .resolveTemplate("bundleId", bundleId)
                     .resolveTemplate("version", version);
 
-            return getRequestBuilder(target).get(ExtensionBundleVersion.class);
+            return getRequestBuilder(target).get(BundleVersion.class);
          });
     }
 
     @Override
+    public List<ExtensionMetadata> getExtensions(final String bundleId, final String version)
+            throws IOException, NiFiRegistryException {
+
+        if (StringUtils.isBlank(bundleId)) {
+            throw new IllegalArgumentException("Bundle id cannot be null or blank");
+        }
+
+        if (StringUtils.isBlank(version)) {
+            throw new IllegalArgumentException("Version cannot be null or blank");
+        }
+
+        return executeAction("Error getting extension bundle metadata", () -> {
+            final WebTarget target = extensionBundlesTarget
+                    .path("{bundleId}/versions/{version}/extensions")
+                    .resolveTemplate("bundleId", bundleId)
+                    .resolveTemplate("version", version);
+
+            final ExtensionMetadata[] extensions = getRequestBuilder(target).get(ExtensionMetadata[].class);
+            return  extensions == null ? Collections.emptyList() : Arrays.asList(extensions);
+        });
+    }
+
+    @Override
+    public Extension getExtension(final String bundleId, final String version, final String name) throws IOException, NiFiRegistryException {
+        if (StringUtils.isBlank(bundleId)) {
+            throw new IllegalArgumentException("Bundle id cannot be null or blank");
+        }
+
+        if (StringUtils.isBlank(version)) {
+            throw new IllegalArgumentException("Version cannot be null or blank");
+        }
+
+        if (StringUtils.isBlank(name)) {
+            throw new IllegalArgumentException("Extension name cannot be null or blank");
+        }
+
+        return executeAction("Error getting extension", () -> {
+            final WebTarget target = extensionBundlesTarget
+                    .path("{bundleId}/versions/{version}/extensions/{name}")
+                    .resolveTemplate("bundleId", bundleId)
+                    .resolveTemplate("version", version)
+                    .resolveTemplate("name", name);
+
+            final Extension extension = getRequestBuilder(target).get(Extension.class);
+            return  extension;
+        });
+    }
+
+    @Override
     public InputStream getBundleVersionContent(final String bundleId, final String version)
             throws IOException, NiFiRegistryException {
 
@@ -266,26 +315,12 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
                     .accept(MediaType.APPLICATION_OCTET_STREAM_TYPE)
                     .get();
 
-            final String contentDispositionHeader = response.getHeaderString("Content-Disposition");
-            if (StringUtils.isBlank(contentDispositionHeader)) {
-                throw new IllegalStateException("Content-Disposition header was blank or missing");
-            }
-
-            final int equalsIndex = contentDispositionHeader.lastIndexOf("=");
-            final String filename = contentDispositionHeader.substring(equalsIndex + 1).trim();
-            final File bundleFile = new File(directory, filename);
-
-            try (final InputStream responseInputStream = response.readEntity(InputStream.class)) {
-                Files.copy(responseInputStream, bundleFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
-                return bundleFile;
-            } catch (Exception e) {
-                throw new IllegalStateException("Unable to write bundle content due to: " + e.getMessage(), e);
-            }
+            return ClientUtils.getExtensionBundleVersionContent(response, directory);
         });
     }
 
     @Override
-    public ExtensionBundleVersion delete(final String bundleId, final String version) throws IOException, NiFiRegistryException {
+    public BundleVersion delete(final String bundleId, final String version) throws IOException, NiFiRegistryException {
         if (StringUtils.isBlank(bundleId)) {
             throw new IllegalArgumentException("Bundle id cannot be null or blank");
         }
@@ -300,7 +335,7 @@ public class JerseyExtensionBundleVersionClient extends AbstractJerseyClient imp
                     .resolveTemplate("bundleId", bundleId)
                     .resolveTemplate("version", version);
 
-            return getRequestBuilder(target).delete(ExtensionBundleVersion.class);
+            return getRequestBuilder(target).delete(BundleVersion.class);
         });
     }
 
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionClient.java
new file mode 100644
index 0000000..b885959
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionClient.java
@@ -0,0 +1,111 @@
+/*
+ * 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.registry.client.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.client.ExtensionClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.extension.bundle.BundleType;
+import org.apache.nifi.registry.extension.component.ExtensionFilterParams;
+import org.apache.nifi.registry.extension.component.ExtensionMetadataContainer;
+import org.apache.nifi.registry.extension.component.TagCount;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionType;
+import org.apache.nifi.registry.extension.component.manifest.ProvidedServiceAPI;
+
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class JerseyExtensionClient extends AbstractJerseyClient implements ExtensionClient {
+
+    private final WebTarget extensionsTarget;
+
+    public JerseyExtensionClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyExtensionClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(headers);
+        this.extensionsTarget = baseTarget.path("extensions");
+    }
+
+    @Override
+    public ExtensionMetadataContainer findExtensions(final ExtensionFilterParams filterParams)
+            throws IOException, NiFiRegistryException {
+
+        return executeAction("Error retrieving extensions", () -> {
+            WebTarget target = extensionsTarget;
+
+            if (filterParams != null) {
+                final BundleType bundleType = filterParams.getBundleType();
+                if (bundleType != null) {
+                    target = target.queryParam("bundleType", bundleType.name());
+                }
+
+                final ExtensionType extensionType = filterParams.getExtensionType();
+                if (extensionType != null) {
+                    target = target.queryParam("extensionType", extensionType.toString());
+                }
+
+                final Set<String> tags = filterParams.getTags();
+                if (tags != null) {
+                    for (final String tag : tags) {
+                        target = target.queryParam("tag", tag);
+                    }
+                }
+            }
+
+            return getRequestBuilder(target).get(ExtensionMetadataContainer.class);
+        });
+    }
+
+    @Override
+    public ExtensionMetadataContainer findExtensions(final ProvidedServiceAPI serviceAPI) throws IOException, NiFiRegistryException {
+        if (serviceAPI == null
+                || StringUtils.isBlank(serviceAPI.getClassName())
+                || StringUtils.isBlank(serviceAPI.getGroupId())
+                || StringUtils.isBlank(serviceAPI.getArtifactId())
+                || StringUtils.isBlank(serviceAPI.getVersion())) {
+            throw new IllegalArgumentException("Provided service API must be specified with a class, group, artifact, and version");
+        }
+
+        return executeAction("Error retrieving extensions", () -> {
+            WebTarget target = extensionsTarget.path("provided-service-api");
+            target = target.queryParam("className", serviceAPI.getClassName());
+            target = target.queryParam("groupId", serviceAPI.getGroupId());
+            target = target.queryParam("artifactId", serviceAPI.getArtifactId());
+            target = target.queryParam("version", serviceAPI.getVersion());
+
+            return getRequestBuilder(target).get(ExtensionMetadataContainer.class);
+        });
+    }
+
+    @Override
+    public List<TagCount> getTagCounts() throws IOException, NiFiRegistryException {
+        return executeAction("Error retrieving tag counts", () -> {
+           final WebTarget target = extensionsTarget.path("tags");
+
+           final TagCount[] tagCounts = getRequestBuilder(target).get(TagCount[].class);
+           return tagCounts == null ? Collections.emptyList() : Arrays.asList(tagCounts);
+        });
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionRepoClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionRepoClient.java
index dc67b97..1235b03 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionRepoClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyExtensionRepoClient.java
@@ -19,8 +19,10 @@ package org.apache.nifi.registry.client.impl;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.client.ExtensionRepoClient;
 import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.extension.component.manifest.Extension;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoArtifact;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoBucket;
+import org.apache.nifi.registry.extension.repo.ExtensionRepoExtensionMetadata;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoGroup;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoVersion;
 import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
@@ -28,6 +30,8 @@ import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
 import javax.ws.rs.NotFoundException;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
@@ -46,7 +50,7 @@ public class JerseyExtensionRepoClient extends AbstractJerseyClient implements E
 
     public JerseyExtensionRepoClient(final WebTarget baseTarget, final Map<String, String> headers) {
         super(headers);
-        this.extensionRepoTarget = baseTarget.path("extensions/repo");
+        this.extensionRepoTarget = baseTarget.path("extension-repository");
     }
 
     @Override
@@ -143,6 +147,50 @@ public class JerseyExtensionRepoClient extends AbstractJerseyClient implements E
     }
 
     @Override
+    public List<ExtensionRepoExtensionMetadata> getVersionExtensions(final String bucketName, final String groupId, final String artifactId, final String version)
+            throws IOException, NiFiRegistryException {
+
+        validate(bucketName, groupId, artifactId, version);
+
+        return executeAction("Error retrieving versions for extension repo", () -> {
+            final WebTarget target = extensionRepoTarget
+                    .path("{bucketName}/{groupId}/{artifactId}/{version}/extensions")
+                    .resolveTemplate("bucketName", bucketName)
+                    .resolveTemplate("groupId", groupId)
+                    .resolveTemplate("artifactId", artifactId)
+                    .resolveTemplate("version", version);
+
+            final ExtensionRepoExtensionMetadata[] extensions = getRequestBuilder(target).get(ExtensionRepoExtensionMetadata[].class);
+            return  extensions == null ? Collections.emptyList() : Arrays.asList(extensions);
+        });
+    }
+
+    @Override
+    public Extension getVersionExtension(final String bucketName, final String groupId, final String artifactId,
+                                         final String version, final String extensionName)
+            throws IOException, NiFiRegistryException {
+
+        validate(bucketName, groupId, artifactId, version);
+
+        if (StringUtils.isBlank(extensionName)) {
+            throw new IllegalArgumentException("Extension name is required");
+        }
+
+        return executeAction("Error retrieving versions for extension repo", () -> {
+            final WebTarget target = extensionRepoTarget
+                    .path("{bucketName}/{groupId}/{artifactId}/{version}/extensions/{extensionName}")
+                    .resolveTemplate("bucketName", bucketName)
+                    .resolveTemplate("groupId", groupId)
+                    .resolveTemplate("artifactId", artifactId)
+                    .resolveTemplate("version", version)
+                    .resolveTemplate("extensionName", extensionName);
+
+            final Extension extension = getRequestBuilder(target).get(Extension.class);
+            return  extension;
+        });
+    }
+
+    @Override
     public InputStream getVersionContent(final String bucketName, final String groupId, final String artifactId, final String version)
             throws IOException, NiFiRegistryException {
 
@@ -164,6 +212,33 @@ public class JerseyExtensionRepoClient extends AbstractJerseyClient implements E
     }
 
     @Override
+    public File writeBundleVersionContent(final String bucketName, final String groupId, final String artifactId, final String version, final File directory)
+            throws IOException, NiFiRegistryException {
+
+        validate(bucketName, groupId, artifactId, version);
+
+        if (directory == null) {
+            throw new IllegalArgumentException("Directory cannot be null");
+        }
+
+        return executeAction("Error retrieving version content for extension repo", () -> {
+            final WebTarget target = extensionRepoTarget
+                    .path("{bucketName}/{groupId}/{artifactId}/{version}/content")
+                    .resolveTemplate("bucketName", bucketName)
+                    .resolveTemplate("groupId", groupId)
+                    .resolveTemplate("artifactId", artifactId)
+                    .resolveTemplate("version", version);
+
+            final Response response = getRequestBuilder(target)
+                    .accept(MediaType.APPLICATION_OCTET_STREAM_TYPE)
+                    .get();
+
+            return ClientUtils.getExtensionBundleVersionContent(response, directory);
+        });
+
+    }
+
+    @Override
     public String getVersionSha256(final String bucketName, final String groupId, final String artifactId, final String version)
             throws IOException, NiFiRegistryException {
 
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
index 972211b..739d225 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
@@ -24,8 +24,9 @@ import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.client.BucketClient;
-import org.apache.nifi.registry.client.ExtensionBundleClient;
-import org.apache.nifi.registry.client.ExtensionBundleVersionClient;
+import org.apache.nifi.registry.client.BundleClient;
+import org.apache.nifi.registry.client.BundleVersionClient;
+import org.apache.nifi.registry.client.ExtensionClient;
 import org.apache.nifi.registry.client.ExtensionRepoClient;
 import org.apache.nifi.registry.client.FlowClient;
 import org.apache.nifi.registry.client.FlowSnapshotClient;
@@ -183,25 +184,25 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
     }
 
     @Override
-    public ExtensionBundleClient getExtensionBundleClient() {
-        return new JerseyExtensionBundleClient(baseTarget);
+    public BundleClient getBundleClient() {
+        return new JerseyBundleClient(baseTarget);
     }
 
     @Override
-    public ExtensionBundleClient getExtensionBundleClient(String... proxiedEntity) {
+    public BundleClient getBundleClient(String... proxiedEntity) {
         final Map<String,String> headers = getHeaders(proxiedEntity);
-        return new JerseyExtensionBundleClient(baseTarget, headers);
+        return new JerseyBundleClient(baseTarget, headers);
     }
 
     @Override
-    public ExtensionBundleVersionClient getExtensionBundleVersionClient() {
-        return new JerseyExtensionBundleVersionClient(baseTarget);
+    public BundleVersionClient getBundleVersionClient() {
+        return new JerseyBundleVersionClient(baseTarget);
     }
 
     @Override
-    public ExtensionBundleVersionClient getExtensionBundleVersionClient(String... proxiedEntity) {
+    public BundleVersionClient getBundleVersionClient(String... proxiedEntity) {
         final Map<String,String> headers = getHeaders(proxiedEntity);
-        return new JerseyExtensionBundleVersionClient(baseTarget, headers);
+        return new JerseyBundleVersionClient(baseTarget, headers);
     }
 
     @Override
@@ -210,6 +211,17 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
     }
 
     @Override
+    public ExtensionClient getExtensionClient() {
+        return new JerseyExtensionClient(baseTarget);
+    }
+
+    @Override
+    public ExtensionClient getExtensionClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyExtensionClient(baseTarget, headers);
+    }
+
+    @Override
     public ExtensionRepoClient getExtensionRepoClient(String... proxiedEntity) {
         final Map<String,String> headers = getHeaders(proxiedEntity);
         return new JerseyExtensionRepoClient(baseTarget, headers);
diff --git a/nifi-registry-core/nifi-registry-data-model/pom.xml b/nifi-registry-core/nifi-registry-data-model/pom.xml
index d67bb41..9ed0305 100644
--- a/nifi-registry-core/nifi-registry-data-model/pom.xml
+++ b/nifi-registry-core/nifi-registry-data-model/pom.xml
@@ -33,9 +33,5 @@
             <groupId>javax.ws.rs</groupId>
             <artifactId>javax.ws.rs-api</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BuildInfo.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BuildInfo.java
new file mode 100644
index 0000000..deb5944
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BuildInfo.java
@@ -0,0 +1,102 @@
+/*
+ * 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.registry.extension.bundle;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+@ApiModel
+public class BuildInfo {
+
+    private String buildTool;
+
+    private String buildFlags;
+
+    private String buildBranch;
+
+    private String buildTag;
+
+    private String buildRevision;
+
+    private long built;
+
+    private String builtBy;
+
+    @ApiModelProperty(value = "The tool used to build the version of the bundle")
+    public String getBuildTool() {
+        return buildTool;
+    }
+
+    public void setBuildTool(String buildTool) {
+        this.buildTool = buildTool;
+    }
+
+    @ApiModelProperty(value = "The flags used to build the version of the bundle")
+    public String getBuildFlags() {
+        return buildFlags;
+    }
+
+    public void setBuildFlags(String buildFlags) {
+        this.buildFlags = buildFlags;
+    }
+
+    @ApiModelProperty(value = "The branch used to build the version of the bundle")
+    public String getBuildBranch() {
+        return buildBranch;
+    }
+
+    public void setBuildBranch(String buildBranch) {
+        this.buildBranch = buildBranch;
+    }
+
+    @ApiModelProperty(value = "The tag used to build the version of the bundle")
+    public String getBuildTag() {
+        return buildTag;
+    }
+
+    public void setBuildTag(String buildTag) {
+        this.buildTag = buildTag;
+    }
+
+    @ApiModelProperty(value = "The revision used to build the version of the bundle")
+    public String getBuildRevision() {
+        return buildRevision;
+    }
+
+    public void setBuildRevision(String buildRevision) {
+        this.buildRevision = buildRevision;
+    }
+
+    @ApiModelProperty(value = "The timestamp the version of the bundle was built")
+    public long getBuilt() {
+        return built;
+    }
+
+    public void setBuilt(long built) {
+        this.built = built;
+    }
+
+    @ApiModelProperty(value = "The identity of the user that performed the build")
+    public String getBuiltBy() {
+        return builtBy;
+    }
+
+    public void setBuiltBy(String builtBy) {
+        this.builtBy = builtBy;
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundle.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/Bundle.java
similarity index 84%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundle.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/Bundle.java
index c9f0e7f..63216fa 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundle.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/Bundle.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
@@ -29,16 +29,16 @@ import javax.xml.bind.annotation.XmlRootElement;
 /**
  * Represents an extension bundle identified by a group and artifact id with in a bucket.
  *
- * Each bundle may then have one or more versions associated with it by creating an {@link ExtensionBundleVersion}.
+ * Each bundle may then have one or more versions associated with it by creating an {@link BundleVersion}.
  *
- * The {@link ExtensionBundleVersion} represents the actually binary bundle which may contain one or more extensions.
+ * The {@link BundleVersion} represents the actually binary bundle which may contain one or more extensions.
  */
 @ApiModel
 @XmlRootElement
-public class ExtensionBundle extends BucketItem {
+public class Bundle extends BucketItem {
 
     @NotNull
-    private ExtensionBundleType bundleType;
+    private BundleType bundleType;
 
     @NotBlank
     private String groupId;
@@ -49,16 +49,16 @@ public class ExtensionBundle extends BucketItem {
     @Min(0)
     private long versionCount;
 
-    public ExtensionBundle() {
+    public Bundle() {
         super(BucketItemType.Extension_Bundle);
     }
 
     @ApiModelProperty(value = "The type of the extension bundle")
-    public ExtensionBundleType getBundleType() {
+    public BundleType getBundleType() {
         return bundleType;
     }
 
-    public void setBundleType(ExtensionBundleType bundleType) {
+    public void setBundleType(BundleType bundleType) {
         this.bundleType = bundleType;
     }
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/filter/ExtensionBundleFilterParams.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleFilterParams.java
similarity index 57%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/filter/ExtensionBundleFilterParams.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleFilterParams.java
index eb4490f..68c6001 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/filter/ExtensionBundleFilterParams.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleFilterParams.java
@@ -14,23 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension.filter;
+package org.apache.nifi.registry.extension.bundle;
 
 /**
  * Filter parameters for retrieving extension bundles.
+ *
+ * Any combination of fields may be provided to filter on the provided values.
+ *
+ * Note: This class is currently not part of the REST API so it doesn't not have the Swagger annotations, but it is used
+ * in the service layer and client to pass around params.
+ *
  */
-public class ExtensionBundleFilterParams {
+public class BundleFilterParams {
 
-    private static final ExtensionBundleFilterParams EMPTY_PARAMS = new Builder().build();
+    private static final BundleFilterParams EMPTY_PARAMS = new Builder().build();
 
+    private final String bucketName;
     private final String groupId;
     private final String artifactId;
 
-    private ExtensionBundleFilterParams(final Builder builder) {
+    private BundleFilterParams(final Builder builder) {
+        this.bucketName = builder.bucketName;
         this.groupId = builder.groupId;
         this.artifactId = builder.artifactId;
     }
 
+    public String getBucketName() {
+        return bucketName;
+    }
+
     public String getGroupId() {
         return groupId;
     }
@@ -39,19 +51,29 @@ public class ExtensionBundleFilterParams {
         return artifactId;
     }
 
-    public static ExtensionBundleFilterParams of(final String groupId, final String artifactId) {
+    public static BundleFilterParams of(final String bucketName, final String groupId, final String artifactId) {
+        return new Builder().bucket(bucketName).group(groupId).artifact(artifactId).build();
+    }
+
+    public static BundleFilterParams of(final String groupId, final String artifactId) {
         return new Builder().group(groupId).artifact(artifactId).build();
     }
 
-    public static ExtensionBundleFilterParams empty() {
+    public static BundleFilterParams empty() {
         return EMPTY_PARAMS;
     }
 
     public static class Builder {
 
+        private String bucketName;
         private String groupId;
         private String artifactId;
 
+        public Builder bucket(final String bucketName) {
+            this.bucketName = bucketName;
+            return this;
+        }
+
         public Builder group(final String groupId) {
             this.groupId = groupId;
             return this;
@@ -62,8 +84,8 @@ public class ExtensionBundleFilterParams {
             return this;
         }
 
-        public ExtensionBundleFilterParams build() {
-            return new ExtensionBundleFilterParams(this);
+        public BundleFilterParams build() {
+            return new BundleFilterParams(this);
         }
     }
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleInfo.java
similarity index 52%
copy from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleInfo.java
index f84649b..324b944 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleInfo.java
@@ -14,27 +14,61 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
-import javax.validation.constraints.NotBlank;
-import java.util.Objects;
-
 @ApiModel
-public class ExtensionBundleVersionDependency {
+public class BundleInfo {
 
-    @NotBlank
-    private String groupId;
+    private String bucketId;
+    private String bucketName;
 
-    @NotBlank
-    private String artifactId;
+    private String bundleId;
+    private BundleType bundleType;
 
-    @NotBlank
+    private String groupId;
+    private String artifactId;
     private String version;
 
-    @ApiModelProperty(value = "The group id of the bundle dependency")
+    @ApiModelProperty(value = "The id of the bucket where the bundle is located")
+    public String getBucketId() {
+        return bucketId;
+    }
+
+    public void setBucketId(String bucketId) {
+        this.bucketId = bucketId;
+    }
+
+    @ApiModelProperty(value = "The name of the bucket where the bundle is located")
+    public String getBucketName() {
+        return bucketName;
+    }
+
+    public void setBucketName(String bucketName) {
+        this.bucketName = bucketName;
+    }
+
+    @ApiModelProperty(value = "The id of the bundle")
+    public String getBundleId() {
+        return bundleId;
+    }
+
+    public void setBundleId(String bundleId) {
+        this.bundleId = bundleId;
+    }
+
+    @ApiModelProperty("The type of bundle (i.e. a NiFi NAR vs MiNiFi CPP)")
+    public BundleType getBundleType() {
+        return bundleType;
+    }
+
+    public void setBundleType(BundleType bundleType) {
+        this.bundleType = bundleType;
+    }
+
+    @ApiModelProperty(value = "The group id of the bundle")
     public String getGroupId() {
         return groupId;
     }
@@ -43,7 +77,7 @@ public class ExtensionBundleVersionDependency {
         this.groupId = groupId;
     }
 
-    @ApiModelProperty(value = "The artifact id of the bundle dependency")
+    @ApiModelProperty(value = "The artifact id of the bundle")
     public String getArtifactId() {
         return artifactId;
     }
@@ -52,7 +86,7 @@ public class ExtensionBundleVersionDependency {
         this.artifactId = artifactId;
     }
 
-    @ApiModelProperty(value = "The version of the bundle dependency")
+    @ApiModelProperty(value = "The version of the bundle")
     public String getVersion() {
         return version;
     }
@@ -60,25 +94,4 @@ public class ExtensionBundleVersionDependency {
     public void setVersion(String version) {
         this.version = version;
     }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(groupId, artifactId, version);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final ExtensionBundleVersionDependency other = (ExtensionBundleVersionDependency) obj;
-
-        return Objects.equals(groupId, other.groupId)
-                && Objects.equals(artifactId, other.artifactId)
-                && Objects.equals(version, other.version);
-    }
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleType.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleType.java
similarity index 74%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleType.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleType.java
index 0eb0447..92c54a6 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleType.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleType.java
@@ -14,39 +14,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
+
+import io.swagger.annotations.ApiModel;
 
 import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 
 /**
  * The possible types of extension bundles.
  */
-@XmlJavaTypeAdapter(ExtensionBundleTypeAdapter.class)
-public enum ExtensionBundleType {
+@ApiModel
+@XmlJavaTypeAdapter(BundleTypeAdapter.class)
+public enum BundleType {
 
-    NIFI_NAR("nifi-nar"),
+    NIFI_NAR(BundleTypeValues.NIFI_NAR_VALUE),
 
-    MINIFI_CPP("minifi-cpp");
+    MINIFI_CPP(BundleTypeValues.MINIFI_CPP_VALUE);
 
     private final String displayName;
 
-    ExtensionBundleType(String displayName) {
+    BundleType(String displayName) {
         this.displayName = displayName;
     }
 
     // Note: This method must be name fromString for JAX-RS/Jersey to use it on query and path params
-    public static ExtensionBundleType fromString(String value) {
+    public static BundleType fromString(String value) {
         if (value == null) {
             throw new IllegalArgumentException("Value cannot be null");
         }
 
-        for (final ExtensionBundleType type : values()) {
+        for (final BundleType type : values()) {
             if (type.toString().equals(value)) {
                 return type;
             }
         }
 
-        throw new IllegalArgumentException("Unknown ExtensionBundleType: " + value);
+        throw new IllegalArgumentException("Unknown BundleType: " + value);
     }
 
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleTypeAdapter.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleTypeAdapter.java
similarity index 76%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleTypeAdapter.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleTypeAdapter.java
index 1a993cf..4a54caa 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleTypeAdapter.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleTypeAdapter.java
@@ -14,23 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
 
 import javax.xml.bind.annotation.adapters.XmlAdapter;
 
-public class ExtensionBundleTypeAdapter extends XmlAdapter<String,ExtensionBundleType> {
+public class BundleTypeAdapter extends XmlAdapter<String, BundleType> {
 
     @Override
-    public ExtensionBundleType unmarshal(String v) throws Exception {
+    public BundleType unmarshal(String v) throws Exception {
         if (v == null) {
             return null;
         }
 
-        return ExtensionBundleType.fromString(v);
+        return BundleType.fromString(v);
     }
 
     @Override
-    public String marshal(final ExtensionBundleType v) throws Exception {
+    public String marshal(final BundleType v) throws Exception {
         if (v == null) {
             return null;
         }
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleTypeValues.java
similarity index 73%
copy from nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleTypeValues.java
index b072b4a..6933882 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleTypeValues.java
@@ -14,14 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.db.entity;
+package org.apache.nifi.registry.extension.bundle;
 
-public enum ExtensionEntityCategory {
+public class BundleTypeValues {
 
-    PROCESSOR,
-
-    CONTROLLER_SERVICE,
-
-    REPORTING_TASK;
+    public static final String NIFI_NAR_VALUE = "nifi-nar";
+    public static final String MINIFI_CPP_VALUE = "minifi-cpp";
 
+    public static final String ALL_VALUES = NIFI_NAR_VALUE + ", " + MINIFI_CPP_VALUE;
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersion.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersion.java
similarity index 72%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersion.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersion.java
index a8ef0c3..852f841 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersion.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersion.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
@@ -29,46 +29,47 @@ import java.util.Set;
 
 @ApiModel
 @XmlRootElement
-public class ExtensionBundleVersion extends LinkableEntity {
+public class BundleVersion extends LinkableEntity {
 
     @Valid
     @NotNull
-    private ExtensionBundleVersionMetadata versionMetadata;
+    private BundleVersionMetadata versionMetadata;
 
     // read-only, only populated from retrieval of an individual bundle version
-    private Set<ExtensionBundleVersionDependency> dependencies;
+    private Set<BundleVersionDependency> dependencies;
 
     // read-only, only populated from retrieval of an individual bundle version
-    private ExtensionBundle extensionBundle;
+    private Bundle bundle;
 
     // read-only, only populated from retrieval of an individual bundle version
     private Bucket bucket;
 
+
     @ApiModelProperty(value = "The metadata about this version of the extension bundle")
-    public ExtensionBundleVersionMetadata getVersionMetadata() {
+    public BundleVersionMetadata getVersionMetadata() {
         return versionMetadata;
     }
 
-    public void setVersionMetadata(ExtensionBundleVersionMetadata versionMetadata) {
+    public void setVersionMetadata(BundleVersionMetadata versionMetadata) {
         this.versionMetadata = versionMetadata;
     }
 
     @ApiModelProperty(value = "The set of other bundle versions that this version is dependent on", readOnly = true)
-    public Set<ExtensionBundleVersionDependency> getDependencies() {
+    public Set<BundleVersionDependency> getDependencies() {
         return dependencies;
     }
 
-    public void setDependencies(Set<ExtensionBundleVersionDependency> dependencies) {
+    public void setDependencies(Set<BundleVersionDependency> dependencies) {
         this.dependencies = dependencies;
     }
 
     @ApiModelProperty(value = "The bundle this version is for", readOnly = true)
-    public ExtensionBundle getExtensionBundle() {
-        return extensionBundle;
+    public Bundle getBundle() {
+        return bundle;
     }
 
-    public void setExtensionBundle(ExtensionBundle extensionBundle) {
-        this.extensionBundle = extensionBundle;
+    public void setBundle(Bundle bundle) {
+        this.bundle = bundle;
     }
 
     @ApiModelProperty(value = "The bucket that the extension bundle belongs to")
@@ -82,16 +83,16 @@ public class ExtensionBundleVersion extends LinkableEntity {
 
     @XmlTransient
     public String getFilename() {
-        final String filename = extensionBundle.getArtifactId() + "-" + versionMetadata.getVersion();
+        final String filename = bundle.getArtifactId() + "-" + versionMetadata.getVersion();
 
-        switch (extensionBundle.getBundleType()) {
+        switch (bundle.getBundleType()) {
             case NIFI_NAR:
                 return filename + ".nar";
             case MINIFI_CPP:
                 // TODO should CPP get a special extension
                 return filename;
             default:
-                throw new IllegalStateException("Unknown bundle type: " + extensionBundle.getBundleType());
+                throw new IllegalStateException("Unknown bundle type: " + bundle.getBundleType());
         }
     }
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionDependency.java
similarity index 92%
copy from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionDependency.java
index f84649b..2eb792b 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionDependency.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
@@ -23,7 +23,7 @@ import javax.validation.constraints.NotBlank;
 import java.util.Objects;
 
 @ApiModel
-public class ExtensionBundleVersionDependency {
+public class BundleVersionDependency {
 
     @NotBlank
     private String groupId;
@@ -75,7 +75,7 @@ public class ExtensionBundleVersionDependency {
             return false;
         }
 
-        final ExtensionBundleVersionDependency other = (ExtensionBundleVersionDependency) obj;
+        final BundleVersionDependency other = (BundleVersionDependency) obj;
 
         return Objects.equals(groupId, other.groupId)
                 && Objects.equals(artifactId, other.artifactId)
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/filter/ExtensionBundleVersionFilterParams.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionFilterParams.java
similarity index 71%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/filter/ExtensionBundleVersionFilterParams.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionFilterParams.java
index 1402989..2192b9a 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/filter/ExtensionBundleVersionFilterParams.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionFilterParams.java
@@ -14,20 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension.filter;
+package org.apache.nifi.registry.extension.bundle;
 
 /**
  * Filter parameters for extension bundle versions.
+ *
+ * Any combination of fields may be populated to filter on the provided values.
+ *
+ * Note: This class is currently not part of the REST API so it doesn't not have the Swagger annotations, but it is used
+ * in the service layer and client to pass around params.
  */
-public class ExtensionBundleVersionFilterParams {
+public class BundleVersionFilterParams {
 
-    private static final ExtensionBundleVersionFilterParams EMPTY_PARAMS = new Builder().build();
+    private static final BundleVersionFilterParams EMPTY_PARAMS = new Builder().build();
 
     private final String groupId;
     private final String artifactId;
     private final String version;
 
-    private ExtensionBundleVersionFilterParams(final Builder builder) {
+    private BundleVersionFilterParams(final Builder builder) {
         this.groupId = builder.groupId;
         this.artifactId = builder.artifactId;
         this.version = builder.version;
@@ -45,11 +50,11 @@ public class ExtensionBundleVersionFilterParams {
         return version;
     }
 
-    public static ExtensionBundleVersionFilterParams of(final String groupId, final String artifactId, final String version) {
+    public static BundleVersionFilterParams of(final String groupId, final String artifactId, final String version) {
         return new Builder().group(groupId).artifact(artifactId).version(version).build();
     }
 
-    public static ExtensionBundleVersionFilterParams empty() {
+    public static BundleVersionFilterParams empty() {
         return EMPTY_PARAMS;
     }
 
@@ -74,8 +79,8 @@ public class ExtensionBundleVersionFilterParams {
             return this;
         }
 
-        public ExtensionBundleVersionFilterParams build() {
-            return new ExtensionBundleVersionFilterParams(this);
+        public BundleVersionFilterParams build() {
+            return new BundleVersionFilterParams(this);
         }
     }
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionMetadata.java
similarity index 78%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionMetadata.java
index 6902083..39402d2 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/bundle/BundleVersionMetadata.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.bundle;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 import org.apache.nifi.registry.link.LinkableEntity;
 
+import javax.validation.Valid;
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotBlank;
 import javax.validation.constraints.NotNull;
@@ -28,13 +29,13 @@ import java.util.Objects;
 
 @ApiModel
 @XmlRootElement
-public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
+public class BundleVersionMetadata extends LinkableEntity implements Comparable<BundleVersionMetadata> {
 
     @NotBlank
     private String id;
 
     @NotBlank
-    private String extensionBundleId;
+    private String bundleId;
 
     @NotBlank
     private String bucketId;
@@ -60,6 +61,13 @@ public class ExtensionBundleVersionMetadata extends LinkableEntity implements Co
     @Min(0)
     private long contentSize;
 
+    @NotBlank
+    private String systemApiVersion;
+
+    @Valid
+    @NotNull
+    private BuildInfo buildInfo;
+
 
     @ApiModelProperty(value = "The id of this version of the extension bundle")
     public String getId() {
@@ -71,12 +79,12 @@ public class ExtensionBundleVersionMetadata extends LinkableEntity implements Co
     }
 
     @ApiModelProperty(value = "The id of the extension bundle this version is for")
-    public String getExtensionBundleId() {
-        return extensionBundleId;
+    public String getBundleId() {
+        return bundleId;
     }
 
-    public void setExtensionBundleId(String extensionBundleId) {
-        this.extensionBundleId = extensionBundleId;
+    public void setBundleId(String bundleId) {
+        this.bundleId = bundleId;
     }
 
     @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
@@ -151,8 +159,26 @@ public class ExtensionBundleVersionMetadata extends LinkableEntity implements Co
         this.contentSize = contentSize;
     }
 
+    @ApiModelProperty(value = "The version of the system API that this bundle version was built against")
+    public String getSystemApiVersion() {
+        return systemApiVersion;
+    }
+
+    public void setSystemApiVersion(String systemApiVersion) {
+        this.systemApiVersion = systemApiVersion;
+    }
+
+    @ApiModelProperty(value = "The build information about this version")
+    public BuildInfo getBuildInfo() {
+        return buildInfo;
+    }
+
+    public void setBuildInfo(BuildInfo buildInfo) {
+        this.buildInfo = buildInfo;
+    }
+
     @Override
-    public int compareTo(final ExtensionBundleVersionMetadata o) {
+    public int compareTo(final BundleVersionMetadata o) {
         return o == null ? -1 : version.compareTo(o.version);
     }
 
@@ -170,7 +196,7 @@ public class ExtensionBundleVersionMetadata extends LinkableEntity implements Co
             return false;
         }
 
-        final ExtensionBundleVersionMetadata other = (ExtensionBundleVersionMetadata) obj;
+        final BundleVersionMetadata other = (BundleVersionMetadata) obj;
         return Objects.equals(this.id, other.id);
     }
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionFilterParams.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionFilterParams.java
new file mode 100644
index 0000000..963b3b7
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionFilterParams.java
@@ -0,0 +1,101 @@
+/*
+ * 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.registry.extension.component;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.registry.extension.bundle.BundleType;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionType;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Parameters for filtering on extensions. All parameters will be AND'd together, but tags will be OR'd.
+ */
+@ApiModel
+public class ExtensionFilterParams {
+
+    private final BundleType bundleType;
+    private final ExtensionType extensionType;
+    private final Set<String> tags;
+
+    // Used by Jackson
+    private ExtensionFilterParams() {
+        bundleType = null;
+        extensionType = null;
+        tags = null;
+    }
+
+    private ExtensionFilterParams(final Builder builder) {
+        this.bundleType = builder.bundleType;
+        this.extensionType = builder.extensionType;
+        this.tags = Collections.unmodifiableSet(new HashSet<>(builder.tags));
+    }
+
+    @ApiModelProperty("The type of bundle")
+    public BundleType getBundleType() {
+        return bundleType;
+    }
+
+    @ApiModelProperty("The type of extension")
+    public ExtensionType getExtensionType() {
+        return extensionType;
+    }
+
+    @ApiModelProperty("The tags")
+    public Set<String> getTags() {
+        return tags;
+    }
+
+    public static class Builder {
+
+        private BundleType bundleType;
+        private ExtensionType extensionType;
+        private Set<String> tags = new HashSet<>();
+
+        public Builder bundleType(final BundleType bundleType) {
+            this.bundleType = bundleType;
+            return this;
+        }
+
+        public Builder extensionType(final ExtensionType extensionType) {
+            this.extensionType = extensionType;
+            return this;
+        }
+
+        public Builder tag(final String tag) {
+            if (tag != null) {
+                tags.add(tag);
+            }
+            return this;
+        }
+
+        public Builder addTags(final Collection<String> tags) {
+            if (tags != null) {
+                this.tags.addAll(tags);
+            }
+            return this;
+        }
+
+        public ExtensionFilterParams build() {
+            return new ExtensionFilterParams(this);
+        }
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionMetadata.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionMetadata.java
new file mode 100644
index 0000000..a8ff0b7
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionMetadata.java
@@ -0,0 +1,144 @@
+/*
+ * 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.registry.extension.component;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.registry.extension.bundle.BundleInfo;
+import org.apache.nifi.registry.extension.component.manifest.DeprecationNotice;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionType;
+import org.apache.nifi.registry.extension.component.manifest.ProvidedServiceAPI;
+import org.apache.nifi.registry.extension.component.manifest.Restricted;
+import org.apache.nifi.registry.link.LinkableEntity;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+@ApiModel
+public class ExtensionMetadata extends LinkableEntity implements Comparable<ExtensionMetadata> {
+
+    private String name;
+    private String displayName;
+    private ExtensionType type;
+    private String description;
+    private DeprecationNotice deprecationNotice;
+    private List<String> tags;
+    private Restricted restricted;
+    private List<ProvidedServiceAPI> providedServiceAPIs;
+    private BundleInfo bundleInfo;
+
+    @ApiModelProperty(value = "The name of the extension")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The display name of the extension")
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
+    @ApiModelProperty(value = "The type of the extension")
+    public ExtensionType getType() {
+        return type;
+    }
+
+    public void setType(ExtensionType type) {
+        this.type = type;
+    }
+
+    @ApiModelProperty(value = "The description of the extension")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "The deprecation notice of the extension")
+    public DeprecationNotice getDeprecationNotice() {
+        return deprecationNotice;
+    }
+
+    public void setDeprecationNotice(DeprecationNotice deprecationNotice) {
+        this.deprecationNotice = deprecationNotice;
+    }
+
+    @ApiModelProperty(value = "The tags of the extension")
+    public List<String> getTags() {
+        return tags;
+    }
+
+    public void setTags(List<String> tags) {
+        this.tags = tags;
+    }
+
+    @ApiModelProperty(value = "The restrictions of the extension")
+    public Restricted getRestricted() {
+        return restricted;
+    }
+
+    public void setRestricted(Restricted restricted) {
+        this.restricted = restricted;
+    }
+
+    @ApiModelProperty(value = "The service APIs provided by the extension")
+    public List<ProvidedServiceAPI> getProvidedServiceAPIs() {
+        return providedServiceAPIs;
+    }
+
+    public void setProvidedServiceAPIs(List<ProvidedServiceAPI> providedServiceAPIs) {
+        this.providedServiceAPIs = providedServiceAPIs;
+    }
+
+    @ApiModelProperty(value = "The information for the bundle where this extension is located")
+    public BundleInfo getBundleInfo() {
+        return bundleInfo;
+    }
+
+    public void setBundleInfo(BundleInfo bundleInfo) {
+        this.bundleInfo = bundleInfo;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ExtensionMetadata extension = (ExtensionMetadata) o;
+        return Objects.equals(name, extension.name);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(name);
+    }
+
+    @Override
+    public int compareTo(final ExtensionMetadata o) {
+        return Comparator.comparing(ExtensionMetadata::getDisplayName).compare(this, o);
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionMetadataContainer.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionMetadataContainer.java
new file mode 100644
index 0000000..867f8b0
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/ExtensionMetadataContainer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.registry.extension.component;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.SortedSet;
+
+@ApiModel
+public class ExtensionMetadataContainer {
+
+    private int numResults;
+    private ExtensionFilterParams filterParams;
+    private SortedSet<ExtensionMetadata> extensions;
+
+    @ApiModelProperty("The number of extensions in the response")
+    public int getNumResults() {
+        return numResults;
+    }
+
+    public void setNumResults(int numResults) {
+        this.numResults = numResults;
+    }
+
+    @ApiModelProperty("The filter parameters submitted for the request")
+    public ExtensionFilterParams getFilterParams() {
+        return filterParams;
+    }
+
+    public void setFilterParams(ExtensionFilterParams filterParams) {
+        this.filterParams = filterParams;
+    }
+
+    @ApiModelProperty("The metadata for the extensions")
+    public SortedSet<ExtensionMetadata> getExtensions() {
+        return extensions;
+    }
+
+    public void setExtensions(SortedSet<ExtensionMetadata> extensions) {
+        this.extensions = extensions;
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/TagCount.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/TagCount.java
new file mode 100644
index 0000000..d040fce
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/TagCount.java
@@ -0,0 +1,66 @@
+/*
+ * 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.registry.extension.component;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Comparator;
+import java.util.Objects;
+
+@ApiModel
+public class TagCount implements Comparable<TagCount> {
+
+    private String tag;
+    private int count;
+
+    @ApiModelProperty("The tag label")
+    public String getTag() {
+        return tag;
+    }
+
+    public void setTag(String tag) {
+        this.tag = tag;
+    }
+
+    @ApiModelProperty("The number of occurrences of the given tag")
+    public int getCount() {
+        return count;
+    }
+
+    public void setCount(int count) {
+        this.count = count;
+    }
+
+    @Override
+    public int compareTo(TagCount o) {
+        return Comparator.comparing(TagCount::getTag).compare(this, o);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TagCount tagCount = (TagCount) o;
+        return count == tagCount.count && Objects.equals(tag, tagCount.tag);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(tag, count);
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/AllowableValue.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/AllowableValue.java
new file mode 100644
index 0000000..debcc8f
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/AllowableValue.java
@@ -0,0 +1,59 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AllowableValue {
+
+    private String value;
+    private String displayName;
+    private String description;
+
+    @ApiModelProperty(value = "The value of the allowable value")
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+    @ApiModelProperty(value = "The display name of the allowable value")
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
+    @ApiModelProperty(value = "The description of the allowable value")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/ExtensionBundleVersionCoordinate.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Attribute.java
similarity index 51%
copy from nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/ExtensionBundleVersionCoordinate.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Attribute.java
index 38904e5..acfcd66 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/ExtensionBundleVersionCoordinate.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Attribute.java
@@ -14,29 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.service.extension;
+package org.apache.nifi.registry.extension.component.manifest;
 
-import org.apache.commons.lang3.Validate;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
 
-/**
- * The unique coordinate for a version of an extension bundle.
- */
-public class ExtensionBundleVersionCoordinate extends ExtensionBundleCoordinate {
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Attribute {
 
-    private final String version;
+    private String name;
+    private String description;
+
+    @ApiModelProperty(value = "The name of the attribute")
+    public String getName() {
+        return name;
+    }
 
-    public ExtensionBundleVersionCoordinate(final String bucketId, final String groupId, final String artifactId, final String version) {
-        super(bucketId, groupId, artifactId);
-        this.version = version;
-        Validate.notBlank(this.version, "Version cannot be null or blank");
+    public void setName(String name) {
+        this.name = name;
     }
 
-    public String getVersion() {
-        return version;
+    @ApiModelProperty(value = "The description of the attribute")
+    public String getDescription() {
+        return description;
     }
 
-    @Override
-    public String toString() {
-        return super.toString() + ":" + version;
+    public void setDescription(String description) {
+        this.description = description;
     }
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ControllerServiceDefinition.java
similarity index 55%
copy from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ControllerServiceDefinition.java
index f84649b..a303bc0 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ControllerServiceDefinition.java
@@ -14,27 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.component.manifest;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
-import javax.validation.constraints.NotBlank;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
 import java.util.Objects;
 
 @ApiModel
-public class ExtensionBundleVersionDependency {
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ControllerServiceDefinition {
 
-    @NotBlank
+    private String className;
     private String groupId;
-
-    @NotBlank
     private String artifactId;
-
-    @NotBlank
     private String version;
 
-    @ApiModelProperty(value = "The group id of the bundle dependency")
+    @ApiModelProperty(value = "The class name of the service API")
+    public String getClassName() {
+        return className;
+    }
+
+    public void setClassName(String className) {
+        this.className = className;
+    }
+
+    @ApiModelProperty(value = "The group id of the service API")
     public String getGroupId() {
         return groupId;
     }
@@ -43,7 +50,7 @@ public class ExtensionBundleVersionDependency {
         this.groupId = groupId;
     }
 
-    @ApiModelProperty(value = "The artifact id of the bundle dependency")
+    @ApiModelProperty(value = "The artifact id of the service API")
     public String getArtifactId() {
         return artifactId;
     }
@@ -52,7 +59,7 @@ public class ExtensionBundleVersionDependency {
         this.artifactId = artifactId;
     }
 
-    @ApiModelProperty(value = "The version of the bundle dependency")
+    @ApiModelProperty(value = "The version of the service API")
     public String getVersion() {
         return version;
     }
@@ -62,23 +69,19 @@ public class ExtensionBundleVersionDependency {
     }
 
     @Override
-    public int hashCode() {
-        return Objects.hash(groupId, artifactId, version);
+    public boolean equals(final Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        final ControllerServiceDefinition that = (ControllerServiceDefinition) o;
+        return Objects.equals(className, that.className)
+                && Objects.equals(groupId, that.groupId)
+                && Objects.equals(artifactId, that.artifactId)
+                && Objects.equals(version, that.version);
     }
 
     @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final ExtensionBundleVersionDependency other = (ExtensionBundleVersionDependency) obj;
-
-        return Objects.equals(groupId, other.groupId)
-                && Objects.equals(artifactId, other.artifactId)
-                && Objects.equals(version, other.version);
+    public int hashCode() {
+        return Objects.hash(className, groupId, artifactId, version);
     }
+
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DeprecationNotice.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DeprecationNotice.java
new file mode 100644
index 0000000..32fe548
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DeprecationNotice.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import java.util.List;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class DeprecationNotice {
+
+    private String reason;
+
+    @XmlElementWrapper
+    @XmlElement(name = "alternative")
+    private List<String> alternatives;
+
+    @ApiModelProperty(value = "The reason for the deprecation")
+    public String getReason() {
+        return reason;
+    }
+
+    public void setReason(String reason) {
+        this.reason = reason;
+    }
+
+    @ApiModelProperty(value = "The alternatives to use")
+    public List<String> getAlternatives() {
+        return alternatives;
+    }
+
+    public void setAlternatives(List<String> alternatives) {
+        this.alternatives = alternatives;
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DynamicProperty.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DynamicProperty.java
new file mode 100644
index 0000000..cc1be0d
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DynamicProperty.java
@@ -0,0 +1,79 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class DynamicProperty {
+
+    private String name;
+    private String value;
+    private String description;
+    private ExpressionLanguageScope expressionLanguageScope;
+    private boolean expressionLanguageSupported;
+
+    @ApiModelProperty(value = "The description of the dynamic property name")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The description of the dynamic property value")
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+    @ApiModelProperty(value = "The description of the dynamic property")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "Whether or not expression language is supported")
+    public boolean isExpressionLanguageSupported() {
+        return expressionLanguageSupported;
+    }
+
+    public void setExpressionLanguageSupported(boolean expressionLanguageSupported) {
+        this.expressionLanguageSupported = expressionLanguageSupported;
+    }
+
+    @ApiModelProperty(value = "The scope of the expression language support")
+    public ExpressionLanguageScope getExpressionLanguageScope() {
+        return expressionLanguageScope;
+    }
+
+    public void setExpressionLanguageScope(ExpressionLanguageScope expressionLanguageScope) {
+        this.expressionLanguageScope = expressionLanguageScope;
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DynamicRelationship.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DynamicRelationship.java
new file mode 100644
index 0000000..e14e363
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/DynamicRelationship.java
@@ -0,0 +1,50 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class DynamicRelationship {
+
+    private String name;
+    private String description;
+
+    @ApiModelProperty(value = "The description of the dynamic relationship name")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The description of the dynamic relationship")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExpressionLanguageScope.java
similarity index 64%
copy from nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExpressionLanguageScope.java
index b072b4a..c9e22e1 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExpressionLanguageScope.java
@@ -14,14 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.db.entity;
+package org.apache.nifi.registry.extension.component.manifest;
 
-public enum ExtensionEntityCategory {
+import io.swagger.annotations.ApiModel;
 
-    PROCESSOR,
+@ApiModel
+public enum ExpressionLanguageScope {
 
-    CONTROLLER_SERVICE,
+    /**
+     * Expression language is disabled
+     */
+    NONE,
 
-    REPORTING_TASK;
+    /**
+     * Expression language is evaluated against variables in registry
+     */
+    VARIABLE_REGISTRY,
+
+    /**
+     * Expression language is evaluated per flow file using attributes
+     */
+    FLOWFILE_ATTRIBUTES;
 
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Extension.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Extension.java
new file mode 100644
index 0000000..143507f
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Extension.java
@@ -0,0 +1,258 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.validation.Valid;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import java.util.List;
+import java.util.Objects;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Extension {
+
+    @Valid
+    @XmlElement(required = true)
+    private String name;
+
+    @Valid
+    @XmlElement(required = true)
+    private ExtensionType type;
+
+    private DeprecationNotice deprecationNotice;
+
+    private String description;
+
+    @XmlElementWrapper
+    @XmlElement(name = "tag")
+    private List<String> tags;
+
+    @XmlElementWrapper
+    @XmlElement(name = "property")
+    private List<Property> properties;
+
+    @XmlElementWrapper
+    @XmlElement(name = "dynamicProperty")
+    private List<DynamicProperty> dynamicProperties;
+
+    @XmlElementWrapper
+    @XmlElement(name = "relationship")
+    private List<Relationship> relationships;
+
+    private DynamicRelationship dynamicRelationship;
+
+    @XmlElementWrapper
+    @XmlElement(name = "readsAttribute")
+    private List<Attribute> readsAttributes;
+
+    @XmlElementWrapper
+    @XmlElement(name = "writesAttribute")
+    private List<Attribute> writesAttributes;
+
+    private Stateful stateful;
+
+    @Valid
+    private Restricted restricted;
+
+    private InputRequirement inputRequirement;
+
+    @XmlElementWrapper
+    @XmlElement(name = "systemResourceConsideration")
+    private List<SystemResourceConsideration> systemResourceConsiderations;
+
+    @XmlElementWrapper
+    @XmlElement(name = "see")
+    private List<String> seeAlso;
+
+    @Valid
+    @XmlElementWrapper
+    @XmlElement(name = "providedServiceAPI")
+    private List<ProvidedServiceAPI> providedServiceAPIs;
+
+
+    @ApiModelProperty(value = "The name of the extension")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The type of the extension")
+    public ExtensionType getType() {
+        return type;
+    }
+
+    public void setType(ExtensionType type) {
+        this.type = type;
+    }
+
+    @ApiModelProperty(value = "The deprecation notice of the extension")
+    public DeprecationNotice getDeprecationNotice() {
+        return deprecationNotice;
+    }
+
+    public void setDeprecationNotice(DeprecationNotice deprecationNotice) {
+        this.deprecationNotice = deprecationNotice;
+    }
+
+    @ApiModelProperty(value = "The description of the extension")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "The tags of the extension")
+    public List<String> getTags() {
+        return tags;
+    }
+
+    public void setTags(List<String> tags) {
+        this.tags = tags;
+    }
+
+    @ApiModelProperty(value = "The properties of the extension")
+    public List<Property> getProperties() {
+        return properties;
+    }
+
+    public void setProperties(List<Property> properties) {
+        this.properties = properties;
+    }
+
+    @ApiModelProperty(value = "The dynamic properties of the extension")
+    public List<DynamicProperty> getDynamicProperties() {
+        return dynamicProperties;
+    }
+
+    public void setDynamicProperties(List<DynamicProperty> dynamicProperties) {
+        this.dynamicProperties = dynamicProperties;
+    }
+
+    @ApiModelProperty(value = "The relationships of the extension")
+    public List<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    public void setRelationships(List<Relationship> relationships) {
+        this.relationships = relationships;
+    }
+
+    @ApiModelProperty(value = "The dynamic relationships of the extension")
+    public DynamicRelationship getDynamicRelationship() {
+        return dynamicRelationship;
+    }
+
+    public void setDynamicRelationship(DynamicRelationship dynamicRelationship) {
+        this.dynamicRelationship = dynamicRelationship;
+    }
+
+    @ApiModelProperty(value = "The attributes read from flow files by the extension")
+    public List<Attribute> getReadsAttributes() {
+        return readsAttributes;
+    }
+
+    public void setReadsAttributes(List<Attribute> readsAttributes) {
+        this.readsAttributes = readsAttributes;
+    }
+
+    @ApiModelProperty(value = "The attributes written to flow files by the extension")
+    public List<Attribute> getWritesAttributes() {
+        return writesAttributes;
+    }
+
+    public void setWritesAttributes(List<Attribute> writesAttributes) {
+        this.writesAttributes = writesAttributes;
+    }
+
+    @ApiModelProperty(value = "The information about how the extension stores state")
+    public Stateful getStateful() {
+        return stateful;
+    }
+
+    public void setStateful(Stateful stateful) {
+        this.stateful = stateful;
+    }
+
+    @ApiModelProperty(value = "The restrictions of the extension")
+    public Restricted getRestricted() {
+        return restricted;
+    }
+
+    public void setRestricted(Restricted restricted) {
+        this.restricted = restricted;
+    }
+
+    @ApiModelProperty(value = "The input requirement of the extension")
+    public InputRequirement getInputRequirement() {
+        return inputRequirement;
+    }
+
+    public void setInputRequirement(InputRequirement inputRequirement) {
+        this.inputRequirement = inputRequirement;
+    }
+
+    @ApiModelProperty(value = "The resource considerations of the extension")
+    public List<SystemResourceConsideration> getSystemResourceConsiderations() {
+        return systemResourceConsiderations;
+    }
+
+    public void setSystemResourceConsiderations(List<SystemResourceConsideration> systemResourceConsiderations) {
+        this.systemResourceConsiderations = systemResourceConsiderations;
+    }
+
+    @ApiModelProperty(value = "The names of other extensions to see")
+    public List<String> getSeeAlso() {
+        return seeAlso;
+    }
+
+    public void setSeeAlso(List<String> seeAlso) {
+        this.seeAlso = seeAlso;
+    }
+
+    @ApiModelProperty(value = "The service APIs provided by this extension")
+    public List<ProvidedServiceAPI> getProvidedServiceAPIs() {
+        return providedServiceAPIs;
+    }
+
+    public void setProvidedServiceAPIs(List<ProvidedServiceAPI> providedServiceAPIs) {
+        this.providedServiceAPIs = providedServiceAPIs;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        Extension extension = (Extension) o;
+        return Objects.equals(name, extension.name);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(name);
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExtensionManifest.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExtensionManifest.java
new file mode 100644
index 0000000..c71d771
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExtensionManifest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.List;
+
+@ApiModel
+@XmlRootElement(name = "extensionManifest")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ExtensionManifest {
+
+    @XmlElement(required = true)
+    private String systemApiVersion;
+
+    @XmlElementWrapper
+    @XmlElement(name = "extension")
+    private List<Extension> extensions;
+
+    public ExtensionManifest() {
+    }
+
+    public ExtensionManifest(String systemApiVersion, List<Extension> extensions) {
+        this.systemApiVersion = systemApiVersion;
+        this.extensions = extensions;
+    }
+
+    public String getSystemApiVersion() {
+        return systemApiVersion;
+    }
+
+    public void setSystemApiVersion(String systemApiVersion) {
+        this.systemApiVersion = systemApiVersion;
+    }
+
+    public List<Extension> getExtensions() {
+        return extensions;
+    }
+
+    public void setExtensions(List<Extension> extensions) {
+        this.extensions = extensions;
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExtensionType.java
similarity index 86%
copy from nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java
copy to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExtensionType.java
index b072b4a..f93c75b 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionEntityCategory.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ExtensionType.java
@@ -14,9 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.db.entity;
+package org.apache.nifi.registry.extension.component.manifest;
 
-public enum ExtensionEntityCategory {
+/**
+ * Possible types of extensions.
+ */
+public enum ExtensionType {
 
     PROCESSOR,
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/InputRequirement.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/InputRequirement.java
new file mode 100644
index 0000000..d1dc9ca
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/InputRequirement.java
@@ -0,0 +1,48 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+
+@ApiModel
+public enum InputRequirement {
+
+    /**
+     * This value is used to indicate that the Processor requires input from other Processors
+     * in order to run. As a result, the Processor will not be valid if it does not have any
+     * incoming connections.
+     */
+    INPUT_REQUIRED,
+
+    /**
+     * This value is used to indicate that the Processor will consume data from an incoming
+     * connection but does not require an incoming connection in order to perform its task.
+     * If the {@link InputRequirement} annotation is not present, this is the default value
+     * that is used.
+     */
+    INPUT_ALLOWED,
+
+    /**
+     * This value is used to indicate that the Processor is a "Source Processor" and does
+     * not accept incoming connections. Because the Processor does not pull FlowFiles from
+     * an incoming connection, it can be very confusing for users who create incoming connections
+     * to the Processor. As a result, this value can be used in order to clarify that incoming
+     * connections will not be used. This prevents the user from even creating such a connection.
+     */
+    INPUT_FORBIDDEN;
+
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Property.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Property.java
new file mode 100644
index 0000000..f3723f3
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Property.java
@@ -0,0 +1,159 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import java.util.List;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Property {
+
+    private String name;
+    private String displayName;
+    private String description;
+    private String defaultValue;
+    private ControllerServiceDefinition controllerServiceDefinition;
+
+    @XmlElementWrapper
+    @XmlElement(name = "allowableValue")
+    private List<AllowableValue> allowableValues;
+
+    private boolean required;
+    private boolean sensitive;
+
+    private boolean expressionLanguageSupported;
+    private ExpressionLanguageScope expressionLanguageScope;
+
+    private boolean dynamicallyModifiesClasspath;
+    private boolean dynamic;
+
+
+    @ApiModelProperty(value = "The name of the property")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The display name")
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
+    @ApiModelProperty(value = "The description")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "The default value")
+    public String getDefaultValue() {
+        return defaultValue;
+    }
+
+    public void setDefaultValue(String defaultValue) {
+        this.defaultValue = defaultValue;
+    }
+
+    @ApiModelProperty(value = "The controller service required by this property, or null if none is required")
+    public ControllerServiceDefinition getControllerServiceDefinition() {
+        return controllerServiceDefinition;
+    }
+
+    public void setControllerServiceDefinition(ControllerServiceDefinition controllerServiceDefinition) {
+        this.controllerServiceDefinition = controllerServiceDefinition;
+    }
+
+    @ApiModelProperty(value = "The allowable values for this property")
+    public List<AllowableValue> getAllowableValues() {
+        return allowableValues;
+    }
+
+    public void setAllowableValues(List<AllowableValue> allowableValues) {
+        this.allowableValues = allowableValues;
+    }
+
+    @ApiModelProperty(value = "Whether or not the property is required")
+    public boolean isRequired() {
+        return required;
+    }
+
+    public void setRequired(boolean required) {
+        this.required = required;
+    }
+
+    @ApiModelProperty(value = "Whether or not the property is sensitive")
+    public boolean isSensitive() {
+        return sensitive;
+    }
+
+    public void setSensitive(boolean sensitive) {
+        this.sensitive = sensitive;
+    }
+
+    @ApiModelProperty(value = "Whether or not expression language is supported")
+    public boolean isExpressionLanguageSupported() {
+        return expressionLanguageSupported;
+    }
+
+    public void setExpressionLanguageSupported(boolean expressionLanguageSupported) {
+        this.expressionLanguageSupported = expressionLanguageSupported;
+    }
+
+    @ApiModelProperty(value = "The scope of expression language support")
+    public ExpressionLanguageScope getExpressionLanguageScope() {
+        return expressionLanguageScope;
+    }
+
+    public void setExpressionLanguageScope(ExpressionLanguageScope expressionLanguageScope) {
+        this.expressionLanguageScope = expressionLanguageScope;
+    }
+
+    @ApiModelProperty(value = "Whether or not the processor dynamically modifies the classpath")
+    public boolean isDynamicallyModifiesClasspath() {
+        return dynamicallyModifiesClasspath;
+    }
+
+    public void setDynamicallyModifiesClasspath(boolean dynamicallyModifiesClasspath) {
+        this.dynamicallyModifiesClasspath = dynamicallyModifiesClasspath;
+    }
+
+    @ApiModelProperty(value = "Whether or not the processor is dynamic")
+    public boolean isDynamic() {
+        return dynamic;
+    }
+
+    public void setDynamic(boolean dynamic) {
+        this.dynamic = dynamic;
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ProvidedServiceAPI.java
similarity index 56%
rename from nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ProvidedServiceAPI.java
index f84649b..abad607 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionDependency.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/ProvidedServiceAPI.java
@@ -14,27 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.extension;
+package org.apache.nifi.registry.extension.component.manifest;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
 import javax.validation.constraints.NotBlank;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
 import java.util.Objects;
 
 @ApiModel
-public class ExtensionBundleVersionDependency {
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ProvidedServiceAPI {
 
     @NotBlank
+    private String className;
+    @NotBlank
     private String groupId;
-
     @NotBlank
     private String artifactId;
-
     @NotBlank
     private String version;
 
-    @ApiModelProperty(value = "The group id of the bundle dependency")
+    @ApiModelProperty(value = "The class name of the service API being provided")
+    public String getClassName() {
+        return className;
+    }
+
+    public void setClassName(String className) {
+        this.className = className;
+    }
+
+    @ApiModelProperty(value = "The group id of the service API being provided")
     public String getGroupId() {
         return groupId;
     }
@@ -43,7 +55,7 @@ public class ExtensionBundleVersionDependency {
         this.groupId = groupId;
     }
 
-    @ApiModelProperty(value = "The artifact id of the bundle dependency")
+    @ApiModelProperty(value = "The artifact id of the service API being provided")
     public String getArtifactId() {
         return artifactId;
     }
@@ -52,7 +64,7 @@ public class ExtensionBundleVersionDependency {
         this.artifactId = artifactId;
     }
 
-    @ApiModelProperty(value = "The version of the bundle dependency")
+    @ApiModelProperty(value = "The version of the service API being provided")
     public String getVersion() {
         return version;
     }
@@ -62,23 +74,18 @@ public class ExtensionBundleVersionDependency {
     }
 
     @Override
-    public int hashCode() {
-        return Objects.hash(groupId, artifactId, version);
+    public boolean equals(final Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        final ProvidedServiceAPI that = (ProvidedServiceAPI) o;
+        return Objects.equals(className, that.className)
+                && Objects.equals(groupId, that.groupId)
+                && Objects.equals(artifactId, that.artifactId)
+                && Objects.equals(version, that.version);
     }
 
     @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final ExtensionBundleVersionDependency other = (ExtensionBundleVersionDependency) obj;
-
-        return Objects.equals(groupId, other.groupId)
-                && Objects.equals(artifactId, other.artifactId)
-                && Objects.equals(version, other.version);
+    public int hashCode() {
+        return Objects.hash(className, groupId, artifactId, version);
     }
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Relationship.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Relationship.java
new file mode 100644
index 0000000..db64045
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Relationship.java
@@ -0,0 +1,59 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Relationship {
+
+    private String name;
+    private String description;
+    private boolean autoTerminated;
+
+    @ApiModelProperty(value = "The name of the relationship")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The description of the relationship")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "Whether or not the relationship is auto-terminated by default")
+    public boolean isAutoTerminated() {
+        return autoTerminated;
+    }
+
+    public void setAutoTerminated(boolean autoTerminated) {
+        this.autoTerminated = autoTerminated;
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Restricted.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Restricted.java
new file mode 100644
index 0000000..82ab09d
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Restricted.java
@@ -0,0 +1,58 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.validation.Valid;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import java.util.List;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Restricted {
+
+    private String generalRestrictionExplanation;
+
+    @Valid
+    @XmlElementWrapper
+    @XmlElement(name = "restriction")
+    private List<Restriction> restrictions;
+
+    @ApiModelProperty(value = "The general restriction for the extension, or null if only specific restrictions exist")
+    public String getGeneralRestrictionExplanation() {
+        return generalRestrictionExplanation;
+    }
+
+    public void setGeneralRestrictionExplanation(String generalRestrictionExplanation) {
+        this.generalRestrictionExplanation = generalRestrictionExplanation;
+    }
+
+    @ApiModelProperty(value = "The specific restrictions")
+    public List<Restriction> getRestrictions() {
+        return restrictions;
+    }
+
+    public void setRestrictions(List<Restriction> restrictions) {
+        this.restrictions = restrictions;
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Restriction.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Restriction.java
new file mode 100644
index 0000000..cb2501c
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Restriction.java
@@ -0,0 +1,67 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.validation.constraints.NotBlank;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import java.util.Objects;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Restriction {
+
+    @NotBlank
+    private String requiredPermission;
+    @NotBlank
+    private String explanation;
+
+    @ApiModelProperty(value = "The permission required for this restriction")
+    public String getRequiredPermission() {
+        return requiredPermission;
+    }
+
+    public void setRequiredPermission(String requiredPermission) {
+        this.requiredPermission = requiredPermission;
+    }
+
+    @ApiModelProperty(value = "The explanation of this restriction")
+    public String getExplanation() {
+        return explanation;
+    }
+
+    public void setExplanation(String explanation) {
+        this.explanation = explanation;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        final Restriction that = (Restriction) o;
+        return Objects.equals(requiredPermission, that.requiredPermission)
+                && Objects.equals(explanation, that.explanation);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(requiredPermission, explanation);
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionBundleEntityType.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Scope.java
similarity index 80%
rename from nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionBundleEntityType.java
rename to nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Scope.java
index 0f4950c..e07d183 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/ExtensionBundleEntityType.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Scope.java
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.db.entity;
+package org.apache.nifi.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
 
 /**
- * The possible types of extension bundles.
+ * Possible scopes for storing state.
  */
-public enum ExtensionBundleEntityType {
-
-    NIFI_NAR,
+@ApiModel
+public enum Scope {
 
-    MINIFI_CPP;
+    CLUSTER,
 
+    LOCAL;
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Stateful.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Stateful.java
new file mode 100644
index 0000000..3d20070
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/Stateful.java
@@ -0,0 +1,56 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import java.util.List;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class Stateful {
+
+    private String description;
+
+    @XmlElementWrapper
+    @XmlElement(name = "scope")
+    private List<Scope> scopes;
+
+    @ApiModelProperty(value = "The description for how the extension stores state")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @ApiModelProperty(value = "The scopes used to store state")
+    public List<Scope> getScopes() {
+        return scopes;
+    }
+
+    public void setScopes(List<Scope> scopes) {
+        this.scopes = scopes;
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/SystemResourceConsideration.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/SystemResourceConsideration.java
new file mode 100644
index 0000000..589eed6
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/component/manifest/SystemResourceConsideration.java
@@ -0,0 +1,50 @@
+/*
+ * 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.registry.extension.component.manifest;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+
+@ApiModel
+@XmlAccessorType(XmlAccessType.FIELD)
+public class SystemResourceConsideration {
+
+    private String resource;
+    private String description;
+
+    @ApiModelProperty(value = "The resource to consider")
+    public String getResource() {
+        return resource;
+    }
+
+    public void setResource(String resource) {
+        this.resource = resource;
+    }
+
+    @ApiModelProperty(value = "The description of how the resource is affected")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoArtifact.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoArtifact.java
index 6b42678..ed10643 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoArtifact.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoArtifact.java
@@ -29,9 +29,7 @@ import java.util.Objects;
 public class ExtensionRepoArtifact extends LinkableEntity implements Comparable<ExtensionRepoArtifact> {
 
     private String bucketName;
-
     private String groupId;
-
     private String artifactId;
 
     @ApiModelProperty(value = "The bucket name")
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoExtensionMetadata.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoExtensionMetadata.java
new file mode 100644
index 0000000..76de267
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoExtensionMetadata.java
@@ -0,0 +1,51 @@
+/*
+ * 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.registry.extension.repo;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.registry.extension.component.ExtensionMetadata;
+import org.apache.nifi.registry.link.LinkableEntity;
+
+import java.util.Comparator;
+
+@ApiModel
+public class ExtensionRepoExtensionMetadata extends LinkableEntity implements Comparable<ExtensionRepoExtensionMetadata> {
+
+    private ExtensionMetadata extensionMetadata;
+
+    public ExtensionRepoExtensionMetadata() {
+    }
+
+    public ExtensionRepoExtensionMetadata(final ExtensionMetadata extensionMetadata) {
+        this.extensionMetadata = extensionMetadata;
+    }
+
+    @ApiModelProperty(value = "The extension metadata")
+    public ExtensionMetadata getExtensionMetadata() {
+        return extensionMetadata;
+    }
+
+    public void setExtensionMetadata(ExtensionMetadata extensionMetadata) {
+        this.extensionMetadata = extensionMetadata;
+    }
+
+    @Override
+    public int compareTo(ExtensionRepoExtensionMetadata o) {
+        return Comparator.comparing(ExtensionRepoExtensionMetadata::getExtensionMetadata).compare(this, o);
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoGroup.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoGroup.java
index 86e25f2..df28fba 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoGroup.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoGroup.java
@@ -29,7 +29,6 @@ import java.util.Objects;
 public class ExtensionRepoGroup extends LinkableEntity implements Comparable<ExtensionRepoGroup> {
 
     private String bucketName;
-
     private String groupId;
 
     @ApiModelProperty(value = "The bucket name")
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersion.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersion.java
index 4dff6e6..dea1eda 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersion.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersion.java
@@ -29,14 +29,24 @@ import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 @XmlRootElement
 public class ExtensionRepoVersion {
 
+    private Link extensionsLink;
     private Link downloadLink;
-
     private Link sha256Link;
-
     private Boolean sha256Supplied;
 
     @XmlElement
     @XmlJavaTypeAdapter(LinkAdapter.class)
+    @ApiModelProperty(value = "The WebLink to view the metadata about the extensions contained in the extension bundle.", readOnly = true)
+    public Link getExtensionsLink() {
+        return extensionsLink;
+    }
+
+    public void setExtensionsLink(Link extensionsLink) {
+        this.extensionsLink = extensionsLink;
+    }
+
+    @XmlElement
+    @XmlJavaTypeAdapter(LinkAdapter.class)
     @ApiModelProperty(value = "The WebLink to download this version of the extension bundle.", readOnly = true)
     public Link getDownloadLink() {
         return downloadLink;
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersionSummary.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersionSummary.java
index f73d32e..41eed21 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersionSummary.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersionSummary.java
@@ -31,11 +31,12 @@ public class ExtensionRepoVersionSummary extends LinkableEntity implements Compa
     private String bucketName;
 
     private String groupId;
-
     private String artifactId;
-
     private String version;
 
+    private String author;
+    private Long timestamp;
+
     @ApiModelProperty(value = "The bucket name")
     public String getBucketName() {
         return bucketName;
@@ -72,6 +73,24 @@ public class ExtensionRepoVersionSummary extends LinkableEntity implements Compa
         this.version = version;
     }
 
+    @ApiModelProperty("The identity of the user that created this version")
+    public String getAuthor() {
+        return author;
+    }
+
+    public void setAuthor(String author) {
+        this.author = author;
+    }
+
+    @ApiModelProperty("The timestamp of when this version was created")
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
     @Override
     public int compareTo(ExtensionRepoVersionSummary o) {
         return Comparator.comparing(ExtensionRepoVersionSummary::getVersion)
@@ -83,7 +102,7 @@ public class ExtensionRepoVersionSummary extends LinkableEntity implements Compa
 
     @Override
     public int hashCode() {
-        return Objects.hash(this.bucketName, this.groupId, this.artifactId, this.version);
+        return Objects.hash(this.bucketName, this.groupId, this.artifactId, this.version, this.author, this.timestamp);
     }
 
     @Override
@@ -100,6 +119,8 @@ public class ExtensionRepoVersionSummary extends LinkableEntity implements Compa
         return Objects.equals(this.getBucketName(), other.getBucketName())
                 && Objects.equals(this.getGroupId(), other.getGroupId())
                 && Objects.equals(this.getArtifactId(), other.getArtifactId())
+                && Objects.equals(this.getVersion(), other.getVersion())
+                && Objects.equals(this.getAuthor(), other.getAuthor())
                 && Objects.equals(this.getVersion(), other.getVersion());
     }
 
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/Position.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/Position.java
index bee14d2..aaba748 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/Position.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/Position.java
@@ -17,12 +17,11 @@
 
 package org.apache.nifi.registry.flow;
 
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.util.Objects;
+
 @ApiModel(description = "The position of a component on the graph")
 public class Position {
     private double x;
@@ -61,27 +60,14 @@ public class Position {
 
     @Override
     public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
         Position position = (Position) o;
-
-        return new EqualsBuilder()
-                .append(x, position.x)
-                .append(y, position.y)
-                .isEquals();
+        return Double.compare(position.x, x) == 0 && Double.compare(position.y, y) == 0;
     }
 
     @Override
     public int hashCode() {
-        return new HashCodeBuilder(17, 37)
-                .append(x)
-                .append(y)
-                .toHashCode();
+        return Objects.hash(x, y);
     }
 }
diff --git a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedRemoteProcessGroup.java b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedRemoteProcessGroup.java
index 834afac..ec219b7 100644
--- a/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedRemoteProcessGroup.java
+++ b/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedRemoteProcessGroup.java
@@ -18,7 +18,6 @@
 package org.apache.nifi.registry.flow;
 
 import io.swagger.annotations.ApiModelProperty;
-import org.apache.commons.lang3.StringUtils;
 
 import java.util.Set;
 
@@ -47,10 +46,10 @@ public class VersionedRemoteProcessGroup extends VersionedComponent {
                     " Please migrate to using targetUris only.")
     public String getTargetUri() {
 
-        if (!StringUtils.isEmpty(targetUri)) {
+        if (!isEmpty(targetUri)) {
             return targetUri;
         }
-        return !StringUtils.isEmpty(targetUris) ? targetUris.split(",", 2)[0] : null;
+        return !isEmpty(targetUris) ? targetUris.split(",", 2)[0] : null;
 
     }
 
@@ -64,13 +63,17 @@ public class VersionedRemoteProcessGroup extends VersionedComponent {
                     " If neither target uris nor target uri is set, then returns null.")
     public String getTargetUris() {
 
-        if (!StringUtils.isEmpty(targetUris)) {
+        if (!isEmpty(targetUris)) {
             return targetUris;
         }
-        return !StringUtils.isEmpty(targetUri) ? targetUri : null;
+        return !isEmpty(targetUri) ? targetUri : null;
 
     }
 
+    private boolean isEmpty(final String value) {
+        return (value == null || value.isEmpty());
+    }
+
     public void setTargetUris(String targetUris) {
         this.targetUris = targetUris;
     }
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java b/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java
index 1d4f721..b652308 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java
+++ b/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java
@@ -20,7 +20,7 @@ import org.apache.nifi.registry.db.entity.KeyEntity;
 import org.apache.nifi.registry.db.mapper.KeyEntityRowMapper;
 import org.apache.nifi.registry.security.key.Key;
 import org.apache.nifi.registry.security.key.KeyService;
-import org.apache.nifi.registry.service.DataModelMapper;
+import org.apache.nifi.registry.service.mapper.KeyMappings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -67,7 +67,7 @@ public class DatabaseKeyService implements KeyService {
             }
 
             if (keyEntity != null) {
-                key = DataModelMapper.map(keyEntity);
+                key = KeyMappings.map(keyEntity);
             } else {
                 logger.debug("No signing key found with id='" + id + "'");
             }
@@ -106,9 +106,9 @@ public class DatabaseKeyService implements KeyService {
                 final String insertSql = "INSERT INTO signing_key (ID, TENANT_IDENTITY, KEY_VALUE) VALUES (?, ?, ?)";
                 jdbcTemplate.update(insertSql, newKeyEntity.getId(), newKeyEntity.getTenantIdentity(), newKeyEntity.getKeyValue());
 
-                key = DataModelMapper.map(newKeyEntity);
+                key = KeyMappings.map(newKeyEntity);
             } else {
-                key = DataModelMapper.map(existingKeyEntity);
+                key = KeyMappings.map(existingKeyEntity);
             }
         } finally {
             writeLock.unlock();
diff --git a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java b/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java
index d80485d..bbd566c 100644
--- a/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java
+++ b/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java
@@ -20,32 +20,38 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.db.entity.BucketEntity;
 import org.apache.nifi.registry.db.entity.BucketItemEntity;
 import org.apache.nifi.registry.db.entity.BucketItemEntityType;
-import org.apache.nifi.registry.db.entity.ExtensionBundleEntity;
-import org.apache.nifi.registry.db.entity.ExtensionBundleVersionDependencyEntity;
-import org.apache.nifi.registry.db.entity.ExtensionBundleVersionEntity;
+import org.apache.nifi.registry.db.entity.BundleEntity;
+import org.apache.nifi.registry.db.entity.BundleVersionDependencyEntity;
+import org.apache.nifi.registry.db.entity.BundleVersionEntity;
 import org.apache.nifi.registry.db.entity.ExtensionEntity;
-import org.apache.nifi.registry.db.entity.ExtensionEntityCategory;
+import org.apache.nifi.registry.db.entity.ExtensionProvidedServiceApiEntity;
+import org.apache.nifi.registry.db.entity.ExtensionRestrictionEntity;
 import org.apache.nifi.registry.db.entity.FlowEntity;
 import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.apache.nifi.registry.db.entity.TagCountEntity;
 import org.apache.nifi.registry.db.mapper.BucketEntityRowMapper;
 import org.apache.nifi.registry.db.mapper.BucketItemEntityRowMapper;
-import org.apache.nifi.registry.db.mapper.ExtensionBundleEntityRowMapper;
-import org.apache.nifi.registry.db.mapper.ExtensionBundleEntityWithBucketNameRowMapper;
-import org.apache.nifi.registry.db.mapper.ExtensionBundleVersionDependencyEntityRowMapper;
-import org.apache.nifi.registry.db.mapper.ExtensionBundleVersionEntityRowMapper;
+import org.apache.nifi.registry.db.mapper.BundleEntityRowMapper;
+import org.apache.nifi.registry.db.mapper.BundleVersionDependencyEntityRowMapper;
+import org.apache.nifi.registry.db.mapper.BundleVersionEntityRowMapper;
 import org.apache.nifi.registry.db.mapper.ExtensionEntityRowMapper;
 import org.apache.nifi.registry.db.mapper.FlowEntityRowMapper;
 import org.apache.nifi.registry.db.mapper.FlowSnapshotEntityRowMapper;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleFilterParams;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleVersionFilterParams;
+import org.apache.nifi.registry.db.mapper.TagCountEntityMapper;
+import org.apache.nifi.registry.extension.bundle.BundleFilterParams;
+import org.apache.nifi.registry.extension.bundle.BundleType;
+import org.apache.nifi.registry.extension.bundle.BundleVersionFilterParams;
+import org.apache.nifi.registry.extension.component.ExtensionFilterParams;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionType;
+import org.apache.nifi.registry.extension.component.manifest.ProvidedServiceAPI;
 import org.apache.nifi.registry.service.MetadataService;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.dao.EmptyResultDataAccessException;
 import org.springframework.jdbc.core.JdbcTemplate;
-import org.springframework.jdbc.core.RowCallbackHandler;
 import org.springframework.stereotype.Repository;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
@@ -114,14 +120,8 @@ public class DatabaseMetadataService implements MetadataService {
             return Collections.emptyList();
         }
 
-        final StringBuilder sqlBuilder = new StringBuilder("SELECT * FROM bucket WHERE id IN (");
-        for (int i=0; i < bucketIds.size(); i++) {
-            if (i > 0) {
-                sqlBuilder.append(", ");
-            }
-            sqlBuilder.append("?");
-        }
-        sqlBuilder.append(") ");
+        final StringBuilder sqlBuilder = new StringBuilder("SELECT * FROM bucket WHERE ");
+        addIdentifiersInClause(sqlBuilder, "id", bucketIds);
         sqlBuilder.append("ORDER BY name ASC");
 
         return jdbcTemplate.query(sqlBuilder.toString(), bucketIds.toArray(), new BucketEntityRowMapper());
@@ -150,7 +150,7 @@ public class DatabaseMetadataService implements MetadataService {
                 "eb.artifact_id as BUNDLE_ARTIFACT_ID " +
             "FROM bucket_item item " +
             "INNER JOIN bucket b ON item.bucket_id = b.id " +
-            "LEFT JOIN extension_bundle eb ON item.id = eb.id ";
+            "LEFT JOIN bundle eb ON item.id = eb.id ";
 
     @Override
     public List<BucketItemEntity> getBucketItems(final String bucketIdentifier) {
@@ -193,8 +193,8 @@ public class DatabaseMetadataService implements MetadataService {
             } else if (item.getType() == BucketItemEntityType.EXTENSION_BUNDLE) {
                 final Long versionCount = extensionBundleVersionCounts.get(item.getId());
                 if (versionCount != null) {
-                    final ExtensionBundleEntity extensionBundleEntity = (ExtensionBundleEntity) item;
-                    extensionBundleEntity.setVersionCount(versionCount);
+                    final BundleEntity bundleEntity = (BundleEntity) item;
+                    bundleEntity.setVersionCount(versionCount);
                 }
             }
 
@@ -223,7 +223,7 @@ public class DatabaseMetadataService implements MetadataService {
     }
 
     private Map<String,Long> getExtensionBundleVersionCounts() {
-        final String sql = "SELECT extension_bundle_id, count(*) FROM extension_bundle_version GROUP BY extension_bundle_id";
+        final String sql = "SELECT bundle_id, count(*) FROM bundle_version GROUP BY bundle_id";
 
         final Map<String,Long> results = new HashMap<>();
         jdbcTemplate.query(sql, (rs) -> {
@@ -233,7 +233,7 @@ public class DatabaseMetadataService implements MetadataService {
     }
 
     private Long getExtensionBundleVersionCount(final String extensionBundleIdentifier) {
-        final String sql = "SELECT count(*) FROM extension_bundle_version WHERE extension_bundle_id = ?";
+        final String sql = "SELECT count(*) FROM bundle_version WHERE bundle_id = ?";
 
         return jdbcTemplate.queryForObject(sql, new Object[] {extensionBundleIdentifier}, (rs, num) -> {
             return rs.getLong(1);
@@ -416,7 +416,7 @@ public class DatabaseMetadataService implements MetadataService {
     //----------------- Extension Bundles ---------------------------------
 
     @Override
-    public ExtensionBundleEntity createExtensionBundle(final ExtensionBundleEntity extensionBundle) {
+    public BundleEntity createBundle(final BundleEntity extensionBundle) {
         final String itemSql =
                 "INSERT INTO bucket_item (" +
                     "ID, " +
@@ -434,11 +434,11 @@ public class DatabaseMetadataService implements MetadataService {
                 extensionBundle.getDescription(),
                 extensionBundle.getCreated(),
                 extensionBundle.getModified(),
-                extensionBundle.getType().toString(),
+                extensionBundle.getType().name(),
                 extensionBundle.getBucketId());
 
         final String bundleSql =
-                "INSERT INTO extension_bundle (" +
+                "INSERT INTO bundle (" +
                     "ID, " +
                     "BUCKET_ID, " +
                     "BUNDLE_TYPE, " +
@@ -449,21 +449,38 @@ public class DatabaseMetadataService implements MetadataService {
         jdbcTemplate.update(bundleSql,
                 extensionBundle.getId(),
                 extensionBundle.getBucketId(),
-                extensionBundle.getBundleType().toString(),
+                extensionBundle.getBundleType().name(),
                 extensionBundle.getGroupId(),
                 extensionBundle.getArtifactId());
 
         return extensionBundle;
     }
 
+    private static final String BASE_BUNDLE_SQL =
+            "SELECT " +
+                "item.id as ID," +
+                "item.name as NAME, " +
+                "item.description as DESCRIPTION, " +
+                "item.created as CREATED, " +
+                "item.modified as MODIFIED, " +
+                "eb.bundle_type as BUNDLE_TYPE, " +
+                "eb.group_id as GROUP_ID, " +
+                "eb.artifact_id as ARTIFACT_ID, " +
+                "b.id as BUCKET_ID, " +
+                "b.name as BUCKET_NAME " +
+            "FROM " +
+                "bundle eb, " +
+                "bucket_item item," +
+                "bucket b " +
+            "WHERE " +
+                "eb.id = item.id AND " +
+                "item.bucket_id = b.id";
+
     @Override
-    public ExtensionBundleEntity getExtensionBundle(final String extensionBundleId) {
-        final String sql =
-                "SELECT * " +
-                "FROM extension_bundle eb, bucket_item item " +
-                "WHERE eb.id = ? AND item.id = eb.id";
+    public BundleEntity getBundle(final String extensionBundleId) {
+        final StringBuilder sqlBuilder = new StringBuilder(BASE_BUNDLE_SQL).append(" AND eb.id = ?");
         try {
-            final ExtensionBundleEntity entity = jdbcTemplate.queryForObject(sql, new ExtensionBundleEntityRowMapper(), extensionBundleId);
+            final BundleEntity entity = jdbcTemplate.queryForObject(sqlBuilder.toString(), new BundleEntityRowMapper(), extensionBundleId);
 
             final Long versionCount = getExtensionBundleVersionCount(extensionBundleId);
             if (versionCount != null) {
@@ -477,19 +494,14 @@ public class DatabaseMetadataService implements MetadataService {
     }
 
     @Override
-    public ExtensionBundleEntity getExtensionBundle(final String bucketId, final String groupId, final String artifactId) {
-        final String sql =
-                "SELECT * " +
-                "FROM " +
-                        "extension_bundle eb, " +
-                        "bucket_item item " +
-                "WHERE " +
-                        "item.id = eb.id AND " +
-                        "eb.bucket_id = ? AND " +
-                        "eb.group_id = ? AND " +
-                        "eb.artifact_id = ?";
+    public BundleEntity getBundle(final String bucketId, final String groupId, final String artifactId) {
+        final StringBuilder sqlBuilder = new StringBuilder(BASE_BUNDLE_SQL)
+                .append(" AND eb.bucket_id = ? ")
+                .append("AND eb.group_id = ? ")
+                .append("AND eb.artifact_id = ? ");
+
         try {
-            final ExtensionBundleEntity entity = jdbcTemplate.queryForObject(sql, new ExtensionBundleEntityRowMapper(), bucketId, groupId, artifactId);
+            final BundleEntity entity = jdbcTemplate.queryForObject(sqlBuilder.toString(), new BundleEntityRowMapper(), bucketId, groupId, artifactId);
 
             final Long versionCount = getExtensionBundleVersionCount(entity.getId());
             if (versionCount != null) {
@@ -503,7 +515,7 @@ public class DatabaseMetadataService implements MetadataService {
     }
 
     @Override
-    public List<ExtensionBundleEntity> getExtensionBundles(final Set<String> bucketIds, final ExtensionBundleFilterParams filterParams) {
+    public List<BundleEntity> getBundles(final Set<String> bucketIds, final BundleFilterParams filterParams) {
         if (bucketIds == null || bucketIds.isEmpty()) {
             return Collections.emptyList();
         }
@@ -512,19 +524,19 @@ public class DatabaseMetadataService implements MetadataService {
 
         final StringBuilder sqlBuilder = new StringBuilder(
                 "SELECT " +
-                        "item.id as ID, " +
-                        "item.name as NAME, " +
-                        "item.description as DESCRIPTION, " +
-                        "item.created as CREATED, " +
-                        "item.modified as MODIFIED, " +
-                        "item.item_type as ITEM_TYPE, " +
-                        "b.id as BUCKET_ID, " +
-                        "b.name as BUCKET_NAME ," +
-                        "eb.bundle_type as BUNDLE_TYPE, " +
-                        "eb.group_id as BUNDLE_GROUP_ID, " +
-                        "eb.artifact_id as BUNDLE_ARTIFACT_ID " +
+                    "item.id as ID, " +
+                    "item.name as NAME, " +
+                    "item.description as DESCRIPTION, " +
+                    "item.created as CREATED, " +
+                    "item.modified as MODIFIED, " +
+                    "item.item_type as ITEM_TYPE, " +
+                    "b.id as BUCKET_ID, " +
+                    "b.name as BUCKET_NAME ," +
+                    "eb.bundle_type as BUNDLE_TYPE, " +
+                    "eb.group_id as BUNDLE_GROUP_ID, " +
+                    "eb.artifact_id as BUNDLE_ARTIFACT_ID " +
                 "FROM " +
-                    "extension_bundle eb, " +
+                    "bundle eb, " +
                     "bucket_item item, " +
                     "bucket b " +
                 "WHERE " +
@@ -532,6 +544,12 @@ public class DatabaseMetadataService implements MetadataService {
                     "b.id = item.bucket_id");
 
         if (filterParams != null) {
+            final String bucketName = filterParams.getBucketName();
+            if (!StringUtils.isBlank(bucketName)) {
+                sqlBuilder.append(" AND b.name LIKE ? ");
+                args.add(bucketName);
+            }
+
             final String groupId = filterParams.getGroupId();
             if (!StringUtils.isBlank(groupId)) {
                 sqlBuilder.append(" AND eb.group_id LIKE ? ");
@@ -545,52 +563,41 @@ public class DatabaseMetadataService implements MetadataService {
             }
         }
 
-        addBucketIdentifiersClause(sqlBuilder, "item.bucket_id", bucketIds);
+        sqlBuilder.append(" AND ");
+        addIdentifiersInClause(sqlBuilder, "item.bucket_id", bucketIds);
         sqlBuilder.append("ORDER BY eb.group_id ASC, eb.artifact_id ASC");
 
         args.addAll(bucketIds);
 
-        final List<ExtensionBundleEntity> bundleEntities = jdbcTemplate.query(sqlBuilder.toString(), args.toArray(), new ExtensionBundleEntityWithBucketNameRowMapper());
+        final List<BundleEntity> bundleEntities = jdbcTemplate.query(sqlBuilder.toString(), args.toArray(), new BundleEntityRowMapper());
         return populateVersionCounts(bundleEntities);
     }
 
     @Override
-    public List<ExtensionBundleEntity> getExtensionBundlesByBucket(final String bucketId) {
-        final String sql =
-                "SELECT * " +
-                "FROM " +
-                    "extension_bundle eb, " +
-                    "bucket_item item " +
-                "WHERE " +
-                    "item.id = eb.id AND " +
-                    "item.bucket_id = ? " +
-                    "ORDER BY eb.group_id ASC, eb.artifact_id ASC";
+    public List<BundleEntity> getBundlesByBucket(final String bucketId) {
+        final StringBuilder sqlBuilder = new StringBuilder(BASE_BUNDLE_SQL)
+                .append(" AND b.id = ?")
+                .append(" ORDER BY eb.group_id ASC, eb.artifact_id ASC");
 
-        final List<ExtensionBundleEntity> bundles = jdbcTemplate.query(sql, new Object[]{bucketId}, new ExtensionBundleEntityRowMapper());
+        final List<BundleEntity> bundles = jdbcTemplate.query(sqlBuilder.toString(), new Object[]{bucketId}, new BundleEntityRowMapper());
         return populateVersionCounts(bundles);
     }
 
     @Override
-    public List<ExtensionBundleEntity> getExtensionBundlesByBucketAndGroup(String bucketId, String groupId) {
-        final String sql =
-                "SELECT * " +
-                    "FROM " +
-                        "extension_bundle eb, " +
-                        "bucket_item item " +
-                    "WHERE " +
-                        "item.id = eb.id AND " +
-                        "item.bucket_id = ? AND " +
-                        "eb.group_id = ?" +
-                    "ORDER BY eb.group_id ASC, eb.artifact_id ASC";
+    public List<BundleEntity> getBundlesByBucketAndGroup(String bucketId, String groupId) {
+        final StringBuilder sqlBuilder = new StringBuilder(BASE_BUNDLE_SQL)
+                .append(" AND b.id = ?")
+                .append(" AND eb.group_id = ?")
+                .append(" ORDER BY eb.group_id ASC, eb.artifact_id ASC");
 
-        final List<ExtensionBundleEntity> bundles = jdbcTemplate.query(sql, new Object[]{bucketId, groupId}, new ExtensionBundleEntityRowMapper());
+        final List<BundleEntity> bundles = jdbcTemplate.query(sqlBuilder.toString(), new Object[]{bucketId, groupId}, new BundleEntityRowMapper());
         return populateVersionCounts(bundles);
     }
 
-    private List<ExtensionBundleEntity> populateVersionCounts(final List<ExtensionBundleEntity> bundles) {
+    private List<BundleEntity> populateVersionCounts(final List<BundleEntity> bundles) {
         if (!bundles.isEmpty()) {
             final Map<String, Long> versionCounts = getExtensionBundleVersionCounts();
-            for (final ExtensionBundleEntity entity : bundles) {
+            for (final BundleEntity entity : bundles) {
                 final Long versionCount = versionCounts.get(entity.getId());
                 if (versionCount != null) {
                     entity.setVersionCount(versionCount);
@@ -602,12 +609,12 @@ public class DatabaseMetadataService implements MetadataService {
     }
 
     @Override
-    public void deleteExtensionBundle(final ExtensionBundleEntity extensionBundle) {
-        deleteExtensionBundle(extensionBundle.getId());
+    public void deleteBundle(final BundleEntity extensionBundle) {
+        deleteBundle(extensionBundle.getId());
     }
 
     @Override
-    public void deleteExtensionBundle(final String extensionBundleId) {
+    public void deleteBundle(final String extensionBundleId) {
         // NOTE: All of the foreign key constraints for extension related tables are set to cascade on delete
         final String itemDeleteSql = "DELETE FROM bucket_item WHERE id = ?";
         jdbcTemplate.update(itemDeleteSql, extensionBundleId);
@@ -616,30 +623,46 @@ public class DatabaseMetadataService implements MetadataService {
     //----------------- Extension Bundle Versions ---------------------------------
 
     @Override
-    public ExtensionBundleVersionEntity createExtensionBundleVersion(final ExtensionBundleVersionEntity extensionBundleVersion) {
+    public BundleVersionEntity createBundleVersion(final BundleVersionEntity extensionBundleVersion) {
         final String sql =
-                "INSERT INTO extension_bundle_version (" +
+                "INSERT INTO bundle_version (" +
                     "ID, " +
-                    "EXTENSION_BUNDLE_ID, " +
+                    "BUNDLE_ID, " +
                     "VERSION, " +
                     "CREATED, " +
                     "CREATED_BY, " +
                     "DESCRIPTION, " +
                     "SHA_256_HEX, " +
                     "SHA_256_SUPPLIED," +
-                    "CONTENT_SIZE " +
-                ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    "CONTENT_SIZE, " +
+                    "SYSTEM_API_VERSION, " +
+                    "BUILD_TOOL, " +
+                    "BUILD_FLAGS, " +
+                    "BUILD_BRANCH, " +
+                    "BUILD_TAG, " +
+                    "BUILD_REVISION, " +
+                    "BUILT, " +
+                    "BUILT_BY" +
+                ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
         jdbcTemplate.update(sql,
                 extensionBundleVersion.getId(),
-                extensionBundleVersion.getExtensionBundleId(),
+                extensionBundleVersion.getBundleId(),
                 extensionBundleVersion.getVersion(),
                 extensionBundleVersion.getCreated(),
                 extensionBundleVersion.getCreatedBy(),
                 extensionBundleVersion.getDescription(),
                 extensionBundleVersion.getSha256Hex(),
                 extensionBundleVersion.getSha256Supplied() ? 1 : 0,
-                extensionBundleVersion.getContentSize());
+                extensionBundleVersion.getContentSize(),
+                extensionBundleVersion.getSystemApiVersion(),
+                extensionBundleVersion.getBuildTool(),
+                extensionBundleVersion.getBuildFlags(),
+                extensionBundleVersion.getBuildBranch(),
+                extensionBundleVersion.getBuildTag(),
+                extensionBundleVersion.getBuildRevision(),
+                extensionBundleVersion.getBuilt(),
+                extensionBundleVersion.getBuiltBy());
 
         return extensionBundleVersion;
     }
@@ -647,7 +670,7 @@ public class DatabaseMetadataService implements MetadataService {
     private static final String BASE_EXTENSION_BUNDLE_VERSION_SQL =
             "SELECT " +
                 "ebv.id AS ID," +
-                "ebv.extension_bundle_id AS EXTENSION_BUNDLE_ID, " +
+                "ebv.bundle_id AS BUNDLE_ID, " +
                 "ebv.version AS VERSION, " +
                 "ebv.created AS CREATED, " +
                 "ebv.created_by AS CREATED_BY, " +
@@ -655,23 +678,31 @@ public class DatabaseMetadataService implements MetadataService {
                 "ebv.sha_256_hex AS SHA_256_HEX, " +
                 "ebv.sha_256_supplied AS SHA_256_SUPPLIED ," +
                 "ebv.content_size AS CONTENT_SIZE, " +
+                "ebv.system_api_version AS SYSTEM_API_VERSION, " +
+                "ebv.build_tool AS BUILD_TOOL, " +
+                "ebv.build_flags AS BUILD_FLAGS, " +
+                "ebv.build_branch AS BUILD_BRANCH, " +
+                "ebv.build_tag AS BUILD_TAG, " +
+                "ebv.build_revision AS BUILD_REVISION, " +
+                "ebv.built AS BUILT, " +
+                "ebv.built_by AS BUILT_BY, " +
                 "eb.bucket_id AS BUCKET_ID " +
-            "FROM extension_bundle eb, extension_bundle_version ebv " +
-            "WHERE eb.id = ebv.extension_bundle_id ";
+            "FROM bundle eb, bundle_version ebv " +
+            "WHERE eb.id = ebv.bundle_id ";
 
     @Override
-    public ExtensionBundleVersionEntity getExtensionBundleVersion(final String extensionBundleId, final String version) {
+    public BundleVersionEntity getBundleVersion(final String extensionBundleId, final String version) {
         final String sql = BASE_EXTENSION_BUNDLE_VERSION_SQL +
-                " AND ebv.extension_bundle_id = ? AND ebv.version = ?";
+                " AND ebv.bundle_id = ? AND ebv.version = ?";
         try {
-            return jdbcTemplate.queryForObject(sql, new ExtensionBundleVersionEntityRowMapper(), extensionBundleId, version);
+            return jdbcTemplate.queryForObject(sql, new BundleVersionEntityRowMapper(), extensionBundleId, version);
         } catch (EmptyResultDataAccessException e) {
             return null;
         }
     }
 
     @Override
-    public ExtensionBundleVersionEntity getExtensionBundleVersion(final String bucketId, final String groupId, final String artifactId, final String version) {
+    public BundleVersionEntity getBundleVersion(final String bucketId, final String groupId, final String artifactId, final String version) {
         final String sql = BASE_EXTENSION_BUNDLE_VERSION_SQL +
                     "AND eb.bucket_id = ? " +
                     "AND eb.group_id = ? " +
@@ -679,14 +710,14 @@ public class DatabaseMetadataService implements MetadataService {
                     "AND ebv.version = ?";
 
         try {
-            return jdbcTemplate.queryForObject(sql, new ExtensionBundleVersionEntityRowMapper(), bucketId, groupId, artifactId, version);
+            return jdbcTemplate.queryForObject(sql, new BundleVersionEntityRowMapper(), bucketId, groupId, artifactId, version);
         } catch (EmptyResultDataAccessException e) {
             return null;
         }
     }
 
     @Override
-    public List<ExtensionBundleVersionEntity> getExtensionBundleVersions(final Set<String> bucketIdentifiers, final ExtensionBundleVersionFilterParams filterParams) {
+    public List<BundleVersionEntity> getBundleVersions(final Set<String> bucketIdentifiers, final BundleVersionFilterParams filterParams) {
         if (bucketIdentifiers == null || bucketIdentifiers.isEmpty()) {
             return Collections.emptyList();
         }
@@ -714,18 +745,19 @@ public class DatabaseMetadataService implements MetadataService {
             }
         }
 
-        addBucketIdentifiersClause(sqlBuilder, "eb.bucket_id", bucketIdentifiers);
+        sqlBuilder.append(" AND ");
+        addIdentifiersInClause(sqlBuilder, "eb.bucket_id", bucketIdentifiers);
         args.addAll(bucketIdentifiers);
 
-        final List<ExtensionBundleVersionEntity> bundleVersionEntities = jdbcTemplate.query(
-                sqlBuilder.toString(), args.toArray(), new ExtensionBundleVersionEntityRowMapper());
+        final List<BundleVersionEntity> bundleVersionEntities = jdbcTemplate.query(
+                sqlBuilder.toString(), args.toArray(), new BundleVersionEntityRowMapper());
 
         return bundleVersionEntities;
     }
 
-    private void addBucketIdentifiersClause(StringBuilder sqlBuilder, String bucketField, Set<String> bucketIdentifiers) {
-        sqlBuilder.append(" AND ").append(bucketField).append(" IN (");
-        for (int i = 0; i < bucketIdentifiers.size(); i++) {
+    private void addIdentifiersInClause(StringBuilder sqlBuilder, String idFieldName, Set<String> identifiers) {
+        sqlBuilder.append(idFieldName).append(" IN (");
+        for (int i = 0; i < identifiers.size(); i++) {
             if (i > 0) {
                 sqlBuilder.append(", ");
             }
@@ -735,53 +767,53 @@ public class DatabaseMetadataService implements MetadataService {
     }
 
     @Override
-    public List<ExtensionBundleVersionEntity> getExtensionBundleVersions(final String extensionBundleId) {
-        final String sql = BASE_EXTENSION_BUNDLE_VERSION_SQL + " AND ebv.extension_bundle_id = ?";
-        return jdbcTemplate.query(sql, new Object[]{extensionBundleId}, new ExtensionBundleVersionEntityRowMapper());
+    public List<BundleVersionEntity> getBundleVersions(final String extensionBundleId) {
+        final String sql = BASE_EXTENSION_BUNDLE_VERSION_SQL + " AND ebv.bundle_id = ?";
+        return jdbcTemplate.query(sql, new Object[]{extensionBundleId}, new BundleVersionEntityRowMapper());
     }
 
     @Override
-    public List<ExtensionBundleVersionEntity> getExtensionBundleVersions(final String bucketId, final String groupId, final String artifactId) {
+    public List<BundleVersionEntity> getBundleVersions(final String bucketId, final String groupId, final String artifactId) {
         final String sql = BASE_EXTENSION_BUNDLE_VERSION_SQL +
                     "AND eb.bucket_id = ? " +
                     "AND eb.group_id = ? " +
                     "AND eb.artifact_id = ? ";
 
         final Object[] args = {bucketId, groupId, artifactId};
-        return jdbcTemplate.query(sql, args, new ExtensionBundleVersionEntityRowMapper());
+        return jdbcTemplate.query(sql, args, new BundleVersionEntityRowMapper());
     }
... 9542 lines suppressed ...