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 2018/06/04 21:54:48 UTC

nifi-registry git commit: NIFIREG-173 Refactor metadata DB to be independent of H2

Repository: nifi-registry
Updated Branches:
  refs/heads/master 3d765be3d -> 44bc4adb1


NIFIREG-173 Refactor metadata DB to be independent of H2

- Adding a property to specify an external directory containing driver jars and adding it to the web-api WAR's classpath during startup
- Updating admin guide

This closes #121.

Signed-off-by: Kevin Doran <kd...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/nifi-registry/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi-registry/commit/44bc4adb
Tree: http://git-wip-us.apache.org/repos/asf/nifi-registry/tree/44bc4adb
Diff: http://git-wip-us.apache.org/repos/asf/nifi-registry/diff/44bc4adb

Branch: refs/heads/master
Commit: 44bc4adb12ecb333a64fe2a72a957912ce3b9ba5
Parents: 3d765be
Author: Bryan Bende <bb...@apache.org>
Authored: Wed May 30 14:31:26 2018 -0400
Committer: Kevin Doran <kd...@apache.org>
Committed: Mon Jun 4 17:54:34 2018 -0400

----------------------------------------------------------------------
 nifi-registry-assembly/pom.xml                  |  13 +-
 .../src/main/asciidoc/administration-guide.adoc |  20 +++
 .../db/CustomFlywayMigrationStrategy.java       | 147 +++++++++++++++++++
 .../nifi/registry/db/DataSourceFactory.java     |  80 +++++-----
 .../registry/db/migration/BucketEntityV1.java   |  86 +++++++++++
 .../registry/db/migration/FlowEntityV1.java     | 106 +++++++++++++
 .../db/migration/FlowSnapshotEntityV1.java      |  96 ++++++++++++
 .../db/migration/LegacyDataSourceFactory.java   |  81 ++++++++++
 .../db/migration/LegacyDatabaseService.java     |  77 ++++++++++
 .../db/migration/LegacyEntityMapper.java        |  63 ++++++++
 .../db/migration/V1.2__IncreaseColumnSizes.sql  |  25 ----
 .../V1.3__DropBucketItemNameUniqueness.sql      |  27 ----
 .../main/resources/db/migration/V1__Initial.sql |  54 -------
 .../main/resources/db/migration/V2__Initial.sql |  60 ++++++++
 .../db/original/V1.2__IncreaseColumnSizes.sql   |  25 ++++
 .../V1.3__DropBucketItemNameUniqueness.sql      |  27 ++++
 .../main/resources/db/original/V1__Initial.sql  |  54 +++++++
 .../registry/db/DatabaseTestApplication.java    |   7 +
 .../db/migration/TestLegacyDatabaseService.java | 141 ++++++++++++++++++
 .../db/migration/TestLegacyEntityMapper.java    |  79 ++++++++++
 .../apache/nifi/registry/jetty/JettyServer.java |  81 +++++++++-
 .../properties/NiFiRegistryProperties.java      |  48 +++++-
 .../resources/conf/nifi-registry.properties     |  12 +-
 23 files changed, 1260 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-assembly/pom.xml b/nifi-registry-assembly/pom.xml
index cbf2bdc..130241b 100644
--- a/nifi-registry-assembly/pom.xml
+++ b/nifi-registry-assembly/pom.xml
@@ -160,9 +160,18 @@
         <!-- nifi-registry.properties: provider properties -->
         <nifi.registry.providers.configuration.file>./conf/providers.xml</nifi.registry.providers.configuration.file>
 
+        <!-- nifi-registry.properties: legacy database properties, used to migrate data from old DB to the new DB below -->
+        <nifi.registry.db.directory></nifi.registry.db.directory>
+        <nifi.registry.db.url.append></nifi.registry.db.url.append>
+
         <!-- nifi-registry.properties: database properties -->
-        <nifi.registry.db.directory>./database</nifi.registry.db.directory>
-        <nifi.registry.db.url.append>;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE</nifi.registry.db.url.append>
+        <nifi.registry.db.url>jdbc:h2:./database/nifi-registry-primary;AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE</nifi.registry.db.url>
+        <nifi.registry.db.driver.class>org.h2.Driver</nifi.registry.db.driver.class>
+        <nifi.registry.db.driver.directory></nifi.registry.db.driver.directory>
+        <nifi.registry.db.username>nifireg</nifi.registry.db.username>
+        <nifi.registry.db.password>nifireg</nifi.registry.db.password>
+        <nifi.registry.db.maxConnections>5</nifi.registry.db.maxConnections>
+        <nifi.registry.db.sql.debug>false</nifi.registry.db.sql.debug>
 
         <!-- nifi-registry.properties: kerberos properties -->
         <nifi.registry.kerberos.krb5.file />

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-registry-docs/src/main/asciidoc/administration-guide.adoc b/nifi-registry-docs/src/main/asciidoc/administration-guide.adoc
index 93a75c1..b4d213c 100644
--- a/nifi-registry-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-registry-docs/src/main/asciidoc/administration-guide.adoc
@@ -867,12 +867,32 @@ content of the flows saved to the registry. For further details on persistence p
 
 These properties define the settings for the Registry database, which keeps track of metadata about buckets and all items stored in buckets.
 
+The 0.1.0 release leveraged an embedded H2 database that was configured via the following properties:
+
 |====
 |*Property*|*Description*
 |nifi.registry.db.directory|The location of the Registry database directory. The default value is `./database`.
 |nifi.registry.db.url.append|This property specifies additional arguments to add to the connection string for the Registry database. The default value should be used and should not be changed. It is: `;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE`.
 |====
 
+The 0.2.0 release introduced a more flexible approach which allows leveraging an external database. This new approach
+is configured via the following properties:
+
+|====
+|*Property*|*Description*
+|nifi.registry.db.url| The full JDBC connection string. The default value will specify a new H2 database in the same location as the previous one. For example, 'jdbc:h2:./database/nifi-registry-primary;'.
+|nifi.registry.db.driver.class| The class name of the JDBC driver. The default value is 'org.h2.Driver'.
+|nifi.registry.db.driver.directory| An optional directory containing one or more JARs to add to the classpath. If not specified, it is assumed that the driver JAR is already on the classpath by copying it to the lib directory. The H2 driver is bundled with Registry so it is not necessary to do anything for the default case.
+|nifi.registry.db.driver.username| The username for the database. The default value is 'nifireg'.
+|nifi.registry.db.driver.password| The password for the database. The default value is 'nifireg'.
+|nifi.registry.db.driver.maxConnections| The max number of connections for the connection pool. The default value is '5'.
+|nifi.registry.db.sql.debug| Whether or not enable debug logging for SQL statements. The default value is 'false'.
+|====
+
+NOTE: When upgrading from 0.1.0 to a future version, if 'nifi.registry.db.directory' remains populated, the application will
+attempt to migrate the data from the original database to the new database specified with the new properties. This will only
+happen the first time the application starts with the new database properties.
+
 === Extension Directories
 
 Each property beginning with "nifi.registry.extension.dir." will be treated as location for an extension, and a class loader will be created for each location, with the system class loader as the parent.

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/CustomFlywayMigrationStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/CustomFlywayMigrationStrategy.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/CustomFlywayMigrationStrategy.java
new file mode 100644
index 0000000..376de6a
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/CustomFlywayMigrationStrategy.java
@@ -0,0 +1,147 @@
+/*
+ * 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.db;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.db.migration.BucketEntityV1;
+import org.apache.nifi.registry.db.migration.FlowEntityV1;
+import org.apache.nifi.registry.db.migration.FlowSnapshotEntityV1;
+import org.apache.nifi.registry.db.migration.LegacyDataSourceFactory;
+import org.apache.nifi.registry.db.migration.LegacyDatabaseService;
+import org.apache.nifi.registry.db.migration.LegacyEntityMapper;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.service.MetadataService;
+import org.flywaydb.core.Flyway;
+import org.flywaydb.core.api.FlywayException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.flyway.FlywayMigrationStrategy;
+import org.springframework.jdbc.core.JdbcTemplate;
+import org.springframework.stereotype.Component;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+
+/**
+ * Custom Flyway migration strategy that lets us perform data migration from the original database used in the
+ * 0.1.0 release, to the new database. The data migration will be triggered when it is determined that new database
+ * is brand new AND the legacy DB properties are specified. If the primary database already contains the 'BUCKET' table,
+ * or if the legacy database properties are not specified, then no data migration is performed.
+ */
+@Component
+public class CustomFlywayMigrationStrategy implements FlywayMigrationStrategy {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(CustomFlywayMigrationStrategy.class);
+
+    private NiFiRegistryProperties properties;
+
+    @Autowired
+    public CustomFlywayMigrationStrategy(final NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    @Override
+    public void migrate(Flyway flyway) {
+        final boolean newDatabase = isNewDatabase(flyway.getDataSource());
+        if (newDatabase) {
+            LOGGER.info("First time initializing database...");
+        } else {
+            LOGGER.info("Found existing database...");
+        }
+
+        final boolean existingLegacyDatabase = !StringUtils.isBlank(properties.getLegacyDatabaseDirectory());
+        if (existingLegacyDatabase) {
+            LOGGER.info("Found legacy database properties...");
+        }
+
+        // If newDatabase is true, then we need to run the Flyway migration first to create all the tables, then the data migration
+        // If newDatabase is false, then we need to run the Flyway migration to run any schema updates, but no data migration
+
+        flyway.migrate();
+
+        if (newDatabase && existingLegacyDatabase) {
+            final LegacyDataSourceFactory legacyDataSourceFactory = new LegacyDataSourceFactory(properties);
+            final DataSource legacyDataSource = legacyDataSourceFactory.getDataSource();
+            final DataSource primaryDataSource = flyway.getDataSource();
+            migrateData(legacyDataSource, primaryDataSource);
+        }
+    }
+
+    /**
+     * Determines if the database represented by this data source is being initialized for the first time based on
+     * whether or not the table named 'BUCKET' or 'bucket' already exists.
+     *
+     * @param dataSource the data source
+     * @return true if the database has never been initialized before, false otherwise
+     */
+    private boolean isNewDatabase(final DataSource dataSource) {
+        try (final Connection connection = dataSource.getConnection();
+             final ResultSet rsUpper = connection.getMetaData().getTables(null, null, "BUCKET", null);
+             final ResultSet rsLower = connection.getMetaData().getTables(null, null, "bucket", null)) {
+            return !rsUpper.next() && !rsLower.next();
+        } catch (SQLException e) {
+            LOGGER.error(e.getMessage(), e);
+            throw new FlywayException("Unable to obtain connection from Flyway DataSource", e);
+        }
+    }
+
+    /**
+     * Transfers all data from the source to the destination.
+     *
+     * @param source the legacy H2 DataSource
+     * @param dest the new destination DataSource
+     */
+    private void migrateData(final DataSource source, final DataSource dest) {
+        final LegacyDatabaseService legacyDatabaseService = new LegacyDatabaseService(source);
+
+        final JdbcTemplate destJdbcTemplate = new JdbcTemplate(dest);
+        final MetadataService destMetadataService = new DatabaseMetadataService(destJdbcTemplate);
+
+        LOGGER.info("Migrating data from legacy database to new new database...");
+
+        // Migrate buckets
+        final List<BucketEntityV1> sourceBuckets = legacyDatabaseService.getAllBuckets();
+        LOGGER.info("Migrating {} buckets..", new Object[]{sourceBuckets.size()});
+
+        sourceBuckets.stream()
+                .map(b -> LegacyEntityMapper.createBucketEntity(b))
+                .forEach(b -> destMetadataService.createBucket(b));
+
+        // Migrate flows
+        final List<FlowEntityV1> sourceFlows = legacyDatabaseService.getAllFlows();
+        LOGGER.info("Migrating {} flows..", new Object[]{sourceFlows.size()});
+
+        sourceFlows.stream()
+                .map(f -> LegacyEntityMapper.createFlowEntity(f))
+                .forEach(f -> destMetadataService.createFlow(f));
+
+        // Migrate flow snapshots
+        final List<FlowSnapshotEntityV1> sourceSnapshots = legacyDatabaseService.getAllFlowSnapshots();
+        LOGGER.info("Migrating {} flow snapshots..", new Object[]{sourceSnapshots.size()});
+
+        sourceSnapshots.stream()
+                .map(fs -> LegacyEntityMapper.createFlowSnapshotEntity(fs))
+                .forEach(fs -> destMetadataService.createFlowSnapshot(fs));
+
+        LOGGER.info("Data migration complete!");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DataSourceFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DataSourceFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DataSourceFactory.java
index 5ec2043..29c132e 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DataSourceFactory.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DataSourceFactory.java
@@ -16,16 +16,18 @@
  */
 package org.apache.nifi.registry.db;
 
+import com.zaxxer.hikari.HikariDataSource;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.h2.jdbcx.JdbcConnectionPool;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.jdbc.DataSourceBuilder;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
 import org.springframework.context.annotation.Primary;
 
 import javax.sql.DataSource;
-import java.io.File;
 
 /**
  * Overriding Spring Boot's normal automatic creation of a DataSource in order to use the properties
@@ -34,15 +36,11 @@ import java.io.File;
 @Configuration
 public class DataSourceFactory {
 
-    private static final String DB_USERNAME_PASSWORD = "nifireg";
-    private static final int MAX_CONNECTIONS = 5;
-
-    // database file name
-    private static final String DATABASE_FILE_NAME = "nifi-registry";
+    private static final Logger LOGGER = LoggerFactory.getLogger(DataSourceFactory.class);
 
     private final NiFiRegistryProperties properties;
 
-    private JdbcConnectionPool connectionPool;
+    private DataSource dataSource;
 
     @Autowired
     public DataSourceFactory(final NiFiRegistryProperties properties) {
@@ -52,34 +50,48 @@ public class DataSourceFactory {
     @Bean
     @Primary
     public DataSource getDataSource() {
-        if (connectionPool == null) {
-            // locate the repository directory
-            final String repositoryDirectoryPath = properties.getDatabaseDirectory();
-
-            // ensure the repository directory is specified
-            if (repositoryDirectoryPath == null) {
-                throw new NullPointerException("Database directory must be specified.");
-            }
-
-            // create a handle to the repository directory
-            final File repositoryDirectory = new File(repositoryDirectoryPath);
-
-            // get a handle to the database file
-            final File databaseFile = new File(repositoryDirectory, DATABASE_FILE_NAME);
-
-            // format the database url
-            String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3";
-            String databaseUrlAppend = properties.getDatabaseUrlAppend();
-            if (StringUtils.isNotBlank(databaseUrlAppend)) {
-                databaseUrl += databaseUrlAppend;
-            }
-
-            // create the pool
-            connectionPool = JdbcConnectionPool.create(databaseUrl, DB_USERNAME_PASSWORD, DB_USERNAME_PASSWORD);
-            connectionPool.setMaxConnections(MAX_CONNECTIONS);
+        if (dataSource == null) {
+            dataSource = createDataSource();
+        }
+
+        return dataSource;
+    }
+
+    private DataSource createDataSource() {
+        final String databaseUrl = properties.getDatabaseUrl();
+        if (StringUtils.isBlank(databaseUrl)) {
+            throw new IllegalStateException(NiFiRegistryProperties.DATABASE_URL + " is required");
+        }
+
+        final String databaseDriver = properties.getDatabaseDriverClassName();
+        if (StringUtils.isBlank(databaseDriver)) {
+            throw new IllegalStateException(NiFiRegistryProperties.DATABASE_DRIVER_CLASS_NAME + " is required");
+        }
+
+        final String databaseUsername = properties.getDatabaseUsername();
+        if (StringUtils.isBlank(databaseUsername)) {
+            throw new IllegalStateException(NiFiRegistryProperties.DATABASE_USERNAME + " is required");
+        }
+
+        String databasePassword = properties.getDatabasePassword();
+        if (StringUtils.isBlank(databasePassword)) {
+            throw new IllegalStateException(NiFiRegistryProperties.DATABASE_PASSWORD + " is required");
+        }
+
+        final DataSource dataSource = DataSourceBuilder
+                .create()
+                .url(databaseUrl)
+                .driverClassName(databaseDriver)
+                .username(databaseUsername)
+                .password(databasePassword)
+                .build();
+
+        if (dataSource instanceof HikariDataSource) {
+            LOGGER.info("Setting maximum pool size on HikariDataSource to {}", new Object[]{properties.getDatabaseMaxConnections()});
+            ((HikariDataSource)dataSource).setMaximumPoolSize(properties.getDatabaseMaxConnections());
         }
 
-        return connectionPool;
+        return dataSource;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/BucketEntityV1.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/BucketEntityV1.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/BucketEntityV1.java
new file mode 100644
index 0000000..94000a5
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/BucketEntityV1.java
@@ -0,0 +1,86 @@
+/*
+ * 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.db.migration;
+
+import java.util.Date;
+import java.util.Objects;
+
+/**
+ * Bucket DB entity from the original database schema in 0.1.0, used for migration purposes.
+ */
+public class BucketEntityV1 {
+
+    private String id;
+
+    private String name;
+
+    private String description;
+
+    private Date created;
+
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public Date getCreated() {
+        return created;
+    }
+
+    public void setCreated(Date created) {
+        this.created = created;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(this.id);
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        final BucketEntityV1 other = (BucketEntityV1) obj;
+        return Objects.equals(this.id, other.id);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowEntityV1.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowEntityV1.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowEntityV1.java
new file mode 100644
index 0000000..961c3bd
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowEntityV1.java
@@ -0,0 +1,106 @@
+/*
+ * 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.db.migration;
+
+import java.util.Date;
+import java.util.Objects;
+
+/**
+ * Flow DB entity from the original database schema in 0.1.0, used for migration purposes.
+ */
+public class FlowEntityV1 {
+
+    private String id;
+
+    private String name;
+
+    private String description;
+
+    private Date created;
+
+    private Date modified;
+
+    private String bucketId;
+
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    public Date getCreated() {
+        return created;
+    }
+
+    public void setCreated(Date created) {
+        this.created = created;
+    }
+
+    public Date getModified() {
+        return modified;
+    }
+
+    public void setModified(Date modified) {
+        this.modified = modified;
+    }
+
+    public String getBucketId() {
+        return bucketId;
+    }
+
+    public void setBucketId(String bucketId) {
+        this.bucketId = bucketId;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(this.id);
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        final FlowEntityV1 other = (FlowEntityV1) obj;
+        return Objects.equals(this.id, other.id);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowSnapshotEntityV1.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowSnapshotEntityV1.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowSnapshotEntityV1.java
new file mode 100644
index 0000000..ec6b9a5
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/FlowSnapshotEntityV1.java
@@ -0,0 +1,96 @@
+/*
+ * 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.db.migration;
+
+import java.util.Date;
+import java.util.Objects;
+
+/**
+ * FlowSnapshot DB entity from the original database schema in 0.1.0, used for migration purposes.
+ */
+public class FlowSnapshotEntityV1 {
+
+    private String flowId;
+
+    private Integer version;
+
+    private Date created;
+
+    private String createdBy;
+
+    private String comments;
+
+    public String getFlowId() {
+        return flowId;
+    }
+
+    public void setFlowId(String flowId) {
+        this.flowId = flowId;
+    }
+
+    public Integer getVersion() {
+        return version;
+    }
+
+    public void setVersion(Integer version) {
+        this.version = version;
+    }
+
+    public Date getCreated() {
+        return created;
+    }
+
+    public void setCreated(Date created) {
+        this.created = created;
+    }
+
+    public String getCreatedBy() {
+        return createdBy;
+    }
+
+    public void setCreatedBy(String createdBy) {
+        this.createdBy = createdBy;
+    }
+
+    public String getComments() {
+        return comments;
+    }
+
+    public void setComments(String comments) {
+        this.comments = comments;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(this.flowId, this.version);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+
+        if (!(obj instanceof FlowSnapshotEntityV1)) {
+            return false;
+        }
+
+        final FlowSnapshotEntityV1 other = (FlowSnapshotEntityV1) obj;
+        return Objects.equals(this.flowId, other.flowId) && Objects.equals(this.version, other.version);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDataSourceFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDataSourceFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDataSourceFactory.java
new file mode 100644
index 0000000..72d3acf
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDataSourceFactory.java
@@ -0,0 +1,81 @@
+/*
+ * 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.db.migration;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.h2.jdbcx.JdbcConnectionPool;
+
+import javax.sql.DataSource;
+import java.io.File;
+
+/**
+ * NOTE: This DataSource factory was used in the original 0.1.0 release and remains to migrate data from the old database.
+ * This class is intentionally not a Spring bean, and will be used manually in the custom Flyway migration.
+ */
+public class LegacyDataSourceFactory {
+
+    private static final String DB_USERNAME_PASSWORD = "nifireg";
+    private static final int MAX_CONNECTIONS = 5;
+
+    // database file name
+    private static final String DATABASE_FILE_NAME = "nifi-registry";
+
+    private final NiFiRegistryProperties properties;
+
+    private JdbcConnectionPool connectionPool;
+
+    public LegacyDataSourceFactory(final NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    public DataSource getDataSource() {
+        if (connectionPool == null) {
+            final String databaseUrl = getDatabaseUrl(properties);
+            connectionPool = JdbcConnectionPool.create(databaseUrl, DB_USERNAME_PASSWORD, DB_USERNAME_PASSWORD);
+            connectionPool.setMaxConnections(MAX_CONNECTIONS);
+        }
+
+        return connectionPool;
+    }
+
+    public static String getDatabaseUrl(final NiFiRegistryProperties properties) {
+        // locate the repository directory
+        final String repositoryDirectoryPath = properties.getLegacyDatabaseDirectory();
+
+        // ensure the repository directory is specified
+        if (repositoryDirectoryPath == null) {
+            throw new NullPointerException("Database directory must be specified.");
+        }
+
+        // create a handle to the repository directory
+        final File repositoryDirectory = new File(repositoryDirectoryPath);
+
+        // get a handle to the database file
+        final File databaseFile = new File(repositoryDirectory, DATABASE_FILE_NAME);
+
+        // format the database url
+        String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3";
+        String databaseUrlAppend = properties.getLegacyDatabaseUrlAppend();
+        if (StringUtils.isNotBlank(databaseUrlAppend)) {
+            databaseUrl += databaseUrlAppend;
+        }
+
+        return databaseUrl;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDatabaseService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDatabaseService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDatabaseService.java
new file mode 100644
index 0000000..533fadd
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyDatabaseService.java
@@ -0,0 +1,77 @@
+/*
+ * 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.db.migration;
+
+import org.springframework.jdbc.core.JdbcTemplate;
+
+import javax.sql.DataSource;
+import java.util.List;
+
+/**
+ * Service used to load data from original database used in the 0.1.0 release.
+ */
+public class LegacyDatabaseService {
+
+    private final JdbcTemplate jdbcTemplate;
+
+    public LegacyDatabaseService(final DataSource dataSource) {
+        this.jdbcTemplate = new JdbcTemplate(dataSource);
+    }
+
+    public List<BucketEntityV1> getAllBuckets() {
+        final String sql = "SELECT * FROM bucket ORDER BY name ASC";
+
+        return jdbcTemplate.query(sql, (rs, i) -> {
+            final BucketEntityV1 b = new BucketEntityV1();
+            b.setId(rs.getString("ID"));
+            b.setName(rs.getString("NAME"));
+            b.setDescription(rs.getString("DESCRIPTION"));
+            b.setCreated(rs.getTimestamp("CREATED"));
+            return b;
+        });
+    }
+
+    public List<FlowEntityV1> getAllFlows() {
+        final String sql = "SELECT * FROM flow f, bucket_item item WHERE item.id = f.id";
+
+        return jdbcTemplate.query(sql, (rs, i) -> {
+            final FlowEntityV1 flowEntity = new FlowEntityV1();
+            flowEntity.setId(rs.getString("ID"));
+            flowEntity.setName(rs.getString("NAME"));
+            flowEntity.setDescription(rs.getString("DESCRIPTION"));
+            flowEntity.setCreated(rs.getTimestamp("CREATED"));
+            flowEntity.setModified(rs.getTimestamp("MODIFIED"));
+            flowEntity.setBucketId(rs.getString("BUCKET_ID"));
+            return flowEntity;
+        });
+    }
+
+    public List<FlowSnapshotEntityV1> getAllFlowSnapshots() {
+        final String sql = "SELECT * FROM flow_snapshot fs";
+
+        return jdbcTemplate.query(sql, (rs, i) -> {
+            final FlowSnapshotEntityV1 fs = new FlowSnapshotEntityV1();
+            fs.setFlowId(rs.getString("FLOW_ID"));
+            fs.setVersion(rs.getInt("VERSION"));
+            fs.setCreated(rs.getTimestamp("CREATED"));
+            fs.setCreatedBy(rs.getString("CREATED_BY"));
+            fs.setComments(rs.getString("COMMENTS"));
+            return fs;
+        });
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyEntityMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyEntityMapper.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyEntityMapper.java
new file mode 100644
index 0000000..bf82aae
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/migration/LegacyEntityMapper.java
@@ -0,0 +1,63 @@
+/*
+ * 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.db.migration;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntityType;
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+
+/**
+ * Utility methods to map legacy DB entities to current DB entities.
+ *
+ * The initial implementations of these mappings will be almost a direct translation, but if future changes are made
+ * to the original tables these methods will handle the translation from old entity to new entity.
+ */
+public class LegacyEntityMapper {
+
+    public static BucketEntity createBucketEntity(final BucketEntityV1 bucketEntityV1) {
+        final BucketEntity bucketEntity = new BucketEntity();
+        bucketEntity.setId(bucketEntityV1.getId());
+        bucketEntity.setName(bucketEntityV1.getName());
+        bucketEntity.setDescription(bucketEntityV1.getDescription());
+        bucketEntity.setCreated(bucketEntityV1.getCreated());
+        return bucketEntity;
+    }
+
+    public static FlowEntity createFlowEntity(final FlowEntityV1 flowEntityV1) {
+        final FlowEntity flowEntity = new FlowEntity();
+        flowEntity.setId(flowEntityV1.getId());
+        flowEntity.setName(flowEntityV1.getName());
+        flowEntity.setDescription(flowEntityV1.getDescription());
+        flowEntity.setCreated(flowEntityV1.getCreated());
+        flowEntity.setModified(flowEntityV1.getModified());
+        flowEntity.setBucketId(flowEntityV1.getBucketId());
+        flowEntity.setType(BucketItemEntityType.FLOW);
+        return flowEntity;
+    }
+
+    public static FlowSnapshotEntity createFlowSnapshotEntity(final FlowSnapshotEntityV1 flowSnapshotEntityV1) {
+        final FlowSnapshotEntity flowSnapshotEntity = new FlowSnapshotEntity();
+        flowSnapshotEntity.setFlowId(flowSnapshotEntityV1.getFlowId());
+        flowSnapshotEntity.setVersion(flowSnapshotEntityV1.getVersion());
+        flowSnapshotEntity.setComments(flowSnapshotEntityV1.getComments());
+        flowSnapshotEntity.setCreated(flowSnapshotEntityV1.getCreated());
+        flowSnapshotEntity.setCreatedBy(flowSnapshotEntityV1.getCreatedBy());
+        return flowSnapshotEntity;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/migration/V1.2__IncreaseColumnSizes.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/migration/V1.2__IncreaseColumnSizes.sql b/nifi-registry-framework/src/main/resources/db/migration/V1.2__IncreaseColumnSizes.sql
deleted file mode 100644
index b2e92d5..0000000
--- a/nifi-registry-framework/src/main/resources/db/migration/V1.2__IncreaseColumnSizes.sql
+++ /dev/null
@@ -1,25 +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.
-
-ALTER TABLE BUCKET ALTER COLUMN NAME VARCHAR2(1000);
-ALTER TABLE BUCKET ALTER COLUMN DESCRIPTION VARCHAR2(65535);
-
-ALTER TABLE BUCKET_ITEM ALTER COLUMN NAME VARCHAR2(1000);
-ALTER TABLE BUCKET_ITEM ALTER COLUMN DESCRIPTION VARCHAR2(65535);
-
-ALTER TABLE FLOW_SNAPSHOT ALTER COLUMN CREATED_BY VARCHAR2(4096);
-ALTER TABLE FLOW_SNAPSHOT ALTER COLUMN COMMENTS VARCHAR2(65535);
-
-ALTER TABLE SIGNING_KEY ALTER COLUMN TENANT_IDENTITY VARCHAR2(4096);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/migration/V1.3__DropBucketItemNameUniqueness.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/migration/V1.3__DropBucketItemNameUniqueness.sql b/nifi-registry-framework/src/main/resources/db/migration/V1.3__DropBucketItemNameUniqueness.sql
deleted file mode 100644
index f29b4d0..0000000
--- a/nifi-registry-framework/src/main/resources/db/migration/V1.3__DropBucketItemNameUniqueness.sql
+++ /dev/null
@@ -1,27 +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.
-
-CREATE ALIAS IF NOT EXISTS EXECUTE AS $$ void executeSql(Connection conn, String sql)
-throws SQLException { conn.createStatement().executeUpdate(sql); } $$;
-
-call execute('ALTER TABLE BUCKET_ITEM DROP CONSTRAINT ' ||
-    (
-     SELECT DISTINCT CONSTRAINT_NAME
-     FROM INFORMATION_SCHEMA.CONSTRAINTS
-     WHERE TABLE_NAME = 'BUCKET_ITEM'
-     AND COLUMN_LIST = 'NAME'
-     AND CONSTRAINT_TYPE = 'UNIQUE'
-     )
-);

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql b/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
deleted file mode 100644
index a6b4960..0000000
--- a/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
+++ /dev/null
@@ -1,54 +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.
-
-CREATE TABLE BUCKET (
-    ID VARCHAR2(50) NOT NULL PRIMARY KEY,
-    NAME VARCHAR2(200) NOT NULL UNIQUE,
-    DESCRIPTION VARCHAR(4096),
-    CREATED TIMESTAMP NOT NULL
-);
-
-CREATE TABLE BUCKET_ITEM (
-    ID VARCHAR2(50) NOT NULL PRIMARY KEY,
-    NAME VARCHAR2(200) NOT NULL UNIQUE,
-    DESCRIPTION VARCHAR(4096),
-    CREATED TIMESTAMP NOT NULL,
-    MODIFIED TIMESTAMP NOT NULL,
-    ITEM_TYPE VARCHAR(50) NOT NULL,
-    BUCKET_ID VARCHAR2(50) NOT NULL,
-    FOREIGN KEY (BUCKET_ID) REFERENCES BUCKET(ID)
-);
-
-CREATE TABLE FLOW (
-    ID VARCHAR2(50) NOT NULL PRIMARY KEY,
-    FOREIGN KEY (ID) REFERENCES BUCKET_ITEM(ID)
-);
-
-CREATE TABLE FLOW_SNAPSHOT (
-    FLOW_ID VARCHAR2(50) NOT NULL,
-    VERSION INT NOT NULL,
-    CREATED TIMESTAMP NOT NULL,
-    CREATED_BY VARCHAR2(200) NOT NULL,
-    COMMENTS VARCHAR(4096),
-    PRIMARY KEY (FLOW_ID, VERSION),
-    FOREIGN KEY (FLOW_ID) REFERENCES FLOW(ID)
-);
-
-CREATE TABLE SIGNING_KEY (
-    ID VARCHAR2(50) NOT NULL,
-    TENANT_IDENTITY VARCHAR2(50) NOT NULL UNIQUE,
-    KEY_VALUE VARCHAR2(50) NOT NULL,
-    PRIMARY KEY (ID)
-);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/migration/V2__Initial.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/migration/V2__Initial.sql b/nifi-registry-framework/src/main/resources/db/migration/V2__Initial.sql
new file mode 100644
index 0000000..b992d23
--- /dev/null
+++ b/nifi-registry-framework/src/main/resources/db/migration/V2__Initial.sql
@@ -0,0 +1,60 @@
+-- Licensed to the Apache Software Foundation (ASF) under one or more
+-- contributor license agreements.  See the NOTICE file distributed with
+-- this work for additional information regarding copyright ownership.
+-- The ASF licenses this file to You under the Apache License, Version 2.0
+-- (the "License"); you may not use this file except in compliance with
+-- the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+-- The NAME column has a max size of 768 because this is the largest size that MySQL allows when using a unique constraint.
+CREATE TABLE BUCKET (
+    ID VARCHAR(50) NOT NULL,
+    NAME VARCHAR(1000) NOT NULL,
+    DESCRIPTION TEXT,
+    CREATED TIMESTAMP NOT NULL,
+    CONSTRAINT PK__BUCKET_ID PRIMARY KEY (ID),
+    CONSTRAINT UNIQUE__BUCKET_NAME UNIQUE (NAME)
+);
+
+CREATE TABLE BUCKET_ITEM (
+    ID VARCHAR(50) NOT NULL,
+    NAME VARCHAR(1000) NOT NULL,
+    DESCRIPTION TEXT,
+    CREATED TIMESTAMP NOT NULL,
+    MODIFIED TIMESTAMP NOT NULL,
+    ITEM_TYPE VARCHAR(50) NOT NULL,
+    BUCKET_ID VARCHAR(50) NOT NULL,
+    CONSTRAINT PK__BUCKET_ITEM_ID PRIMARY KEY (ID),
+    CONSTRAINT FK__BUCKET_ITEM_BUCKET_ID FOREIGN KEY (BUCKET_ID) REFERENCES BUCKET(ID)
+);
+
+CREATE TABLE FLOW (
+    ID VARCHAR(50) NOT NULL,
+    CONSTRAINT PK__FLOW_ID PRIMARY KEY (ID),
+    CONSTRAINT FK__FLOW_BUCKET_ITEM_ID FOREIGN KEY (ID) REFERENCES BUCKET_ITEM(ID)
+);
+
+CREATE TABLE FLOW_SNAPSHOT (
+    FLOW_ID VARCHAR(50) NOT NULL,
+    VERSION INT NOT NULL,
+    CREATED TIMESTAMP NOT NULL,
+    CREATED_BY VARCHAR(4096) NOT NULL,
+    COMMENTS TEXT,
+    CONSTRAINT PK__FLOW_SNAPSHOT_FLOW_ID_AND_VERSION PRIMARY KEY (FLOW_ID, VERSION),
+    CONSTRAINT FK__FLOW_SNAPSHOT_FLOW_ID FOREIGN KEY (FLOW_ID) REFERENCES FLOW(ID)
+);
+
+CREATE TABLE SIGNING_KEY (
+    ID VARCHAR(50) NOT NULL,
+    TENANT_IDENTITY VARCHAR(4096) NOT NULL,
+    KEY_VALUE VARCHAR(50) NOT NULL,
+    CONSTRAINT PK__SIGNING_KEY_ID PRIMARY KEY (ID),
+    CONSTRAINT UNIQUE__SIGNING_KEY_TENANT_IDENTITY UNIQUE (TENANT_IDENTITY)
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/original/V1.2__IncreaseColumnSizes.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/original/V1.2__IncreaseColumnSizes.sql b/nifi-registry-framework/src/main/resources/db/original/V1.2__IncreaseColumnSizes.sql
new file mode 100644
index 0000000..b2e92d5
--- /dev/null
+++ b/nifi-registry-framework/src/main/resources/db/original/V1.2__IncreaseColumnSizes.sql
@@ -0,0 +1,25 @@
+-- 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.
+
+ALTER TABLE BUCKET ALTER COLUMN NAME VARCHAR2(1000);
+ALTER TABLE BUCKET ALTER COLUMN DESCRIPTION VARCHAR2(65535);
+
+ALTER TABLE BUCKET_ITEM ALTER COLUMN NAME VARCHAR2(1000);
+ALTER TABLE BUCKET_ITEM ALTER COLUMN DESCRIPTION VARCHAR2(65535);
+
+ALTER TABLE FLOW_SNAPSHOT ALTER COLUMN CREATED_BY VARCHAR2(4096);
+ALTER TABLE FLOW_SNAPSHOT ALTER COLUMN COMMENTS VARCHAR2(65535);
+
+ALTER TABLE SIGNING_KEY ALTER COLUMN TENANT_IDENTITY VARCHAR2(4096);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/original/V1.3__DropBucketItemNameUniqueness.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/original/V1.3__DropBucketItemNameUniqueness.sql b/nifi-registry-framework/src/main/resources/db/original/V1.3__DropBucketItemNameUniqueness.sql
new file mode 100644
index 0000000..f29b4d0
--- /dev/null
+++ b/nifi-registry-framework/src/main/resources/db/original/V1.3__DropBucketItemNameUniqueness.sql
@@ -0,0 +1,27 @@
+-- Licensed to the Apache Software Foundation (ASF) under one or more
+-- contributor license agreements.  See the NOTICE file distributed with
+-- this work for additional information regarding copyright ownership.
+-- The ASF licenses this file to You under the Apache License, Version 2.0
+-- (the "License"); you may not use this file except in compliance with
+-- the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+CREATE ALIAS IF NOT EXISTS EXECUTE AS $$ void executeSql(Connection conn, String sql)
+throws SQLException { conn.createStatement().executeUpdate(sql); } $$;
+
+call execute('ALTER TABLE BUCKET_ITEM DROP CONSTRAINT ' ||
+    (
+     SELECT DISTINCT CONSTRAINT_NAME
+     FROM INFORMATION_SCHEMA.CONSTRAINTS
+     WHERE TABLE_NAME = 'BUCKET_ITEM'
+     AND COLUMN_LIST = 'NAME'
+     AND CONSTRAINT_TYPE = 'UNIQUE'
+     )
+);

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/main/resources/db/original/V1__Initial.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/original/V1__Initial.sql b/nifi-registry-framework/src/main/resources/db/original/V1__Initial.sql
new file mode 100644
index 0000000..a6b4960
--- /dev/null
+++ b/nifi-registry-framework/src/main/resources/db/original/V1__Initial.sql
@@ -0,0 +1,54 @@
+-- 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.
+
+CREATE TABLE BUCKET (
+    ID VARCHAR2(50) NOT NULL PRIMARY KEY,
+    NAME VARCHAR2(200) NOT NULL UNIQUE,
+    DESCRIPTION VARCHAR(4096),
+    CREATED TIMESTAMP NOT NULL
+);
+
+CREATE TABLE BUCKET_ITEM (
+    ID VARCHAR2(50) NOT NULL PRIMARY KEY,
+    NAME VARCHAR2(200) NOT NULL UNIQUE,
+    DESCRIPTION VARCHAR(4096),
+    CREATED TIMESTAMP NOT NULL,
+    MODIFIED TIMESTAMP NOT NULL,
+    ITEM_TYPE VARCHAR(50) NOT NULL,
+    BUCKET_ID VARCHAR2(50) NOT NULL,
+    FOREIGN KEY (BUCKET_ID) REFERENCES BUCKET(ID)
+);
+
+CREATE TABLE FLOW (
+    ID VARCHAR2(50) NOT NULL PRIMARY KEY,
+    FOREIGN KEY (ID) REFERENCES BUCKET_ITEM(ID)
+);
+
+CREATE TABLE FLOW_SNAPSHOT (
+    FLOW_ID VARCHAR2(50) NOT NULL,
+    VERSION INT NOT NULL,
+    CREATED TIMESTAMP NOT NULL,
+    CREATED_BY VARCHAR2(200) NOT NULL,
+    COMMENTS VARCHAR(4096),
+    PRIMARY KEY (FLOW_ID, VERSION),
+    FOREIGN KEY (FLOW_ID) REFERENCES FLOW(ID)
+);
+
+CREATE TABLE SIGNING_KEY (
+    ID VARCHAR2(50) NOT NULL,
+    TENANT_IDENTITY VARCHAR2(50) NOT NULL UNIQUE,
+    KEY_VALUE VARCHAR2(50) NOT NULL,
+    PRIMARY KEY (ID)
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/DatabaseTestApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/DatabaseTestApplication.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/DatabaseTestApplication.java
index ce72410..0ce3812 100644
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/DatabaseTestApplication.java
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/DatabaseTestApplication.java
@@ -16,8 +16,11 @@
  */
 package org.apache.nifi.registry.db;
 
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.mockito.Mockito;
 import org.springframework.boot.SpringApplication;
 import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.ComponentScan;
 import org.springframework.context.annotation.FilterType;
 
@@ -42,4 +45,8 @@ public class DatabaseTestApplication {
         SpringApplication.run(DatabaseTestApplication.class, args);
     }
 
+    @Bean
+    public NiFiRegistryProperties createNiFiRegistryProperties() {
+        return Mockito.mock(NiFiRegistryProperties.class);
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyDatabaseService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyDatabaseService.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyDatabaseService.java
new file mode 100644
index 0000000..df37e8e
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyDatabaseService.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.db.migration;
+
+import org.apache.nifi.registry.db.entity.BucketItemEntityType;
+import org.flywaydb.core.Flyway;
+import org.junit.Before;
+import org.junit.Test;
+import org.springframework.boot.jdbc.DataSourceBuilder;
+import org.springframework.jdbc.core.JdbcTemplate;
+
+import javax.sql.DataSource;
+import java.util.Date;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test running the legacy Flyway migrations against an in-memory H2 and then using the LegacyDatabaseService to
+ * retrieve data. Purposely not using Spring test annotations here to avoid interfering with the normal DB context/flyway.
+ */
+public class TestLegacyDatabaseService {
+
+    private DataSource dataSource;
+    private JdbcTemplate jdbcTemplate;
+    private Flyway flyway;
+
+    private BucketEntityV1 bucketEntityV1;
+    private FlowEntityV1 flowEntityV1;
+    private FlowSnapshotEntityV1 flowSnapshotEntityV1;
+
+    @Before
+    public void setup() {
+        dataSource = DataSourceBuilder.create()
+                .url("jdbc:h2:mem:legacydb")
+                .driverClassName("org.h2.Driver")
+                .build();
+
+        jdbcTemplate = new JdbcTemplate(dataSource);
+
+        flyway = new Flyway();
+        flyway.setDataSource(dataSource);
+        flyway.setLocations("db/original");
+        flyway.migrate();
+
+        bucketEntityV1 = new BucketEntityV1();
+        bucketEntityV1.setId("1");
+        bucketEntityV1.setName("Bucket1");
+        bucketEntityV1.setDescription("This is bucket 1");
+        bucketEntityV1.setCreated(new Date());
+
+        jdbcTemplate.update("INSERT INTO bucket (ID, NAME, DESCRIPTION, CREATED) VALUES (?, ?, ?, ?)",
+                bucketEntityV1.getId(),
+                bucketEntityV1.getName(),
+                bucketEntityV1.getDescription(),
+                bucketEntityV1.getCreated());
+
+        flowEntityV1 = new FlowEntityV1();
+        flowEntityV1.setId("1");
+        flowEntityV1.setBucketId(bucketEntityV1.getId());
+        flowEntityV1.setName("Flow1");
+        flowEntityV1.setDescription("This is flow1");
+        flowEntityV1.setCreated(new Date());
+        flowEntityV1.setModified(new Date());
+
+        jdbcTemplate.update("INSERT INTO bucket_item (ID, NAME, DESCRIPTION, CREATED, MODIFIED, ITEM_TYPE, BUCKET_ID) VALUES (?, ?, ?, ?, ?, ?, ?)",
+                flowEntityV1.getId(),
+                flowEntityV1.getName(),
+                flowEntityV1.getDescription(),
+                flowEntityV1.getCreated(),
+                flowEntityV1.getModified(),
+                BucketItemEntityType.FLOW.toString(),
+                flowEntityV1.getBucketId());
+
+        jdbcTemplate.update("INSERT INTO flow (ID) VALUES (?)", flowEntityV1.getId());
+
+        flowSnapshotEntityV1 = new FlowSnapshotEntityV1();
+        flowSnapshotEntityV1.setFlowId(flowEntityV1.getId());
+        flowSnapshotEntityV1.setVersion(1);
+        flowSnapshotEntityV1.setComments("This is v1");
+        flowSnapshotEntityV1.setCreated(new Date());
+        flowSnapshotEntityV1.setCreatedBy("user1");
+
+        jdbcTemplate.update("INSERT INTO flow_snapshot (FLOW_ID, VERSION, CREATED, CREATED_BY, COMMENTS) VALUES (?, ?, ?, ?, ?)",
+                flowSnapshotEntityV1.getFlowId(),
+                flowSnapshotEntityV1.getVersion(),
+                flowSnapshotEntityV1.getCreated(),
+                flowSnapshotEntityV1.getCreatedBy(),
+                flowSnapshotEntityV1.getComments());
+    }
+
+    @Test
+    public void testGetLegacyData() {
+        final LegacyDatabaseService service = new LegacyDatabaseService(dataSource);
+
+        final List<BucketEntityV1> buckets = service.getAllBuckets();
+        assertEquals(1, buckets.size());
+
+        final BucketEntityV1 b = buckets.stream().findFirst().get();
+        assertEquals(bucketEntityV1.getId(), b.getId());
+        assertEquals(bucketEntityV1.getName(), b.getName());
+        assertEquals(bucketEntityV1.getDescription(), b.getDescription());
+        assertEquals(bucketEntityV1.getCreated(), b.getCreated());
+
+        final List<FlowEntityV1> flows = service.getAllFlows();
+        assertEquals(1, flows.size());
+
+        final FlowEntityV1 f = flows.stream().findFirst().get();
+        assertEquals(flowEntityV1.getId(), f.getId());
+        assertEquals(flowEntityV1.getName(), f.getName());
+        assertEquals(flowEntityV1.getDescription(), f.getDescription());
+        assertEquals(flowEntityV1.getCreated(), f.getCreated());
+        assertEquals(flowEntityV1.getModified(), f.getModified());
+        assertEquals(flowEntityV1.getBucketId(), f.getBucketId());
+
+        final List<FlowSnapshotEntityV1> flowSnapshots = service.getAllFlowSnapshots();
+        assertEquals(1, flowSnapshots.size());
+
+        final FlowSnapshotEntityV1 fs = flowSnapshots.stream().findFirst().get();
+        assertEquals(flowSnapshotEntityV1.getFlowId(), fs.getFlowId());
+        assertEquals(flowSnapshotEntityV1.getVersion(), fs.getVersion());
+        assertEquals(flowSnapshotEntityV1.getComments(), fs.getComments());
+        assertEquals(flowSnapshotEntityV1.getCreatedBy(), fs.getCreatedBy());
+        assertEquals(flowSnapshotEntityV1.getCreated(), fs.getCreated());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyEntityMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyEntityMapper.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyEntityMapper.java
new file mode 100644
index 0000000..3de297f
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/migration/TestLegacyEntityMapper.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.db.migration;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.junit.Test;
+
+import java.util.Date;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestLegacyEntityMapper {
+
+    @Test
+    public void testMapLegacyEntities() {
+        final BucketEntityV1 bucketEntityV1 = new BucketEntityV1();
+        bucketEntityV1.setId("1");
+        bucketEntityV1.setName("Bucket1");
+        bucketEntityV1.setDescription("This is bucket 1");
+        bucketEntityV1.setCreated(new Date());
+
+        final BucketEntity bucketEntity = LegacyEntityMapper.createBucketEntity(bucketEntityV1);
+        assertNotNull(bucketEntity);
+        assertEquals(bucketEntityV1.getId(), bucketEntity.getId());
+        assertEquals(bucketEntityV1.getName(), bucketEntity.getName());
+        assertEquals(bucketEntityV1.getDescription(), bucketEntity.getDescription());
+        assertEquals(bucketEntityV1.getCreated(), bucketEntity.getCreated());
+
+        final FlowEntityV1 flowEntityV1 = new FlowEntityV1();
+        flowEntityV1.setId("1");
+        flowEntityV1.setBucketId(bucketEntityV1.getId());
+        flowEntityV1.setName("Flow1");
+        flowEntityV1.setDescription("This is flow1");
+        flowEntityV1.setCreated(new Date());
+        flowEntityV1.setModified(new Date());
+
+        final FlowEntity flowEntity = LegacyEntityMapper.createFlowEntity(flowEntityV1);
+        assertNotNull(flowEntity);
+        assertEquals(flowEntityV1.getId(), flowEntity.getId());
+        assertEquals(flowEntityV1.getBucketId(), flowEntity.getBucketId());
+        assertEquals(flowEntityV1.getName(), flowEntity.getName());
+        assertEquals(flowEntityV1.getDescription(), flowEntity.getDescription());
+        assertEquals(flowEntityV1.getCreated(), flowEntity.getCreated());
+        assertEquals(flowEntityV1.getModified(), flowEntity.getModified());
+
+        final FlowSnapshotEntityV1 flowSnapshotEntityV1 = new FlowSnapshotEntityV1();
+        flowSnapshotEntityV1.setFlowId(flowEntityV1.getId());
+        flowSnapshotEntityV1.setVersion(1);
+        flowSnapshotEntityV1.setComments("This is v1");
+        flowSnapshotEntityV1.setCreated(new Date());
+        flowSnapshotEntityV1.setCreatedBy("user1");
+
+        final FlowSnapshotEntity flowSnapshotEntity = LegacyEntityMapper.createFlowSnapshotEntity(flowSnapshotEntityV1);
+        assertNotNull(flowSnapshotEntity);
+        assertEquals(flowSnapshotEntityV1.getFlowId(), flowSnapshotEntity.getFlowId());
+        assertEquals(flowSnapshotEntityV1.getVersion(), flowSnapshotEntity.getVersion());
+        assertEquals(flowSnapshotEntityV1.getComments(), flowSnapshotEntity.getComments());
+        assertEquals(flowSnapshotEntityV1.getCreatedBy(), flowSnapshotEntity.getCreatedBy());
+        assertEquals(flowSnapshotEntityV1.getCreated(), flowSnapshotEntity.getCreated());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java b/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
index 25c72f4..c202a5b 100644
--- a/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
+++ b/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.registry.jetty;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
 import org.eclipse.jetty.annotations.AnnotationConfiguration;
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.Handler;
@@ -46,14 +46,18 @@ import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.MalformedURLException;
 import java.net.NetworkInterface;
 import java.net.SocketException;
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 
@@ -237,7 +241,7 @@ public class JettyServer {
 
         webUiContext = loadWar(webUiWar, "/nifi-registry");
 
-        webApiContext = loadWar(webApiWar, "/nifi-registry-api");
+        webApiContext = loadWar(webApiWar, "/nifi-registry-api", getWebApiAdditionalClasspath());
         logger.info("Adding {} object to ServletContext with key 'nifi-registry.properties'", properties.getClass().getSimpleName());
         webApiContext.setAttribute("nifi-registry.properties", properties);
         logger.info("Adding {} object to ServletContext with key 'nifi-registry.key'", masterKeyProvider.getClass().getSimpleName());
@@ -257,7 +261,13 @@ public class JettyServer {
         server.setHandler(handlers);
     }
 
-    private WebAppContext loadWar(final File warFile, final String contextPath) throws IOException {
+    private WebAppContext loadWar(final File warFile, final String contextPath)
+            throws IOException {
+        return loadWar(warFile, contextPath, new URL[0]);
+    }
+
+    private WebAppContext loadWar(final File warFile, final String contextPath, final URL[] additionalResources)
+            throws IOException {
         final WebAppContext webappContext = new WebAppContext(warFile.getPath(), contextPath);
         webappContext.setContextPath(contextPath);
         webappContext.setDisplayName(contextPath);
@@ -289,12 +299,75 @@ public class JettyServer {
         // configure the max form size (3x the default)
         webappContext.setMaxFormContentSize(600000);
 
-        webappContext.setClassLoader(new WebAppClassLoader(ClassLoader.getSystemClassLoader(), webappContext));
+        // start out assuming the system ClassLoader will be the parent, but if additional resources were specified then
+        // inject a new ClassLoader in between the system and webapp ClassLoaders that contains the additional resources
+        ClassLoader parentClassLoader = ClassLoader.getSystemClassLoader();
+        if (additionalResources != null && additionalResources.length > 0) {
+            URLClassLoader additionalClassLoader = new URLClassLoader(additionalResources, ClassLoader.getSystemClassLoader());
+            parentClassLoader = additionalClassLoader;
+        }
+
+        webappContext.setClassLoader(new WebAppClassLoader(parentClassLoader, webappContext));
 
         logger.info("Loading WAR: " + warFile.getAbsolutePath() + " with context path set to " + contextPath);
         return webappContext;
     }
 
+    private URL[] getWebApiAdditionalClasspath() {
+        final String dbDriverDir = properties.getDatabaseDriverDirectory();
+
+        if (StringUtils.isBlank(dbDriverDir)) {
+            logger.info("No database driver directory was specified");
+            return new URL[0];
+        }
+
+        final File dirFile = new File(dbDriverDir);
+
+        if (!dirFile.exists()) {
+            logger.warn("Skipping database driver directory that does not exist: " + dbDriverDir);
+            return new URL[0];
+        }
+
+        if (!dirFile.canRead()) {
+            logger.warn("Skipping database driver directory that can not be read: " + dbDriverDir);
+            return new URL[0];
+        }
+
+        final List<URL> resources = new LinkedList<>();
+        try {
+            resources.add(dirFile.toURI().toURL());
+        } catch (final MalformedURLException mfe) {
+            logger.warn("Unable to add {} to classpath due to {}", new Object[]{ dirFile.getAbsolutePath(), mfe.getMessage()}, mfe);
+        }
+
+        if (dirFile.isDirectory()) {
+            final File[] files = dirFile.listFiles();
+            if (files != null) {
+                for (final File resource : files) {
+                    if (resource.isDirectory()) {
+                        logger.warn("Recursive directories are not supported, skipping " + resource.getAbsolutePath());
+                    } else {
+                        try {
+                            resources.add(resource.toURI().toURL());
+                        } catch (final MalformedURLException mfe) {
+                            logger.warn("Unable to add {} to classpath due to {}", new Object[]{ resource.getAbsolutePath(), mfe.getMessage()}, mfe);
+                        }
+                    }
+                }
+            }
+        }
+
+        if (!resources.isEmpty()) {
+            logger.info("Added additional resources to nifi-registry-api classpath: [");
+            for (URL resource : resources) {
+                logger.info(" " + resource.toString());
+            }
+            logger.info("]");
+        }
+
+        return resources.toArray(new URL[resources.size()]);
+    }
+
     private ContextHandler createDocsWebApp(final String contextPath) throws IOException {
         final ResourceHandler resourceHandler = new ResourceHandler();
         resourceHandler.setDirectoriesListed(false);

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
index bf18609..62caf2c 100644
--- a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
@@ -58,9 +58,19 @@ public class NiFiRegistryProperties extends Properties {
 
     public static final String PROVIDERS_CONFIGURATION_FILE = "nifi.registry.providers.configuration.file";
 
+    // Original DB properties
     public static final String DATABASE_DIRECTORY = "nifi.registry.db.directory";
     public static final String DATABASE_URL_APPEND = "nifi.registry.db.url.append";
 
+    // New style DB properties
+    public static final String DATABASE_URL = "nifi.registry.db.url";
+    public static final String DATABASE_DRIVER_CLASS_NAME = "nifi.registry.db.driver.class";
+    public static final String DATABASE_DRIVER_DIR = "nifi.registry.db.driver.directory";
+    public static final String DATABASE_USERNAME = "nifi.registry.db.username";
+    public static final String DATABASE_PASSWORD = "nifi.registry.db.password";
+    public static final String DATABASE_MAX_CONNECTIONS = "nifi.registry.db.maxConnections";
+    public static final String DATABASE_SQL_DEBUG = "nifi.registry.db.sql.debug";
+
     // Kerberos properties
     public static final String KERBEROS_KRB5_FILE = "nifi.registry.kerberos.krb5.file";
     public static final String KERBEROS_SPNEGO_PRINCIPAL = "nifi.registry.kerberos.spnego.principal";
@@ -150,14 +160,48 @@ public class NiFiRegistryProperties extends Properties {
         return getPropertyAsFile(PROVIDERS_CONFIGURATION_FILE, DEFAULT_PROVIDERS_CONFIGURATION_FILE);
     }
 
-    public String getDatabaseDirectory() {
+    public String getLegacyDatabaseDirectory() {
         return getProperty(DATABASE_DIRECTORY);
     }
 
-    public String getDatabaseUrlAppend() {
+    public String getLegacyDatabaseUrlAppend() {
         return getProperty(DATABASE_URL_APPEND);
     }
 
+    public String getDatabaseUrl() {
+        return getProperty(DATABASE_URL);
+    }
+
+    public String getDatabaseDriverClassName() {
+        return getProperty(DATABASE_DRIVER_CLASS_NAME);
+    }
+
+    public String getDatabaseDriverDirectory() {
+        return getProperty(DATABASE_DRIVER_DIR);
+    }
+
+    public String getDatabaseUsername() {
+        return getProperty(DATABASE_USERNAME);
+    }
+
+    public String getDatabasePassword() {
+        return getProperty(DATABASE_PASSWORD);
+    }
+
+    public Integer getDatabaseMaxConnections() {
+        return getPropertyAsInteger(DATABASE_MAX_CONNECTIONS);
+    }
+
+    public boolean getDatabaseSqlDebug() {
+        final String value = getProperty(DATABASE_SQL_DEBUG);
+
+        if (StringUtils.isBlank(value)) {
+            return false;
+        }
+
+        return "true".equalsIgnoreCase(value.trim());
+    }
+
     public File getAuthorizersConfigurationFile() {
         return getPropertyAsFile(SECURITY_AUTHORIZERS_CONFIGURATION_FILE, DEFAULT_SECURITY_AUTHORIZERS_CONFIGURATION_FILE);
     }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/44bc4adb/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties b/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
index 5b6dae2..fb77a07 100644
--- a/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
+++ b/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
@@ -42,10 +42,20 @@ nifi.registry.security.identity.provider=${nifi.registry.security.identity.provi
 # providers properties #
 nifi.registry.providers.configuration.file=${nifi.registry.providers.configuration.file}
 
-# database properties
+# legacy database properties, used to migrate data from original DB to new DB below
+# NOTE: Users upgrading from 0.1.0 should leave these populated, but new installs after 0.1.0 should leave these empty
 nifi.registry.db.directory=${nifi.registry.db.directory}
 nifi.registry.db.url.append=${nifi.registry.db.url.append}
 
+# database properties
+nifi.registry.db.url=${nifi.registry.db.url}
+nifi.registry.db.driver.class=${nifi.registry.db.driver.class}
+nifi.registry.db.driver.directory=${nifi.registry.db.driver.directory}
+nifi.registry.db.username=${nifi.registry.db.username}
+nifi.registry.db.password=${nifi.registry.db.password}
+nifi.registry.db.maxConnections=${nifi.registry.db.maxConnections}
+nifi.registry.db.sql.debug=${nifi.registry.db.sql.debug}
+
 # extension directories #
 # Each property beginning with "nifi.registry.extension.dir." will be treated as location for an extension,
 # and a class loader will be created for each location, with the system class loader as the parent