You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2017/09/21 13:50:35 UTC

[1/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Repository: nifi-registry
Updated Branches:
  refs/heads/master d93eab365 -> d478c20ee


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/link/TestLinkService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/link/TestLinkService.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/link/TestLinkService.java
new file mode 100644
index 0000000..d204b80
--- /dev/null
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/link/TestLinkService.java
@@ -0,0 +1,119 @@
+/*
+ * 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.web.link;
+
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.bucket.BucketItem;
+import org.apache.nifi.registry.flow.VersionedFlow;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestLinkService {
+
+    private LinkService linkService;
+
+    private List<Bucket> buckets;
+    private List<VersionedFlow> flows;
+    private List<VersionedFlowSnapshotMetadata> snapshots;
+    private List<BucketItem> items;
+
+    @Before
+    public void setup() {
+        linkService = new LinkService();
+
+        // setup buckets
+        final Bucket bucket1 = new Bucket();
+        bucket1.setIdentifier("b1");
+        bucket1.setName("Bucket 1");
+
+        final Bucket bucket2 = new Bucket();
+        bucket2.setIdentifier("b2");
+        bucket2.setName("Bucket 2");
+
+        buckets = new ArrayList<>();
+        buckets.add(bucket1);
+        buckets.add(bucket2);
+
+        // setup flows
+        final VersionedFlow flow1 = new VersionedFlow();
+        flow1.setIdentifier("f1");
+        flow1.setName("Flow 1");
+        flow1.setBucketIdentifier(bucket1.getIdentifier());
+
+        final VersionedFlow flow2 = new VersionedFlow();
+        flow2.setIdentifier("f2");
+        flow2.setName("Flow 2");
+        flow2.setBucketIdentifier(bucket1.getIdentifier());
+
+        flows = new ArrayList<>();
+        flows.add(flow1);
+        flows.add(flow2);
+
+        //setup snapshots
+        final VersionedFlowSnapshotMetadata snapshotMetadata1 = new VersionedFlowSnapshotMetadata();
+        snapshotMetadata1.setFlowIdentifier(flow1.getIdentifier());
+        snapshotMetadata1.setVersion(1);
+
+        final VersionedFlowSnapshotMetadata snapshotMetadata2 = new VersionedFlowSnapshotMetadata();
+        snapshotMetadata2.setFlowIdentifier(flow1.getIdentifier());
+        snapshotMetadata2.setVersion(2);
+
+        snapshots = new ArrayList<>();
+        snapshots.add(snapshotMetadata1);
+        snapshots.add(snapshotMetadata2);
+
+        // setup items
+        items = new ArrayList<>();
+        items.add(flow1);
+        items.add(flow2);
+    }
+
+    @Test
+    public void testPopulateBucketLinks() {
+        buckets.stream().forEach(b -> Assert.assertNull(b.getLink()));
+        linkService.populateBucketLinks(buckets);
+        buckets.stream().forEach(b -> Assert.assertEquals("buckets/" + b.getIdentifier(), b.getLink().getUri().toString()));
+    }
+
+    @Test
+    public void testPopulateFlowLinks() {
+        flows.stream().forEach(f -> Assert.assertNull(f.getLink()));
+        linkService.populateFlowLinks(flows);
+        flows.stream().forEach(f -> Assert.assertEquals("flows/" + f.getIdentifier(), f.getLink().getUri().toString()));
+    }
+
+    @Test
+    public void testPopulateSnapshotLinks() {
+        snapshots.stream().forEach(s -> Assert.assertNull(s.getLink()));
+        linkService.populateSnapshotLinks(snapshots);
+        snapshots.stream().forEach(s -> Assert.assertEquals(
+                "flows/" + s.getFlowIdentifier() + "/versions/" + s.getVersion(), s.getLink().getUri().toString()));
+    }
+
+    @Test
+    public void testPopulateItemLinks() {
+        items.stream().forEach(i -> Assert.assertNull(i.getLink()));
+        linkService.populateItemLinks(items);
+        items.stream().forEach(i -> Assert.assertEquals("flows/" + i.getIdentifier(), i.getLink().getUri().toString()));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-ui/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-ui/src/main/webapp/WEB-INF/web.xml b/nifi-registry-web-ui/src/main/webapp/WEB-INF/web.xml
index 2be7fff..da27a62 100644
--- a/nifi-registry-web-ui/src/main/webapp/WEB-INF/web.xml
+++ b/nifi-registry-web-ui/src/main/webapp/WEB-INF/web.xml
@@ -51,3 +51,4 @@
         <welcome-file>/WEB-INF/pages/index.jsp</welcome-file>
     </welcome-file-list>
 </web-app>
+

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1fbdb53..b46f111 100644
--- a/pom.xml
+++ b/pom.xml
@@ -40,7 +40,6 @@
         <module>nifi-registry-security</module>
         <module>nifi-registry-framework</module>
         <module>nifi-registry-provider-api</module>
-        <module>nifi-registry-provider-impl</module>
         <module>nifi-registry-web-api</module>
         <module>nifi-registry-web-ui</module>
         <module>nifi-registry-bootstrap</module>
@@ -103,10 +102,10 @@
         <inceptionYear>2017</inceptionYear>
         <org.slf4j.version>1.7.12</org.slf4j.version>
         <jetty.version>9.4.3.v20170317</jetty.version>
+        <jax.rs.api.version>2.0.1</jax.rs.api.version>
         <jersey.version>2.25.1</jersey.version>
-        <!-- spring.version should always match the version of spring used by the version of spring-boot -->
-        <spring.boot.version>1.5.6.RELEASE</spring.boot.version>
-        <spring.version>4.3.10.RELEASE</spring.version>
+        <spring.boot.version>1.5.7.RELEASE</spring.boot.version>
+        <flyway.version>4.2.0</flyway.version>
     </properties>
 
     <repositories>
@@ -235,6 +234,11 @@
                 <version>3.1.0</version>
             </dependency>
             <dependency>
+                <groupId>javax.ws.rs</groupId>
+                <artifactId>javax.ws.rs-api</artifactId>
+                <version>${jax.rs.api.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>org.glassfish.jersey.core</groupId>
                 <artifactId>jersey-server</artifactId>
                 <version>${jersey.version}</version>
@@ -280,24 +284,6 @@
                 <artifactId>javax.el</artifactId>
                 <version>3.0.1-b08</version>
             </dependency>
-            <!-- These Spring dependencies should end up in the REST API through spring-boot, but we need the framework
-                    module to declare Spring beans, so ensure the Spring version lines up with the version of spring-boot -->
-            <dependency>
-                <groupId>org.springframework</groupId>
-                <artifactId>spring-beans</artifactId>
-                <version>${spring.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.springframework</groupId>
-                <artifactId>spring-core</artifactId>
-                <version>${spring.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.springframework</groupId>
-                <artifactId>spring-context</artifactId>
-                <version>${spring.version}</version>
-            </dependency>
-            <!-- End Spring dependencies -->
             <dependency>
                 <groupId>org.apache.commons</groupId>
                 <artifactId>commons-lang3</artifactId>


[6/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/FileSystemFlowPersistenceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/FileSystemFlowPersistenceProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/FileSystemFlowPersistenceProvider.java
new file mode 100644
index 0000000..ddad946
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/FileSystemFlowPersistenceProvider.java
@@ -0,0 +1,169 @@
+/*
+ * 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.provider.flow;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.flow.FlowPersistenceException;
+import org.apache.nifi.registry.flow.FlowPersistenceProvider;
+import org.apache.nifi.registry.flow.FlowSnapshotContext;
+import org.apache.nifi.registry.provider.ProviderConfigurationContext;
+import org.apache.nifi.registry.provider.ProviderCreationException;
+import org.apache.nifi.registry.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+
+/**
+ * A FlowPersistenceProvider that uses the local filesystem for storage.
+ */
+public class FileSystemFlowPersistenceProvider implements FlowPersistenceProvider {
+
+    static final Logger LOGGER = LoggerFactory.getLogger(FileSystemFlowPersistenceProvider.class);
+
+    static final String FLOW_STORAGE_DIR_PROP = "Flow Storage Directory";
+
+    static final String SNAPSHOT_EXTENSION = ".snapshot";
+
+    private File flowStorageDir;
+
+    @Override
+    public void onConfigured(final ProviderConfigurationContext configurationContext) throws ProviderCreationException {
+        final Map<String,String> props = configurationContext.getProperties();
+        if (!props.containsKey(FLOW_STORAGE_DIR_PROP)) {
+            throw new ProviderCreationException("The property " + FLOW_STORAGE_DIR_PROP + " must be provided");
+        }
+
+        final String flowStorageDirValue = props.get(FLOW_STORAGE_DIR_PROP);
+        if (StringUtils.isBlank(flowStorageDirValue)) {
+            throw new ProviderCreationException("The property " + FLOW_STORAGE_DIR_PROP + " cannot be null or blank");
+        }
+
+        try {
+            flowStorageDir = new File(flowStorageDirValue);
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(flowStorageDir);
+            LOGGER.info("Configured FileSystemFlowPersistenceProvider with Flow Storage Directory {}", new Object[] {flowStorageDir.getAbsolutePath()});
+        } catch (IOException e) {
+            throw new ProviderCreationException(e);
+        }
+    }
+
+    @Override
+    public synchronized void saveSnapshot(final FlowSnapshotContext context, final byte[] content) throws FlowPersistenceException {
+        final File bucketDir = new File(flowStorageDir, context.getBucketId());
+        try {
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(bucketDir);
+        } catch (IOException e) {
+            throw new FlowPersistenceException("Error accessing bucket directory at " + bucketDir.getAbsolutePath(), e);
+        }
+
+        final File flowDir = new File(bucketDir, context.getFlowId());
+        try {
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(flowDir);
+        } catch (IOException e) {
+            throw new FlowPersistenceException("Error accessing flow directory at " + flowDir.getAbsolutePath(), e);
+        }
+
+        final String versionString = String.valueOf(context.getVersion());
+        final File versionDir = new File(flowDir, versionString);
+        try {
+            FileUtils.ensureDirectoryExistAndCanReadAndWrite(versionDir);
+        } catch (IOException e) {
+            throw new FlowPersistenceException("Error accessing version directory at " + versionDir.getAbsolutePath(), e);
+        }
+
+        final File versionFile = new File(versionDir, versionString + SNAPSHOT_EXTENSION);
+        if (versionFile.exists()) {
+            throw new FlowPersistenceException("Unable to save, a snapshot already exists with version " + versionString);
+        }
+
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Saving snapshot with filename {}", new Object[] {versionFile.getAbsolutePath()});
+        }
+
+        try (final OutputStream out = new FileOutputStream(versionFile)) {
+            out.write(content);
+            out.flush();
+        } catch (Exception e) {
+            throw new FlowPersistenceException("Unable to write snapshot to disk due to " + e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public synchronized byte[] getSnapshot(final String bucketId, final String flowId, final int version) throws FlowPersistenceException {
+        final File snapshotFile = getSnapshotFile(bucketId, flowId, version);
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Retrieving snapshot with filename {}", new Object[] {snapshotFile.getAbsolutePath()});
+        }
+
+        if (!snapshotFile.exists()) {
+            return null;
+        }
+
+        try (final InputStream in = new FileInputStream(snapshotFile)){
+            return IOUtils.toByteArray(in);
+        } catch (IOException e) {
+            throw new FlowPersistenceException("Error reading snapshot file: " + snapshotFile.getAbsolutePath(), e);
+        }
+    }
+
+    @Override
+    public synchronized void deleteSnapshots(final String bucketId, final String flowId) throws FlowPersistenceException {
+        final File flowDir = new File(flowStorageDir, bucketId + "/" + flowId);
+        if (!flowDir.exists()) {
+            LOGGER.debug("Snapshot directory does not exist at {}", new Object[] {flowDir.getAbsolutePath()});
+            return;
+        }
+
+        try {
+            org.apache.commons.io.FileUtils.cleanDirectory(flowDir);
+        } catch (IOException e) {
+            throw new FlowPersistenceException("Error deleting snapshots at " + flowDir.getAbsolutePath(), e);
+        }
+    }
+
+    @Override
+    public synchronized void deleteSnapshot(final String bucketId, final String flowId, final int version) throws FlowPersistenceException {
+        final File snapshotFile = getSnapshotFile(bucketId, flowId, version);
+        if (!snapshotFile.exists()) {
+            LOGGER.debug("Snapshot file does not exist at {}", new Object[] {snapshotFile.getAbsolutePath()});
+            return;
+        }
+
+        final boolean deleted = snapshotFile.delete();
+        if (!deleted) {
+            throw new FlowPersistenceException("Unable to delete snapshot at " + snapshotFile.getAbsolutePath());
+        }
+
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Deleted snapshot at {}", new Object[] {snapshotFile.getAbsolutePath()});
+        }
+    }
+
+    protected File getSnapshotFile(final String bucketId, final String flowId, final int version) {
+        final String snapshotFilename = bucketId + "/" + flowId + "/" + version + "/" + version + SNAPSHOT_EXTENSION;
+        return new File(flowStorageDir, snapshotFilename);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/StandardFlowSnapshotContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/StandardFlowSnapshotContext.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/StandardFlowSnapshotContext.java
new file mode 100644
index 0000000..5b1f058
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/StandardFlowSnapshotContext.java
@@ -0,0 +1,157 @@
+/*
+ * 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.provider.flow;
+
+import org.apache.commons.lang3.Validate;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.flow.FlowSnapshotContext;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+
+/**
+ * Standard implementation of FlowSnapshotContext.
+ */
+public class StandardFlowSnapshotContext implements FlowSnapshotContext {
+
+    private final String bucketId;
+    private final String bucketName;
+    private final String flowId;
+    private final String flowName;
+    private final int version;
+    private final String comments;
+    private final long snapshotTimestamp;
+
+    private StandardFlowSnapshotContext(final Builder builder) {
+        this.bucketId = builder.bucketId;
+        this.bucketName = builder.bucketName;
+        this.flowId = builder.flowId;
+        this.flowName = builder.flowName;
+        this.version = builder.version;
+        this.comments = builder.comments;
+        this.snapshotTimestamp = builder.snapshotTimestamp;
+
+        Validate.notBlank(bucketId);
+        Validate.notBlank(bucketName);
+        Validate.notBlank(flowId);
+        Validate.notBlank(flowName);
+        Validate.isTrue(version > 0);
+        Validate.isTrue(snapshotTimestamp > 0);
+    }
+
+    @Override
+    public String getBucketId() {
+        return bucketId;
+    }
+
+    @Override
+    public String getBucketName() {
+        return bucketName;
+    }
+
+    @Override
+    public String getFlowId() {
+        return flowId;
+    }
+
+    @Override
+    public String getFlowName() {
+        return flowName;
+    }
+
+    @Override
+    public int getVersion() {
+        return version;
+    }
+
+    @Override
+    public String getComments() {
+        return comments;
+    }
+
+    @Override
+    public long getSnapshotTimestamp() {
+        return snapshotTimestamp;
+    }
+
+    /**
+     * Builder for creating instances of StandardFlowSnapshotContext.
+     */
+    public static class Builder {
+
+        private String bucketId;
+        private String bucketName;
+        private String flowId;
+        private String flowName;
+        private int version;
+        private String comments;
+        private long snapshotTimestamp;
+
+        public Builder() {
+
+        }
+
+        public Builder(final Bucket bucket, final VersionedFlowSnapshotMetadata snapshotMetadata) {
+            bucketId(bucket.getIdentifier());
+            bucketName(bucket.getName());
+            flowId(snapshotMetadata.getFlowIdentifier());
+            flowName(snapshotMetadata.getFlowName());
+            version(snapshotMetadata.getVersion());
+            comments(snapshotMetadata.getComments());
+            snapshotTimestamp(snapshotMetadata.getTimestamp());
+        }
+
+        public Builder bucketId(final String bucketId) {
+            this.bucketId = bucketId;
+            return this;
+        }
+
+        public Builder bucketName(final String bucketName) {
+            this.bucketName = bucketName;
+            return this;
+        }
+
+        public Builder flowId(final String flowId) {
+            this.flowId = flowId;
+            return this;
+        }
+
+        public Builder flowName(final String flowName) {
+            this.flowName = flowName;
+            return this;
+        }
+
+        public Builder version(final int version) {
+            this.version = version;
+            return this;
+        }
+
+        public Builder comments(final String comments) {
+            this.comments = comments;
+            return this;
+        }
+
+        public Builder snapshotTimestamp(final long snapshotTimestamp) {
+            this.snapshotTimestamp = snapshotTimestamp;
+            return this;
+        }
+
+        public StandardFlowSnapshotContext build() {
+            return new StandardFlowSnapshotContext(this);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/serialization/jaxb/JAXBSerializer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/serialization/jaxb/JAXBSerializer.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/serialization/jaxb/JAXBSerializer.java
index eff655d..515de10 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/serialization/jaxb/JAXBSerializer.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/serialization/jaxb/JAXBSerializer.java
@@ -40,7 +40,7 @@ public class JAXBSerializer<T> implements Serializer<T> {
         try {
             this.jaxbContext = JAXBContext.newInstance(clazz);
         } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.", e);
+            throw new RuntimeException("Unable to create JAXBContext: " + e.getMessage(), e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
index c80dc21..b4448a5 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
@@ -17,15 +17,16 @@
 package org.apache.nifi.registry.service;
 
 import org.apache.nifi.registry.bucket.Bucket;
+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.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntityKey;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.BucketMetadata;
-import org.apache.nifi.registry.metadata.FlowMetadata;
-import org.apache.nifi.registry.metadata.FlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.StandardBucketMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowSnapshotMetadata;
 
+import java.util.Date;
 import java.util.LinkedHashSet;
 import java.util.Set;
 import java.util.SortedSet;
@@ -36,90 +37,92 @@ import java.util.TreeSet;
  */
 public class DataModelMapper {
 
-    public static Bucket map(final BucketMetadata bucketMetadata) {
+    public static BucketEntity map(final Bucket bucket) {
+        final BucketEntity bucketEntity = new BucketEntity();
+        bucketEntity.setId(bucket.getIdentifier());
+        bucketEntity.setName(bucket.getName());
+        bucketEntity.setDescription(bucket.getDescription());
+        bucketEntity.setCreated(new Date(bucket.getCreatedTimestamp()));
+
+        // don't map items on the way in
+
+        return bucketEntity;
+    }
+
+    public static Bucket map(final BucketEntity bucketEntity, final boolean mapChildren) {
         final Bucket bucket = new Bucket();
-        bucket.setIdentifier(bucketMetadata.getIdentifier());
-        bucket.setName(bucketMetadata.getName());
-        bucket.setDescription(bucketMetadata.getDescription());
-        bucket.setCreatedTimestamp(bucketMetadata.getCreatedTimestamp());
+        bucket.setIdentifier(bucketEntity.getId());
+        bucket.setName(bucketEntity.getName());
+        bucket.setDescription(bucketEntity.getDescription());
+        bucket.setCreatedTimestamp(bucketEntity.getCreated().getTime());
 
-        if (bucketMetadata.getFlowMetadata() != null) {
+        if (mapChildren && bucketEntity.getItems() != null) {
             final Set<VersionedFlow> flows = new LinkedHashSet<>();
-            bucketMetadata.getFlowMetadata().stream().forEach(f -> flows.add(map(f)));
+            for (final BucketItemEntity itemEntity : bucketEntity.getItems()) {
+                if (BucketItemEntityType.FLOW == itemEntity.getType()) {
+                    // we never return the snapshots when retrieving a bucket
+                    flows.add(map((FlowEntity) itemEntity, false));
+                }
+            }
             bucket.setVersionedFlows(flows);
         }
 
         return bucket;
     }
 
-    public static BucketMetadata map(final Bucket bucket) {
-        final StandardBucketMetadata.Builder builder = new StandardBucketMetadata.Builder()
-                .identifier(bucket.getIdentifier())
-                .name(bucket.getName())
-                .description(bucket.getDescription())
-                .created(bucket.getCreatedTimestamp());
+    public static FlowEntity map(final VersionedFlow versionedFlow) {
+        final FlowEntity flowEntity = new FlowEntity();
+        flowEntity.setId(versionedFlow.getIdentifier());
+        flowEntity.setName(versionedFlow.getName());
+        flowEntity.setDescription(versionedFlow.getDescription());
+        flowEntity.setCreated(new Date(versionedFlow.getCreatedTimestamp()));
+        flowEntity.setModified(new Date(versionedFlow.getModifiedTimestamp()));
+        flowEntity.setType(BucketItemEntityType.FLOW);
 
-        if (bucket.getVersionedFlows() != null) {
-            bucket.getVersionedFlows().stream().forEach(f -> builder.addFlow(map(f)));
-        }
+        // don't map snapshots on the way in
 
-        return builder.build();
+        return flowEntity;
     }
 
-    public static VersionedFlow map(final FlowMetadata flowMetadata) {
+    public static VersionedFlow map(final FlowEntity flowEntity, boolean mapChildren) {
         final VersionedFlow versionedFlow = new VersionedFlow();
-        versionedFlow.setIdentifier(flowMetadata.getIdentifier());
-        versionedFlow.setName(flowMetadata.getName());
-        versionedFlow.setBucketIdentifier(flowMetadata.getBucketIdentifier());
-        versionedFlow.setDescription(flowMetadata.getDescription());
-        versionedFlow.setCreatedTimestamp(flowMetadata.getCreatedTimestamp());
-        versionedFlow.setModifiedTimestamp(flowMetadata.getModifiedTimestamp());
-
-        if (flowMetadata.getSnapshotMetadata() != null) {
+        versionedFlow.setIdentifier(flowEntity.getId());
+        versionedFlow.setBucketIdentifier(flowEntity.getBucket().getId());
+        versionedFlow.setName(flowEntity.getName());
+        versionedFlow.setDescription(flowEntity.getDescription());
+        versionedFlow.setCreatedTimestamp(flowEntity.getCreated().getTime());
+        versionedFlow.setModifiedTimestamp(flowEntity.getModified().getTime());
+
+        if (mapChildren && flowEntity.getSnapshots() != null) {
             final SortedSet<VersionedFlowSnapshotMetadata> snapshots = new TreeSet<>();
-            flowMetadata.getSnapshotMetadata().stream().forEach(s -> snapshots.add(map(s)));
+            flowEntity.getSnapshots().stream().forEach(s -> snapshots.add(map(s)));
             versionedFlow.setSnapshotMetadata(snapshots);
         }
 
         return versionedFlow;
     }
 
-    public static FlowMetadata map(final VersionedFlow versionedFlow) {
-        final StandardFlowMetadata.Builder builder = new StandardFlowMetadata.Builder()
-                .identifier(versionedFlow.getIdentifier())
-                .name(versionedFlow.getName())
-                .bucketIdentifier(versionedFlow.getBucketIdentifier())
-                .description(versionedFlow.getDescription())
-                .created(versionedFlow.getCreatedTimestamp())
-                .modified(versionedFlow.getModifiedTimestamp());
-
-        if (versionedFlow.getSnapshotMetadata() != null) {
-            versionedFlow.getSnapshotMetadata().stream().forEach(s -> builder.addSnapshot(map(s)));
-        }
+    public static FlowSnapshotEntity map(final VersionedFlowSnapshotMetadata versionedFlowSnapshot) {
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId(versionedFlowSnapshot.getFlowIdentifier());
+        key.setVersion(versionedFlowSnapshot.getVersion());
 
-        return builder.build();
+        final FlowSnapshotEntity flowSnapshotEntity = new FlowSnapshotEntity();
+        flowSnapshotEntity.setId(key);
+        flowSnapshotEntity.setComments(versionedFlowSnapshot.getComments());
+        flowSnapshotEntity.setCreated(new Date(versionedFlowSnapshot.getTimestamp()));
+        return flowSnapshotEntity;
     }
 
-    public static VersionedFlowSnapshotMetadata map(final FlowSnapshotMetadata flowSnapshotMetadata) {
+    public static VersionedFlowSnapshotMetadata map(final FlowSnapshotEntity flowSnapshotEntity) {
         final VersionedFlowSnapshotMetadata metadata = new VersionedFlowSnapshotMetadata();
-        metadata.setBucketIdentifier(flowSnapshotMetadata.getBucketIdentifier());
-        metadata.setFlowIdentifier(flowSnapshotMetadata.getFlowIdentifier());
-        metadata.setFlowName(flowSnapshotMetadata.getFlowName());
-        metadata.setComments(flowSnapshotMetadata.getComments());
-        metadata.setTimestamp(flowSnapshotMetadata.getCreatedTimestamp());
-        metadata.setVersion(flowSnapshotMetadata.getVersion());
+        metadata.setFlowIdentifier(flowSnapshotEntity.getId().getFlowId());
+        metadata.setVersion(flowSnapshotEntity.getId().getVersion());
+        metadata.setBucketIdentifier(flowSnapshotEntity.getFlow().getBucket().getId());
+        metadata.setFlowName(flowSnapshotEntity.getFlow().getName());
+        metadata.setComments(flowSnapshotEntity.getComments());
+        metadata.setTimestamp(flowSnapshotEntity.getCreated().getTime());
         return metadata;
     }
 
-    public static FlowSnapshotMetadata map(final VersionedFlowSnapshotMetadata metadata) {
-        return new StandardFlowSnapshotMetadata.Builder()
-                .bucketIdentifier(metadata.getBucketIdentifier())
-                .flowIdentifier(metadata.getFlowIdentifier())
-                .flowName(metadata.getFlowName())
-                .comments(metadata.getComments())
-                .created(metadata.getTimestamp())
-                .version(metadata.getVersion())
-                .build();
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/MetadataService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/MetadataService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/MetadataService.java
new file mode 100644
index 0000000..3c17cee
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/MetadataService.java
@@ -0,0 +1,186 @@
+/*
+ * 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.service;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.apache.nifi.registry.service.params.QueryParameters;
+
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A service for managing metadata about all objects stored by the registry.
+ *
+ */
+public interface MetadataService {
+
+    /**
+     * Creates the given bucket.
+     *
+     * @param bucket the bucket to create
+     * @return the created bucket
+     */
+    BucketEntity createBucket(BucketEntity bucket);
+
+    /**
+     * Retrieves the bucket with the given id.
+     *
+     * @param bucketIdentifier the id of the bucket to retrieve
+     * @return the bucket with the given id, or null if it does not exist
+     */
+    BucketEntity getBucketById(String bucketIdentifier);
+
+    /**
+     * Retrieves the buckets with the given name. The name comparison must be case-insensitive.
+     *
+     * @param name the name of the bucket to retrieve
+     * @return the buckets with the given name, or empty list if none exist
+     */
+    List<BucketEntity> getBucketsByName(String name);
+
+    /**
+     * Updates the given bucket, only the name and description should be allowed to be updated.
+     *
+     * @param bucket the updated bucket to save
+     * @return the updated bucket, or null if no bucket with the given id exists
+     */
+    BucketEntity updateBucket(BucketEntity bucket);
+
+    /**
+     * Deletes the bucket, as well as any objects that reference the bucket.
+     *
+     * @param bucket the bucket to delete
+     */
+    void deleteBucket(BucketEntity bucket);
+
+    /**
+     * Retrieves all buckets known to this metadata provider.
+     *
+     * @param params the paging and sorting params, or null
+     * @return the set of all buckets
+     */
+    List<BucketEntity> getBuckets(QueryParameters params);
+
+    /**
+     * Retrieves items across all buckets.
+     *
+     * @param queryParameters the parameters for retrieving the items, or null
+     * @return the set of all items
+     */
+    List<BucketItemEntity> getBucketItems(QueryParameters queryParameters);
+
+    /**
+     * Retrieves items for the given bucket.
+     *
+     * @param bucket the bucket to retrieve items for
+     * @param queryParameters the parameters for retrieving the items, or null
+     * @return the set of items for the bucket
+     */
+    List<BucketItemEntity> getBucketItems(QueryParameters queryParameters, BucketEntity bucket);
+
+    /**
+     * Creates a versioned flow in the given bucket.
+     *
+     * @param flow the versioned flow to create
+     * @return the created versioned flow
+     * @throws IllegalStateException if no bucket with the given identifier exists
+     */
+    FlowEntity createFlow(FlowEntity flow);
+
+    /**
+     * Retrieves the versioned flow with the given id.
+     *
+     * @param flowIdentifier the identifier of the flow to retrieve
+     * @return the versioned flow with the given id, or null if no flow with the given id exists
+     */
+    FlowEntity getFlowById(String flowIdentifier);
+
+    /**
+     * Retrieves the versioned flows with the given name. The name comparison must be case-insensitive.
+     *
+     * @param name the name of the flow to retrieve
+     * @return the versioned flows with the given name, or empty list if no flows with the given name exists
+     */
+    List<FlowEntity> getFlowsByName(String name);
+
+    /**
+     * Updates the given versioned flow, only the name and description should be allowed to be updated.
+     *
+     * @param flow the updated versioned flow to save
+     * @return the updated versioned flow
+     */
+    FlowEntity updateFlow(FlowEntity flow);
+
+    /**
+     * Deletes the flow if one exists.
+     *
+     * @param flow the flow to delete
+     */
+    void deleteFlow(FlowEntity flow);
+
+    /**
+     * Retrieves all versioned flows known to this metadata provider.
+     *
+     * @param queryParameters the paging and sorting params, or null
+     * @return the set of all versioned flows
+     */
+    List<FlowEntity> getFlows(QueryParameters queryParameters);
+
+    /**
+     * Creates a versioned flow snapshot.
+     *
+     * @param flowSnapshot the snapshot to create
+     * @return the created snapshot
+     * @throws IllegalStateException if the versioned flow specified by flowSnapshot.getFlowIdentifier() does not exist
+     */
+    FlowSnapshotEntity createFlowSnapshot(FlowSnapshotEntity flowSnapshot);
+
+    /**
+     * Retrieves the snapshot for the given flow identifier and snapshot version.
+     *
+     * @param flowIdentifier the identifier of the flow the snapshot belongs to
+     * @param version the version of the snapshot
+     * @return the versioned flow snapshot for the given flow identifier and version, or null if none exists
+     */
+    FlowSnapshotEntity getFlowSnapshot(String flowIdentifier, Integer version);
+
+    /**
+     * Deletes the flow snapshot.
+     *
+     * @param flowSnapshot the flow snapshot to delete
+     */
+    void deleteFlowSnapshot(FlowSnapshotEntity flowSnapshot);
+
+    /**
+     * @return the set of field names for Buckets
+     */
+    Set<String> getBucketFields();
+
+    /**
+     * @return the set of field names for BucketItems
+     */
+    Set<String> getBucketItemFields();
+
+    /**
+     * @return the set of field names for Flows
+     */
+    Set<String> getFlowFields();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
index 6ca999c..fc067e6 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
@@ -17,23 +17,27 @@
 package org.apache.nifi.registry.service;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.Validate;
 import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.bucket.BucketItem;
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
 import org.apache.nifi.registry.exception.ResourceNotFoundException;
 import org.apache.nifi.registry.flow.FlowPersistenceProvider;
 import org.apache.nifi.registry.flow.FlowSnapshotContext;
-import org.apache.nifi.registry.flow.StandardFlowSnapshotContext;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.BucketMetadata;
-import org.apache.nifi.registry.metadata.FlowMetadata;
-import org.apache.nifi.registry.metadata.FlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.MetadataProvider;
-import org.apache.nifi.registry.metadata.StandardBucketMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowMetadata;
+import org.apache.nifi.registry.provider.flow.StandardFlowSnapshotContext;
 import org.apache.nifi.registry.serialization.Serializer;
+import org.apache.nifi.registry.service.params.QueryParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
+import org.springframework.transaction.annotation.Transactional;
 
 import javax.validation.ConstraintViolation;
 import javax.validation.ConstraintViolationException;
@@ -41,17 +45,29 @@ import javax.validation.Validator;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.InputStream;
-import java.util.Objects;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 
+/**
+ * Main service for all back-end operations, REST resources should only interact with this service.
+ *
+ * This service is marked as @Transactional so that Spring will automatically start a transaction upon entering
+ * any method, and will rollback the transaction if any Exception is thrown out of a method.
+ *
+ */
 @Service
+@Transactional(rollbackFor = Exception.class)
 public class RegistryService {
 
-    private final MetadataProvider metadataProvider;
+    private static final Logger LOGGER = LoggerFactory.getLogger(RegistryService.class);
+
+    private final MetadataService metadataService;
     private final FlowPersistenceProvider flowPersistenceProvider;
     private final Serializer<VersionedFlowSnapshot> snapshotSerializer;
     private final Validator validator;
@@ -60,18 +76,19 @@ public class RegistryService {
     private final Lock readLock = lock.readLock();
     private final Lock writeLock = lock.writeLock();
 
-    public RegistryService(@Autowired final MetadataProvider metadataProvider,
-                           @Autowired final FlowPersistenceProvider flowPersistenceProvider,
-                           @Autowired final Serializer<VersionedFlowSnapshot> snapshotSerializer,
-                           @Autowired final Validator validator) {
-        this.metadataProvider = metadataProvider;
+    @Autowired
+    public RegistryService(final MetadataService metadataService,
+                           final FlowPersistenceProvider flowPersistenceProvider,
+                           final Serializer<VersionedFlowSnapshot> snapshotSerializer,
+                           final Validator validator) {
+        this.metadataService = metadataService;
         this.flowPersistenceProvider = flowPersistenceProvider;
         this.snapshotSerializer = snapshotSerializer;
         this.validator = validator;
-        Objects.requireNonNull(this.metadataProvider);
-        Objects.requireNonNull(this.flowPersistenceProvider);
-        Objects.requireNonNull(this.snapshotSerializer);
-        Objects.requireNonNull(this.validator);
+        Validate.notNull(this.metadataService);
+        Validate.notNull(this.flowPersistenceProvider);
+        Validate.notNull(this.snapshotSerializer);
+        Validate.notNull(this.validator);
     }
 
     private <T>  void validate(T t, String invalidMessage) {
@@ -97,41 +114,41 @@ public class RegistryService {
 
         writeLock.lock();
         try {
-            final BucketMetadata existingBucketWithSameName = metadataProvider.getBucketByName(bucket.getName());
-            if (existingBucketWithSameName != null) {
-                throw new IllegalStateException("A bucket with the same name already exists: " + existingBucketWithSameName.getIdentifier());
+            final List<BucketEntity> bucketsWithSameName = metadataService.getBucketsByName(bucket.getName());
+            if (bucketsWithSameName.size() > 0) {
+                throw new IllegalStateException("A bucket with the same name already exists");
             }
 
-            final BucketMetadata createdBucket = metadataProvider.createBucket(DataModelMapper.map(bucket));
-            return DataModelMapper.map(createdBucket);
+            final BucketEntity createdBucket = metadataService.createBucket(DataModelMapper.map(bucket));
+            return DataModelMapper.map(createdBucket, false);
         } finally {
             writeLock.unlock();
         }
     }
 
-    public Bucket getBucket(final String bucketIdentifier) {
+    public Bucket getBucket(final String bucketIdentifier, final boolean verbose) {
         if (bucketIdentifier == null) {
             throw new IllegalArgumentException("Bucket Identifier cannot be null");
         }
 
         readLock.lock();
         try {
-            final BucketMetadata bucket = metadataProvider.getBucketById(bucketIdentifier);
+            final BucketEntity bucket = metadataService.getBucketById(bucketIdentifier);
             if (bucket == null) {
                 throw new ResourceNotFoundException("Bucket does not exist for identifier: " + bucketIdentifier);
             }
 
-            return DataModelMapper.map(bucket);
+            return DataModelMapper.map(bucket, verbose);
         } finally {
             readLock.unlock();
         }
     }
 
-    public Set<Bucket> getBuckets() {
+    public List<Bucket> getBuckets(final QueryParameters queryParameters) {
         readLock.lock();
         try {
-            final Set<BucketMetadata> buckets = metadataProvider.getBuckets();
-            return buckets.stream().map(b -> DataModelMapper.map(b)).collect(Collectors.toSet());
+            final List<BucketEntity> buckets = metadataService.getBuckets(queryParameters);
+            return buckets.stream().map(b -> DataModelMapper.map(b, false)).collect(Collectors.toList());
         } finally {
             readLock.unlock();
         }
@@ -149,7 +166,7 @@ public class RegistryService {
         writeLock.lock();
         try {
             // ensure a bucket with the given id exists
-            final BucketMetadata existingBucketById = metadataProvider.getBucketById(bucket.getIdentifier());
+            final BucketEntity existingBucketById = metadataService.getBucketById(bucket.getIdentifier());
             if (existingBucketById == null) {
                 throw new ResourceNotFoundException("Bucket does not exist for identifier: " + bucket.getIdentifier());
             }
@@ -157,26 +174,28 @@ public class RegistryService {
             // ensure a different bucket with the same name does not exist
             // since we're allowing partial updates here, only check this if a non-null name is provided
             if (StringUtils.isNotBlank(bucket.getName())) {
-                final BucketMetadata existingBucketWithSameName = metadataProvider.getBucketByName(bucket.getName());
-                if (existingBucketWithSameName != null && !existingBucketWithSameName.getIdentifier().equals(existingBucketById.getIdentifier())) {
-                    throw new IllegalStateException("A bucket with the same name already exists: " + bucket.getName());
+                final List<BucketEntity> bucketsWithSameName = metadataService.getBucketsByName(bucket.getName());
+                if (bucketsWithSameName != null) {
+                    for (final BucketEntity bucketWithSameName : bucketsWithSameName) {
+                        if (!bucketWithSameName.getId().equals(existingBucketById.getId())){
+                            throw new IllegalStateException("A bucket with the same name already exists: " + bucket.getName());
+                        }
+                    }
                 }
             }
 
-            final StandardBucketMetadata.Builder builder = new StandardBucketMetadata.Builder(existingBucketById);
-
             // transfer over the new values to the existing bucket
             if (StringUtils.isNotBlank(bucket.getName())) {
-                builder.name(bucket.getName());
+                existingBucketById.setName(bucket.getName());
             }
 
             if (bucket.getDescription() != null) {
-                builder.description(bucket.getDescription());
+                existingBucketById.setDescription(bucket.getDescription());
             }
 
             // perform the actual update
-            final BucketMetadata updatedBucket = metadataProvider.updateBucket(builder.build());
-            return DataModelMapper.map(updatedBucket);
+            final BucketEntity updatedBucket = metadataService.updateBucket(existingBucketById);
+            return DataModelMapper.map(updatedBucket, false);
         } finally {
             writeLock.unlock();
         }
@@ -190,28 +209,67 @@ public class RegistryService {
         writeLock.lock();
         try {
             // ensure the bucket exists
-            final BucketMetadata existingBucket = metadataProvider.getBucketById(bucketIdentifier);
+            final BucketEntity existingBucket = metadataService.getBucketById(bucketIdentifier);
             if (existingBucket == null) {
                 throw new ResourceNotFoundException("Bucket does not exist for identifier: " + bucketIdentifier);
             }
 
-            // retrieve the versioned flows that are in this bucket
-            final Set<FlowMetadata> bucketFlows = metadataProvider.getFlows(bucketIdentifier);
-
             // for each flow in the bucket, delete all snapshots from the flow persistence provider
-            for (final FlowMetadata bucketFlow : bucketFlows) {
-                flowPersistenceProvider.deleteSnapshots(bucketIdentifier, bucketFlow.getIdentifier());
+            for (final FlowEntity flowEntity : existingBucket.getFlows()) {
+                flowPersistenceProvider.deleteSnapshots(bucketIdentifier, flowEntity.getId());
             }
 
             // now delete the bucket from the metadata provider, which deletes all flows referencing it
-            metadataProvider.deleteBucket(bucketIdentifier);
+            metadataService.deleteBucket(existingBucket);
 
-            return DataModelMapper.map(existingBucket);
+            return DataModelMapper.map(existingBucket, false);
         } finally {
             writeLock.unlock();
         }
     }
 
+    // ---------------------- BucketItem methods ---------------------------------------------
+
+    public List<BucketItem> getBucketItems(final QueryParameters queryParameters) {
+        readLock.lock();
+        try {
+            final List<BucketItem> bucketItems = new ArrayList<>();
+            metadataService.getBucketItems(queryParameters).stream().forEach(b -> addBucketItem(bucketItems, b));
+            return bucketItems;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    public List<BucketItem> getBucketItems(final QueryParameters queryParameters, final String bucketIdentifier) {
+        if (bucketIdentifier == null) {
+            throw new IllegalArgumentException("Bucket Identifier cannot be null");
+        }
+
+        readLock.lock();
+        try {
+            final BucketEntity bucket = metadataService.getBucketById(bucketIdentifier);
+            if (bucket == null) {
+                throw new ResourceNotFoundException("Bucket does not exist for identifier: " + bucketIdentifier);
+            }
+
+            final List<BucketItem> bucketItems = new ArrayList<>();
+            metadataService.getBucketItems(queryParameters, bucket).stream().forEach(b -> addBucketItem(bucketItems, b));
+            return bucketItems;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    private void addBucketItem(final List<BucketItem> bucketItems, final BucketItemEntity itemEntity) {
+        if (itemEntity instanceof FlowEntity) {
+            final FlowEntity flowEntity = (FlowEntity) itemEntity;
+            bucketItems.add(DataModelMapper.map(flowEntity, false));
+        } else {
+            LOGGER.error("Unknown type of BucketItemEntity: " + itemEntity.getClass().getCanonicalName());
+        }
+    }
+
     // ---------------------- VersionedFlow methods ---------------------------------------------
 
     public VersionedFlow createFlow(final String bucketIdentifier, final VersionedFlow versionedFlow) {
@@ -245,66 +303,73 @@ public class RegistryService {
         writeLock.lock();
         try {
             // ensure the bucket exists
-            final BucketMetadata existingBucket = metadataProvider.getBucketById(bucketIdentifier);
+            final BucketEntity existingBucket = metadataService.getBucketById(bucketIdentifier);
             if (existingBucket == null) {
                 throw new ResourceNotFoundException("Bucket does not exist for identifier: " + bucketIdentifier);
             }
 
-            final FlowMetadata existingFlowWithSameName = metadataProvider.getFlowByName(versionedFlow.getName());
-            if (existingFlowWithSameName != null) {
-                throw new IllegalStateException("A VersionedFlow with the same name already exists: " + existingFlowWithSameName.getIdentifier());
+            // ensure another flow with the same name doesn't exist
+            final List<FlowEntity> flowsWithSameName = metadataService.getFlowsByName(versionedFlow.getName());
+            if (flowsWithSameName != null && flowsWithSameName.size() > 0) {
+                throw new IllegalStateException("A VersionedFlow with the same name already exists");
             }
 
-            // create the flow
-            final FlowMetadata createdFlow = metadataProvider.createFlow(bucketIdentifier, DataModelMapper.map(versionedFlow));
-            return DataModelMapper.map(createdFlow);
+            // convert from dto to entity and set the bucket relationship
+            final FlowEntity flowEntity = DataModelMapper.map(versionedFlow);
+            flowEntity.setBucket(existingBucket);
+
+            // persist the flow and return the created entity
+            final FlowEntity createdFlow = metadataService.createFlow(flowEntity);
+            return DataModelMapper.map(createdFlow, false);
         } finally {
             writeLock.unlock();
         }
     }
 
-    public VersionedFlow getFlow(final String flowIdentifier) {
+    public VersionedFlow getFlow(final String flowIdentifier, final boolean verbose) {
         if (StringUtils.isBlank(flowIdentifier)) {
             throw new IllegalArgumentException("Flow Identifier cannot be null or blank");
         }
 
         readLock.lock();
         try {
-            final FlowMetadata flowMetadata = metadataProvider.getFlowById(flowIdentifier);
-            if (flowMetadata == null) {
+            final FlowEntity flowEntity = metadataService.getFlowById(flowIdentifier);
+            if (flowEntity == null) {
                 throw new ResourceNotFoundException("VersionedFlow does not exist for identifier: " + flowIdentifier);
             }
 
-            return DataModelMapper.map(flowMetadata);
+            return DataModelMapper.map(flowEntity, verbose);
         } finally {
             readLock.unlock();
         }
     }
 
-    public Set<VersionedFlow> getFlows() {
+    public List<VersionedFlow> getFlows(final QueryParameters queryParameters) {
         readLock.lock();
         try {
-            final Set<FlowMetadata> flows = metadataProvider.getFlows();
-            return flows.stream().map(f -> DataModelMapper.map(f)).collect(Collectors.toSet());
+            // return non-verbose set of all flows
+            final List<FlowEntity> flows = metadataService.getFlows(queryParameters);
+            return flows.stream().map(f -> DataModelMapper.map(f, false)).collect(Collectors.toList());
         } finally {
             readLock.unlock();
         }
     }
 
-    public Set<VersionedFlow> getFlows(String bucketId) {
+    public List<VersionedFlow> getFlows(final String bucketId) {
         if (StringUtils.isBlank(bucketId)) {
             throw new IllegalArgumentException("Bucket Identifier cannot be null");
         }
 
         readLock.lock();
         try {
-            final BucketMetadata existingBucket = metadataProvider.getBucketById(bucketId);
+            final BucketEntity existingBucket = metadataService.getBucketById(bucketId);
             if (existingBucket == null) {
                 throw new ResourceNotFoundException("Bucket does not exist for identifier: " + bucketId);
             }
 
-            final Set<FlowMetadata> flows = metadataProvider.getFlows(bucketId);
-            return flows.stream().map(f -> DataModelMapper.map(f)).collect(Collectors.toSet());
+            // return non-verbose set of flows for the given bucket
+            final Set<FlowEntity> flows = existingBucket.getFlows();
+            return flows.stream().map(f -> DataModelMapper.map(f, false)).collect(Collectors.toList());
         } finally {
             readLock.unlock();
         }
@@ -322,7 +387,7 @@ public class RegistryService {
         writeLock.lock();
         try {
             // ensure a flow with the given id exists
-            final FlowMetadata existingFlow = metadataProvider.getFlowById(versionedFlow.getIdentifier());
+            final FlowEntity existingFlow = metadataService.getFlowById(versionedFlow.getIdentifier());
             if (existingFlow == null) {
                 throw new ResourceNotFoundException("VersionedFlow does not exist for identifier: " + versionedFlow.getIdentifier());
             }
@@ -330,28 +395,28 @@ public class RegistryService {
             // ensure a different flow with the same name does not exist
             // since we're allowing partial updates here, only check this if a non-null name is provided
             if (StringUtils.isNotBlank(versionedFlow.getName())) {
-                final FlowMetadata existingFlowWithSameName = metadataProvider.getFlowByName(versionedFlow.getName());
-                if (existingFlowWithSameName != null && !existingFlowWithSameName.getIdentifier().equals(existingFlow.getIdentifier())) {
-                    throw new IllegalStateException("A VersionedFlow with the same name already exists: " + versionedFlow.getName());
+                final List<FlowEntity> flowsWithSameName = metadataService.getFlowsByName(versionedFlow.getName());
+                if (flowsWithSameName != null) {
+                    for (final FlowEntity flowWithSameName : flowsWithSameName) {
+                         if(!flowWithSameName.getId().equals(existingFlow.getId())) {
+                            throw new IllegalStateException("A VersionedFlow with the same name already exists: " + versionedFlow.getName());
+                        }
+                    }
                 }
             }
 
-            final StandardFlowMetadata.Builder builder = new StandardFlowMetadata.Builder(existingFlow);
-
             // transfer over the new values to the existing flow
             if (StringUtils.isNotBlank(versionedFlow.getName())) {
-                builder.name(versionedFlow.getName());
+                existingFlow.setName(versionedFlow.getName());
             }
 
             if (versionedFlow.getDescription() != null) {
-                builder.description(versionedFlow.getDescription());
+                existingFlow.setDescription(versionedFlow.getDescription());
             }
 
-            builder.modified(System.currentTimeMillis());
-
             // perform the actual update
-            final FlowMetadata updatedFlow = metadataProvider.updateFlow(builder.build());
-            return DataModelMapper.map(updatedFlow);
+            final FlowEntity updatedFlow = metadataService.updateFlow(existingFlow);
+            return DataModelMapper.map(updatedFlow, false);
         } finally {
             writeLock.unlock();
         }
@@ -365,18 +430,18 @@ public class RegistryService {
         writeLock.lock();
         try {
             // ensure the flow exists
-            final FlowMetadata existingFlow = metadataProvider.getFlowById(flowIdentifier);
+            final FlowEntity existingFlow = metadataService.getFlowById(flowIdentifier);
             if (existingFlow == null) {
                 throw new ResourceNotFoundException("VersionedFlow does not exist for identifier: " + flowIdentifier);
             }
 
             // delete all snapshots from the flow persistence provider
-            flowPersistenceProvider.deleteSnapshots(existingFlow.getBucketIdentifier(), existingFlow.getIdentifier());
+            flowPersistenceProvider.deleteSnapshots(existingFlow.getBucket().getId(), existingFlow.getId());
 
             // now delete the flow from the metadata provider
-            metadataProvider.deleteFlow(flowIdentifier);
+            metadataService.deleteFlow(existingFlow);
 
-            return DataModelMapper.map(existingFlow);
+            return DataModelMapper.map(existingFlow, false);
         } finally {
             writeLock.unlock();
         }
@@ -401,22 +466,22 @@ public class RegistryService {
             final VersionedFlowSnapshotMetadata snapshotMetadata = flowSnapshot.getSnapshotMetadata();
 
             // ensure the bucket exists
-            final BucketMetadata existingBucket = metadataProvider.getBucketById(snapshotMetadata.getBucketIdentifier());
+            final BucketEntity existingBucket = metadataService.getBucketById(snapshotMetadata.getBucketIdentifier());
             if (existingBucket == null) {
                 throw new ResourceNotFoundException("Bucket does not exist for identifier: " + snapshotMetadata.getBucketIdentifier());
             }
 
             // ensure the flow exists
-            final FlowMetadata existingFlowMetadata = metadataProvider.getFlowById(snapshotMetadata.getFlowIdentifier());
-            if (existingFlowMetadata == null) {
+            final FlowEntity existingFlow = metadataService.getFlowById(snapshotMetadata.getFlowIdentifier());
+            if (existingFlow == null) {
                 throw new ResourceNotFoundException("VersionedFlow does not exist for identifier: " + snapshotMetadata.getFlowIdentifier());
             }
 
-            final VersionedFlow existingFlow = DataModelMapper.map(existingFlowMetadata);
+            final VersionedFlow versionedFlow = DataModelMapper.map(existingFlow, true);
 
             // if we already have snapshots we need to verify the new one has the correct version
-            if (existingFlow.getSnapshotMetadata() != null && existingFlow.getSnapshotMetadata().size() > 0) {
-                final VersionedFlowSnapshotMetadata lastSnapshot = existingFlow.getSnapshotMetadata().last();
+            if (versionedFlow.getSnapshotMetadata() != null && versionedFlow.getSnapshotMetadata().size() > 0) {
+                final VersionedFlowSnapshotMetadata lastSnapshot = versionedFlow.getSnapshotMetadata().last();
 
                 if (snapshotMetadata.getVersion() <= lastSnapshot.getVersion()) {
                     throw new IllegalStateException("A VersionedFlowSnapshot with the same version already exists: " + snapshotMetadata.getVersion());
@@ -436,12 +501,17 @@ public class RegistryService {
             snapshotSerializer.serialize(flowSnapshot, out);
 
             // save the serialized snapshot to the persistence provider
-            final Bucket bucket = DataModelMapper.map(existingBucket);
+            final Bucket bucket = DataModelMapper.map(existingBucket, false);
             final FlowSnapshotContext context = new StandardFlowSnapshotContext.Builder(bucket, snapshotMetadata).build();
             flowPersistenceProvider.saveSnapshot(context, out.toByteArray());
 
             // create snapshot in the metadata provider
-            metadataProvider.createFlowSnapshot(DataModelMapper.map(snapshotMetadata));
+            metadataService.createFlowSnapshot(DataModelMapper.map(snapshotMetadata));
+
+            // update the modified date on the flow
+            existingFlow.setModified(new Date());
+            metadataService.updateFlow(existingFlow);
+
             return flowSnapshot;
         } finally {
             writeLock.unlock();
@@ -460,17 +530,17 @@ public class RegistryService {
         readLock.lock();
         try {
             // ensure the snapshot exists
-            final FlowSnapshotMetadata snapshotMetadata = metadataProvider.getFlowSnapshot(flowIdentifier, version);
-            if (snapshotMetadata == null) {
+            final FlowSnapshotEntity snapshotEntity = metadataService.getFlowSnapshot(flowIdentifier, version);
+            if (snapshotEntity == null) {
                 throw new ResourceNotFoundException("VersionedFlowSnapshot does not exist for flow " + flowIdentifier + " and version " + version);
             }
 
+            final FlowEntity flow = snapshotEntity.getFlow();
+            final String flowId = flow.getId();
+            final String bucketId = flow.getBucket().getId();
+
             // get the serialized bytes of the snapshot
-            final byte[] serializedSnapshot = flowPersistenceProvider.getSnapshot(
-                    snapshotMetadata.getBucketIdentifier(),
-                    snapshotMetadata.getFlowIdentifier(),
-                    snapshotMetadata.getVersion()
-            );
+            final byte[] serializedSnapshot = flowPersistenceProvider.getSnapshot(bucketId, flowId, version);
 
             if (serializedSnapshot == null || serializedSnapshot.length == 0) {
                 throw new IllegalStateException("No serialized content found for snapshot with flow identifier "
@@ -496,24 +566,39 @@ public class RegistryService {
         writeLock.lock();
         try {
             // ensure the snapshot exists
-            final FlowSnapshotMetadata snapshotMetadata = metadataProvider.getFlowSnapshot(flowIdentifier, version);
-            if (snapshotMetadata == null) {
+            final FlowSnapshotEntity snapshotEntity = metadataService.getFlowSnapshot(flowIdentifier, version);
+            if (snapshotEntity == null) {
                 throw new ResourceNotFoundException("VersionedFlowSnapshot does not exist for flow "
                         + flowIdentifier + " and version " + version);
             }
 
+            final FlowEntity flow = snapshotEntity.getFlow();
+            final String flowId = flow.getId();
+            final String bucketId = flow.getBucket().getId();
+
             // delete the content of the snapshot
-            flowPersistenceProvider.deleteSnapshot(
-                    snapshotMetadata.getBucketIdentifier(),
-                    snapshotMetadata.getFlowIdentifier(),
-                    snapshotMetadata.getVersion());
+            flowPersistenceProvider.deleteSnapshot(bucketId, flowId, version);
 
             // delete the snapshot itself
-            metadataProvider.deleteFlowSnapshot(flowIdentifier, version);
-            return DataModelMapper.map(snapshotMetadata);
+            metadataService.deleteFlowSnapshot(snapshotEntity);
+            return DataModelMapper.map(snapshotEntity);
         } finally {
             writeLock.unlock();
         }
     }
 
+    // ---------------------- Field methods ---------------------------------------------
+
+    public Set<String> getBucketFields() {
+        return metadataService.getBucketFields();
+    }
+
+    public Set<String> getBucketItemFields() {
+        return metadataService.getBucketItemFields();
+    }
+
+    public Set<String> getFlowFields() {
+        return metadataService.getFlowFields();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/TestService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/TestService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/TestService.java
deleted file mode 100644
index 3160fc1..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/TestService.java
+++ /dev/null
@@ -1,31 +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.service;
-
-import org.springframework.stereotype.Service;
-
-/**
- * Test service to verify spring-boot will correctly inject into JAX-RS resource TestResource.
- */
-@Service
-public class TestService {
-
-    public String test() {
-        return "Testing";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/QueryParameters.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/QueryParameters.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/QueryParameters.java
new file mode 100644
index 0000000..3c62a9d
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/QueryParameters.java
@@ -0,0 +1,109 @@
+/*
+ * 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.service.params;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Parameters to be passed into service layer for methods that require sorting and paging.
+ */
+public class QueryParameters {
+
+    private final Integer pageNum;
+
+    private final Integer numRows;
+
+    private final List<SortParameter> sortParameters;
+
+    private QueryParameters(final Builder builder) {
+        this.pageNum = builder.pageNum;
+        this.numRows = builder.numRows;
+        this.sortParameters = Collections.unmodifiableList(new ArrayList<>(builder.sortParameters));
+
+        if (this.pageNum != null && this.numRows != null) {
+            if (this.pageNum < 0) {
+                throw new IllegalStateException("Offset cannot be negative");
+            }
+
+            if (this.numRows < 0) {
+                throw new IllegalStateException("Number of rows cannot be negative");
+            }
+        }
+    }
+
+    public Integer getPageNum() {
+        return pageNum;
+    }
+
+    public Integer getNumRows() {
+        return numRows;
+    }
+
+    public List<SortParameter> getSortParameters() {
+        return sortParameters;
+    }
+
+    /**
+     * Builder for QueryParameters.
+     */
+    public static class Builder {
+
+        private Integer pageNum;
+        private Integer numRows;
+        private List<SortParameter> sortParameters = new ArrayList<>();
+
+        public Builder pageNum(Integer pageNum) {
+            this.pageNum = pageNum;
+            return this;
+        }
+
+        public Builder numRows(Integer numRows) {
+            this.numRows = numRows;
+            return this;
+        }
+
+        public Builder addSort(final SortParameter sort) {
+            this.sortParameters.add(sort);
+            return this;
+        }
+
+        public Builder addSort(final String fieldName, final SortOrder order) {
+            this.sortParameters.add(new SortParameter(fieldName, order));
+            return this;
+        }
+
+        public Builder addSorts(final Collection<SortParameter> sorts) {
+            if (sorts != null) {
+                this.sortParameters.addAll(sorts);
+            }
+            return this;
+        }
+
+        public Builder clearSorts() {
+            this.sortParameters.clear();
+            return this;
+        }
+
+        public QueryParameters build() {
+            return new QueryParameters(this);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortOrder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortOrder.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortOrder.java
new file mode 100644
index 0000000..43a3016
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortOrder.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.service.params;
+
+public enum SortOrder {
+
+    ASC("asc"),
+
+    DESC("desc");
+
+    private final String name;
+
+    SortOrder(final String name) {
+        this.name = name;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static SortOrder fromString(String order) {
+        if (ASC.getName().equals(order)) {
+            return  ASC;
+        }
+
+        if (DESC.getName().equals(order)) {
+            return DESC;
+        }
+
+        throw new IllegalArgumentException("Unknown Sort Order: " + order);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortParameter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortParameter.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortParameter.java
new file mode 100644
index 0000000..327bd95
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/params/SortParameter.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.service.params;
+
+/**
+ * Sort parameter made up of a field and a sort order.
+ */
+public class SortParameter {
+
+    private final String fieldName;
+
+    private final SortOrder order;
+
+    public SortParameter(final String fieldName, final SortOrder order) {
+        this.fieldName = fieldName;
+        this.order = order;
+
+        if (this.fieldName == null) {
+            throw new IllegalStateException("Field Name cannot be null");
+        }
+
+        if (this.fieldName.trim().isEmpty()) {
+            throw new IllegalStateException("Field Name cannot be blank");
+        }
+
+        if (this.order == null) {
+            throw new IllegalStateException("Order cannot be null");
+        }
+    }
+
+    public String getFieldName() {
+        return fieldName;
+    }
+
+    public SortOrder getOrder() {
+        return order;
+    }
+
+    /**
+     * Parses a sorting expression of the form field:order.
+     *
+     * @param sortExpression the expression
+     * @return the Sort instance
+     */
+    public static SortParameter fromString(final String sortExpression) {
+        if (sortExpression == null) {
+            throw new IllegalArgumentException("Sort cannot be null");
+        }
+
+        final String[] sortParts = sortExpression.split("[:]");
+        if (sortParts.length != 2) {
+            throw new IllegalArgumentException("Sort must be in the form field:order");
+        }
+
+        final String fieldName = sortParts[0];
+        final SortOrder order = SortOrder.fromString(sortParts[1]);
+
+        return new SortParameter(fieldName, order);
+    }
+
+    @Override
+    public String toString() {
+        return fieldName + ":" + order.getName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/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
new file mode 100644
index 0000000..f7640ed
--- /dev/null
+++ b/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
@@ -0,0 +1,46 @@
+-- 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,
+    COMMENTS VARCHAR(4096),
+    PRIMARY KEY (FLOW_ID, VERSION),
+    FOREIGN KEY (FLOW_ID) REFERENCES FLOW(ID)
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/xsd/providers.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/xsd/providers.xsd b/nifi-registry-framework/src/main/xsd/providers.xsd
index cb71ed8..1202f9e 100644
--- a/nifi-registry-framework/src/main/xsd/providers.xsd
+++ b/nifi-registry-framework/src/main/xsd/providers.xsd
@@ -42,7 +42,6 @@
     <xs:element name="providers">
         <xs:complexType>
             <xs:sequence>
-                <xs:element name="metadataProvider" type="Provider" minOccurs="1" maxOccurs="1"/>
                 <xs:element name="flowPersistenceProvider" type="Provider" minOccurs="1" maxOccurs="1" />
             </xs:sequence>
         </xs:complexType>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/RepositoryTestApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/RepositoryTestApplication.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/RepositoryTestApplication.java
new file mode 100644
index 0000000..de8cd79
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/RepositoryTestApplication.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.db;
+
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.FilterType;
+
+/**
+ * Sets up the application context for database repository tests.
+ *
+ * The @SpringBootTest annotation on the repository tests will find this class by working up the package hierarchy.
+ * This class must be in the "db" package in order to find the entities in "db.entity" and repositories in "db.repository".
+ *
+ * The DataSourceFactory is excluded so that Spring Boot will load an in-memory H2 database.
+ */
+@SpringBootApplication
+@ComponentScan(
+        excludeFilters = {
+                @ComponentScan.Filter(
+                        type = FilterType.ASSIGNABLE_TYPE,
+                        value = DataSourceFactory.class)
+        })
+public class RepositoryTestApplication {
+
+    public static void main(String[] args) {
+        SpringApplication.run(RepositoryTestApplication.class, args);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/RepositoryBaseTest.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/RepositoryBaseTest.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/RepositoryBaseTest.java
new file mode 100644
index 0000000..7ccd01f
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/RepositoryBaseTest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.registry.db.RepositoryTestApplication;
+import org.junit.runner.RunWith;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.test.context.TestExecutionListeners;
+import org.springframework.test.context.junit4.SpringRunner;
+import org.springframework.test.context.support.DependencyInjectionTestExecutionListener;
+import org.springframework.test.context.transaction.TransactionalTestExecutionListener;
+import org.springframework.transaction.annotation.Transactional;
+
+@Transactional
+@RunWith(SpringRunner.class)
+@SpringBootTest(classes = RepositoryTestApplication.class, webEnvironment = SpringBootTest.WebEnvironment.NONE)
+@TestExecutionListeners({DependencyInjectionTestExecutionListener.class, TransactionalTestExecutionListener.class})
+public abstract class RepositoryBaseTest {
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketItemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketItemRepository.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketItemRepository.java
new file mode 100644
index 0000000..657d599
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketItemRepository.java
@@ -0,0 +1,98 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.data.domain.Page;
+import org.springframework.data.domain.PageRequest;
+import org.springframework.data.domain.Sort;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestBucketItemRepository extends RepositoryBaseTest {
+
+    @Autowired
+    private BucketRepository bucketRepository;
+
+    @Autowired
+    private BucketItemRepository bucketItemRepository;
+
+    @Test
+    public void testFindAllPageable() {
+        final Page<BucketItemEntity> page = bucketItemRepository.findAll(new PageRequest(0, 10));
+        assertNotNull(page);
+        assertEquals(1, page.getTotalPages());
+        assertEquals(3, page.getTotalElements());
+
+        final List<BucketItemEntity> entities = page.getContent();
+        assertNotNull(entities);
+        assertEquals(3, entities.size());
+    }
+
+    @Test
+    public void testFindAll() {
+        final Iterable<BucketItemEntity> entities = bucketItemRepository.findAll();
+        assertNotNull(entities);
+
+        int count = 0;
+        for (BucketItemEntity entity : entities) {
+            count++;
+        }
+        assertEquals(3, count);
+    }
+
+    @Test
+    public void testFindByBucket() {
+        final BucketEntity bucket = bucketRepository.findOne("1");
+        assertNotNull(bucket);
+
+        final List<BucketItemEntity> entities = bucketItemRepository.findByBucket(bucket);
+        assertNotNull(entities);
+        assertEquals(2, entities.size());
+    }
+
+    @Test
+    public void testFindByBucketPageable() {
+        final BucketEntity bucket = bucketRepository.findOne("1");
+        assertNotNull(bucket);
+
+        final List<BucketItemEntity> entities = bucketItemRepository.findByBucket(bucket, new PageRequest(0, 2, new Sort(Sort.Direction.ASC, "id")));
+        assertNotNull(entities);
+        assertEquals(2, entities.size());
+        assertEquals("1", entities.get(0).getId());
+        assertEquals("2", entities.get(1).getId());
+    }
+
+    @Test
+    public void testFindByBucketSort() {
+        final BucketEntity bucket = bucketRepository.findOne("1");
+        assertNotNull(bucket);
+
+        final List<BucketItemEntity> entities = bucketItemRepository.findByBucket(bucket, new Sort(Sort.Direction.DESC, "id"));
+        assertNotNull(entities);
+        assertEquals(2, entities.size());
+        assertEquals("2", entities.get(0).getId());
+        assertEquals("1", entities.get(1).getId());
+    }
+
+}


[2/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/metadata/TestFileSystemMetadataProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/metadata/TestFileSystemMetadataProvider.java b/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/metadata/TestFileSystemMetadataProvider.java
deleted file mode 100644
index 5c823b6..0000000
--- a/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/metadata/TestFileSystemMetadataProvider.java
+++ /dev/null
@@ -1,549 +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.metadata;
-
-import org.apache.nifi.registry.provider.ProviderConfigurationContext;
-import org.apache.nifi.registry.provider.ProviderCreationException;
-import org.apache.nifi.registry.util.FileUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-public class TestFileSystemMetadataProvider {
-
-    static final String METADATA_EMPTY = "metadata-empty.xml";
-    static final String METADATA_EMPTY_CONCISE = "metadata-empty-concise.xml";
-    static final String METADATA_EXISTING = "metadata-existing.xml";
-    static final String METADATA_NEW_FILE = "metadata-test.xml";
-
-    static final File METADATA_SRC_DIR = new File("src/test/resources/metadata");
-    static final File METADATA_DEST_DIR = new File("target/metadata");
-
-    static final File METADATA_SRC_EMPTY = new File(METADATA_SRC_DIR, METADATA_EMPTY);
-    static final File METADATA_SRC_EMPTY_CONCISE = new File(METADATA_SRC_DIR, METADATA_EMPTY_CONCISE);
-    static final File METADATA_SRC_EXISTING = new File(METADATA_SRC_DIR, METADATA_EXISTING);
-
-    static final File METADATA_DEST_EMPTY = new File(METADATA_DEST_DIR, METADATA_EMPTY);
-    static final File METADATA_DEST_EMPTY_CONCISE = new File(METADATA_DEST_DIR, METADATA_EMPTY_CONCISE);
-    static final File METADATA_DEST_EXISTING = new File(METADATA_DEST_DIR, METADATA_EXISTING);
-    static final File METADATA_DEST_NEW_FILE = new File(METADATA_DEST_DIR, METADATA_NEW_FILE);
-
-    private MetadataProvider metadataProvider;
-
-    @Before
-    public void setup() throws IOException {
-        FileUtils.ensureDirectoryExistAndCanReadAndWrite(METADATA_DEST_DIR);
-        org.apache.commons.io.FileUtils.cleanDirectory(METADATA_DEST_DIR);
-
-        Files.copy(METADATA_SRC_EMPTY.toPath(), METADATA_DEST_EMPTY.toPath(), StandardCopyOption.REPLACE_EXISTING);
-        Files.copy(METADATA_SRC_EMPTY_CONCISE.toPath(), METADATA_DEST_EMPTY_CONCISE.toPath(), StandardCopyOption.REPLACE_EXISTING);
-        Files.copy(METADATA_SRC_EXISTING.toPath(), METADATA_DEST_EXISTING.toPath(), StandardCopyOption.REPLACE_EXISTING);
-
-        metadataProvider = new FileSystemMetadataProvider();
-    }
-
-    private ProviderConfigurationContext createConfigContext(final File metadataFile) {
-        return () -> {
-            final Map<String,String> props = new HashMap<>();
-            props.put(FileSystemMetadataProvider.METADATA_FILE_PROP, metadataFile.getAbsolutePath());
-            return props;
-        };
-    }
-
-    @Test(expected = ProviderCreationException.class)
-    public void testOnConfiguredMissingMetadataFileProperty() {
-        metadataProvider.onConfigured(() -> new HashMap<>());
-    }
-
-    @Test(expected = ProviderCreationException.class)
-    public void testOnConfiguredBlankMetadataFileProperty() {
-        metadataProvider.onConfigured(() -> {
-            final Map<String,String> props = new HashMap<>();
-            props.put(FileSystemMetadataProvider.METADATA_FILE_PROP, "    ");
-            return props;
-        });
-    }
-
-    @Test
-    public void testOnConfiguredWithEmptyMetadata() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EMPTY));
-        assertEquals(0, metadataProvider.getBuckets().size());
-        assertEquals(0, metadataProvider.getFlows().size());
-    }
-
-    @Test
-    public void testOnConfiguredWithEmptyConciseMetadata() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EMPTY_CONCISE));
-        assertEquals(0, metadataProvider.getBuckets().size());
-        assertEquals(0, metadataProvider.getFlows().size());
-    }
-
-    @Test
-    public void testOnConfiguredWithExistingMetadata() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-        assertEquals(1, metadataProvider.getFlows().size());
-
-        final BucketMetadata bucket1 = metadataProvider.getBucketById("bucket1");
-        assertNotNull(bucket1);
-        assertEquals("bucket1", bucket1.getIdentifier());
-
-        final BucketMetadata bucket2 = metadataProvider.getBucketById("bucket2");
-        assertNotNull(bucket2);
-        assertEquals("bucket2", bucket2.getIdentifier());
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow1");
-        assertNotNull(flowMetadata);
-        assertEquals("flow1", flowMetadata.getIdentifier());
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testCreateNullBucket() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_NEW_FILE));
-        metadataProvider.createBucket(null);
-    }
-
-    @Test
-    public void testCreateBucket() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_NEW_FILE));
-
-        final BucketMetadata bucket = new StandardBucketMetadata.Builder()
-                .identifier("bucket1")
-                .name("My Bucket")
-                .description("This is my bucket")
-                .created(System.currentTimeMillis())
-                .build();
-
-        final BucketMetadata returnedBucket = metadataProvider.createBucket(bucket);
-        assertNotNull(returnedBucket);
-        assertEquals(bucket.getIdentifier(), returnedBucket.getIdentifier());
-        assertEquals(bucket.getName(), returnedBucket.getName());
-        assertEquals(bucket.getDescription(), returnedBucket.getDescription());
-        assertEquals(bucket.getCreatedTimestamp(), returnedBucket.getCreatedTimestamp());
-    }
-
-    @Test
-    public void testGetBucketByIdExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket1 = metadataProvider.getBucketById("bucket1");
-        assertNotNull(bucket1);
-        assertEquals("bucket1", bucket1.getIdentifier());
-        assertEquals("Bryan's Bucket", bucket1.getName());
-        assertEquals("The description for Bryan's Bucket.", bucket1.getDescription());
-        assertEquals(111111111, bucket1.getCreatedTimestamp());
-    }
-
-    @Test
-    public void testGetBucketByIdDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket1 = metadataProvider.getBucketById("bucket-does-not-exist");
-        assertNull(bucket1);
-    }
-
-    @Test
-    public void testGetBucketByNameExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket1 = metadataProvider.getBucketByName("Bryan's Bucket");
-        assertNotNull(bucket1);
-        assertEquals("bucket1", bucket1.getIdentifier());
-        assertEquals("Bryan's Bucket", bucket1.getName());
-        assertEquals("The description for Bryan's Bucket.", bucket1.getDescription());
-        assertEquals(111111111, bucket1.getCreatedTimestamp());
-    }
-
-    @Test
-    public void testGetBucketByNameCaseInsensitive() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket1 = metadataProvider.getBucketByName("bryan's bucket");
-        assertNotNull(bucket1);
-        assertEquals("bucket1", bucket1.getIdentifier());
-        assertEquals("Bryan's Bucket", bucket1.getName());
-        assertEquals("The description for Bryan's Bucket.", bucket1.getDescription());
-        assertEquals(111111111, bucket1.getCreatedTimestamp());
-    }
-
-    @Test
-    public void testGetBucketByNameDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket1 = metadataProvider.getBucketByName("bucket-does-not-exist");
-        assertNull(bucket1);
-    }
-
-    @Test
-    public void testUpdateBucketExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket = metadataProvider.getBucketById("bucket1");
-        assertNotNull(bucket);
-
-        final BucketMetadata updatedBucket = new StandardBucketMetadata.Builder(bucket)
-                .name("New Name")
-                .description("New Description")
-                .build();
-
-        final BucketMetadata returnedBucket = metadataProvider.updateBucket(updatedBucket);
-        assertNotNull(returnedBucket);
-        assertEquals("New Name", returnedBucket.getName());
-        assertEquals("New Description", returnedBucket.getDescription());
-    }
-
-    @Test
-    public void testUpdateBucketDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final BucketMetadata bucket = new StandardBucketMetadata.Builder()
-                .identifier("does-not-exist")
-                .name("New Name")
-                .description("New Description")
-                .created(System.currentTimeMillis())
-                .build();
-
-        final BucketMetadata updatedBucket = metadataProvider.updateBucket(bucket);
-        assertNull(updatedBucket);
-    }
-
-    @Test
-    public void testDeleteBucketWithFlows() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final String bucketId = "bucket1";
-        assertNotNull(metadataProvider.getBucketById(bucketId));
-
-        final Set<FlowMetadata> bucketFlows = metadataProvider.getFlows(bucketId);
-        assertNotNull(bucketFlows);
-        assertEquals(1, bucketFlows.size());
-
-        metadataProvider.deleteBucket(bucketId);
-        assertNull(metadataProvider.getBucketById(bucketId));
-
-        final Set<FlowMetadata> bucketFlows2 = metadataProvider.getFlows(bucketId);
-        assertNotNull(bucketFlows2);
-        assertEquals(0, bucketFlows2.size());
-    }
-
-    @Test
-    public void testDeleteBucketDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        final String bucketId = "bucket-does-not-exist";
-        metadataProvider.deleteBucket(bucketId);
-
-        assertEquals(2, metadataProvider.getBuckets().size());
-    }
-
-    @Test
-    public void testCreateFlow() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-        assertEquals(2, metadataProvider.getBuckets().size());
-
-        // verify bucket2 exists and has no flows
-        final BucketMetadata bucket2 = metadataProvider.getBucketById("bucket2");
-        assertNotNull(bucket2);
-        assertEquals(0, metadataProvider.getFlows(bucket2.getIdentifier()).size());
-
-        final FlowMetadata flowMetadata = new StandardFlowMetadata.Builder()
-                .identifier("flow2")
-                .name("New Flow")
-                .description("This is a new flow")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        final FlowMetadata createdFlow = metadataProvider.createFlow(bucket2.getIdentifier(), flowMetadata);
-        assertNotNull(createdFlow);
-        assertEquals(flowMetadata.getIdentifier(), createdFlow.getIdentifier());
-        assertEquals(flowMetadata.getName(), createdFlow.getName());
-        assertEquals(flowMetadata.getDescription(), createdFlow.getDescription());
-        assertEquals(flowMetadata.getCreatedTimestamp(), createdFlow.getCreatedTimestamp());
-        assertEquals(flowMetadata.getModifiedTimestamp(), createdFlow.getModifiedTimestamp());
-        assertEquals(bucket2.getIdentifier(), createdFlow.getBucketIdentifier());
-
-        assertEquals(1, metadataProvider.getFlows(bucket2.getIdentifier()).size());
-    }
-
-    @Test
-    public void testGetFlowByIdExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow1");
-        assertNotNull(flowMetadata);
-        assertEquals("flow1", flowMetadata.getIdentifier());
-        assertEquals("Bryan's Flow", flowMetadata.getName());
-        assertEquals("The description for Bryan's Flow.", flowMetadata.getDescription());
-        assertEquals(333333333, flowMetadata.getCreatedTimestamp());
-        assertEquals(444444444, flowMetadata.getModifiedTimestamp());
-        assertEquals(3, flowMetadata.getSnapshotMetadata().size());
-        assertEquals("bucket1", flowMetadata.getBucketIdentifier());
-    }
-
-    @Test
-    public void testGetFlowByIdDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow-does-not-exist");
-        assertNull(flowMetadata);
-    }
-
-    @Test
-    public void testGetFlowByNameExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowByName("Bryan's Flow");
-        assertNotNull(flowMetadata);
-        assertEquals("flow1", flowMetadata.getIdentifier());
-        assertEquals("Bryan's Flow", flowMetadata.getName());
-        assertEquals("The description for Bryan's Flow.", flowMetadata.getDescription());
-        assertEquals(333333333, flowMetadata.getCreatedTimestamp());
-        assertEquals(444444444, flowMetadata.getModifiedTimestamp());
-        assertEquals(3, flowMetadata.getSnapshotMetadata().size());
-    }
-
-    @Test
-    public void testGetFlowByNameCaseInsensitive() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowByName("bryan's flow");
-        assertNotNull(flowMetadata);
-        assertEquals("flow1", flowMetadata.getIdentifier());
-        assertEquals("Bryan's Flow", flowMetadata.getName());
-        assertEquals("The description for Bryan's Flow.", flowMetadata.getDescription());
-        assertEquals(333333333, flowMetadata.getCreatedTimestamp());
-        assertEquals(444444444, flowMetadata.getModifiedTimestamp());
-        assertEquals(3, flowMetadata.getSnapshotMetadata().size());
-    }
-
-    @Test
-    public void testGetFlowByNameDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowByName("flow-does-not-exist");
-        assertNull(flowMetadata);
-    }
-
-    @Test
-    public void testUpdateFlowExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow1");
-        assertNotNull(flowMetadata);
-
-        final String newFlowName = "New Flow Name";
-        final String newFlowDescription = "New Flow Description";
-        assertNotEquals(flowMetadata.getName(), newFlowName);
-        assertNotEquals(flowMetadata.getDescription(), newFlowDescription);
-
-        final FlowMetadata updatedFlowMetadata = new StandardFlowMetadata.Builder(flowMetadata)
-                .name("New Flow Name")
-                .description("New Flow Description")
-                .build();
-
-        final FlowMetadata returnedFlow = metadataProvider.updateFlow(updatedFlowMetadata);
-        assertEquals(newFlowName, returnedFlow.getName());
-        assertEquals(newFlowDescription, returnedFlow.getDescription());
-        assertEquals(flowMetadata.getCreatedTimestamp(), returnedFlow.getCreatedTimestamp());
-        assertTrue(returnedFlow.getModifiedTimestamp() > flowMetadata.getModifiedTimestamp());
-    }
-
-    @Test
-    public void testUpdateFlowDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = new StandardFlowMetadata.Builder()
-                .identifier("does-not-exist")
-                .name("Does Not Exist")
-                .description("Does Not Exist")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        final FlowMetadata updatedFlow = metadataProvider.updateFlow(flowMetadata);
-        assertNull(updatedFlow);
-    }
-
-    @Test
-    public void testDeleteFlowWithSnapshots() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow1");
-        assertNotNull(flowMetadata);
-        assertNotNull(flowMetadata.getSnapshotMetadata());
-        assertTrue(flowMetadata.getSnapshotMetadata().size() > 0);
-
-        metadataProvider.deleteFlow(flowMetadata.getIdentifier());
-
-        final FlowMetadata deletedFlow = metadataProvider.getFlowById("flow1");
-        assertNull(deletedFlow);
-    }
-
-    @Test
-    public void testDeleteFlowDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        assertEquals(1, metadataProvider.getFlows().size());
-        metadataProvider.deleteFlow("does-not-exist");
-        assertEquals(1, metadataProvider.getFlows().size());
-    }
-
-    @Test
-    public void testCreateFlowSnapshot() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata versionedFlow = metadataProvider.getFlowById("flow1");
-        assertNotNull(versionedFlow);
-        assertNotNull(versionedFlow.getSnapshotMetadata());
-
-        int lastVersion = 1;
-        for (final FlowSnapshotMetadata snapshot : versionedFlow.getSnapshotMetadata()) {
-            if (snapshot.getVersion() > lastVersion) {
-                lastVersion = snapshot.getVersion();
-            }
-        }
-
-        final FlowSnapshotMetadata nextSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .flowIdentifier(versionedFlow.getIdentifier())
-                .flowName(versionedFlow.getName())
-                .version(lastVersion + 1)
-                .comments("This is the next snapshot")
-                .created(System.currentTimeMillis())
-                .build();
-
-        final FlowSnapshotMetadata createdSnapshot = metadataProvider.createFlowSnapshot(nextSnapshot);
-        assertEquals(nextSnapshot.getFlowIdentifier(), createdSnapshot.getFlowIdentifier());
-        assertEquals(nextSnapshot.getFlowName(), createdSnapshot.getFlowName());
-        assertEquals(nextSnapshot.getVersion(), createdSnapshot.getVersion());
-        assertEquals(nextSnapshot.getComments(), createdSnapshot.getComments());
-        assertEquals(nextSnapshot.getCreatedTimestamp(), createdSnapshot.getCreatedTimestamp());
-
-        final FlowMetadata updatedFlow = metadataProvider.getFlowById("flow1");
-        assertNotNull(updatedFlow);
-        assertNotNull(updatedFlow.getSnapshotMetadata());
-        assertEquals(updatedFlow.getSnapshotMetadata().size(), versionedFlow.getSnapshotMetadata().size() + 1);
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void testCreateFlowSnapshotWhenFlowDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowSnapshotMetadata nextSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .flowIdentifier("does-not-exist")
-                .flowName("New Snapshot")
-                .version(1)
-                .comments("This is the next snapshot")
-                .created(System.currentTimeMillis())
-                .build();
-
-        metadataProvider.createFlowSnapshot(nextSnapshot);
-    }
-
-    @Test
-    public void testGetFlowSnapshotExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowSnapshotMetadata snapshot = metadataProvider.getFlowSnapshot("flow1", 1);
-        assertNotNull(snapshot);
-        assertEquals("flow1", snapshot.getFlowIdentifier());
-        assertEquals("Bryan's Flow", snapshot.getFlowName());
-        assertEquals(1, snapshot.getVersion());
-        assertEquals(555555555, snapshot.getCreatedTimestamp());
-        assertEquals("These are the comments for snapshot #1.", snapshot.getComments());
-        assertEquals("bucket1", snapshot.getBucketIdentifier());
-    }
-
-    @Test
-    public void testGetFlowSnapshotNameDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowSnapshotMetadata snapshot = metadataProvider.getFlowSnapshot("does-not-exist", 1);
-        assertNull(snapshot);
-    }
-
-    @Test
-    public void testGetFlowSnapshotVersionDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowSnapshotMetadata snapshot = metadataProvider.getFlowSnapshot("flow1", Integer.MAX_VALUE);
-        assertNull(snapshot);
-    }
-
-    @Test
-    public void testDeleteSnapshotExists() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow1");
-        assertNotNull(flowMetadata);
-        assertNotNull(flowMetadata.getSnapshotMetadata());
-        assertEquals(3, flowMetadata.getSnapshotMetadata().size());
-
-        final FlowSnapshotMetadata firstSnapshot = flowMetadata.getSnapshotMetadata().stream().findFirst().orElse(null);
-        assertNotNull(firstSnapshot);
-
-        metadataProvider.deleteFlowSnapshot(flowMetadata.getIdentifier(), firstSnapshot.getVersion());
-
-        final FlowMetadata updatedFlow = metadataProvider.getFlowById("flow1");
-        assertNotNull(updatedFlow);
-        assertNotNull(updatedFlow.getSnapshotMetadata());
-        assertEquals(2, updatedFlow.getSnapshotMetadata().size());
-
-        final FlowSnapshotMetadata deletedSnapshot = updatedFlow.getSnapshotMetadata().stream()
-                .filter(s -> s.getVersion() == firstSnapshot.getVersion()).findFirst().orElse(null);
-        assertNull(deletedSnapshot);
-    }
-
-    @Test
-    public void testDeleteSnapshotDoesNotExist() {
-        metadataProvider.onConfigured(createConfigContext(METADATA_DEST_EXISTING));
-
-        final FlowMetadata flowMetadata = metadataProvider.getFlowById("flow1");
-        assertNotNull(flowMetadata);
-        assertNotNull(flowMetadata.getSnapshotMetadata());
-        assertEquals(3, flowMetadata.getSnapshotMetadata().size());
-
-        metadataProvider.deleteFlowSnapshot(flowMetadata.getIdentifier(), Integer.MAX_VALUE);
-
-        final FlowMetadata updatedFlow = metadataProvider.getFlowById("flow1");
-        assertNotNull(updatedFlow);
-        assertNotNull(updatedFlow.getSnapshotMetadata());
-        assertEquals(3, updatedFlow.getSnapshotMetadata().size());
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty-concise.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty-concise.xml b/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty-concise.xml
deleted file mode 100644
index 8f4f4f9..0000000
--- a/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty-concise.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<metadata />

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty.xml b/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty.xml
deleted file mode 100644
index 5af6f87..0000000
--- a/nifi-registry-provider-impl/src/test/resources/metadata/metadata-empty.xml
+++ /dev/null
@@ -1,17 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<metadata>
-</metadata>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/test/resources/metadata/metadata-existing.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/test/resources/metadata/metadata-existing.xml b/nifi-registry-provider-impl/src/test/resources/metadata/metadata-existing.xml
deleted file mode 100644
index 29db524..0000000
--- a/nifi-registry-provider-impl/src/test/resources/metadata/metadata-existing.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<metadata>
-    <buckets>
-        <bucket identifier="bucket1" name="Bryan's Bucket" createdTimestamp="111111111">
-            <description>The description for Bryan's Bucket.</description>
-        </bucket>
-        <bucket identifier="bucket2" name="Matt's Bucket" createdTimestamp="222222222" />
-    </buckets>
-    <flows>
-        <flow identifier="flow1" name="Bryan's Flow" createdTimestamp="333333333" modifiedTimestamp="444444444" bucketIdentifier="bucket1">
-            <description>The description for Bryan's Flow.</description>
-            <snapshot version="1" createdTimestamp="555555555">
-                <comments>These are the comments for snapshot #1.</comments>
-            </snapshot>
-            <snapshot version="2" createdTimestamp="666666666" />
-            <snapshot version="3" createdTimestamp="777777777">
-                <comments>These are the comments for snapshot #3.</comments>
-            </snapshot>
-        </flow>
-    </flows>
-</metadata>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-resources/src/main/resources/conf/logback.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-resources/src/main/resources/conf/logback.xml b/nifi-registry-resources/src/main/resources/conf/logback.xml
index 137ed83..72e863c 100644
--- a/nifi-registry-resources/src/main/resources/conf/logback.xml
+++ b/nifi-registry-resources/src/main/resources/conf/logback.xml
@@ -68,6 +68,11 @@
     
     <logger name="org.apache.nifi.registry" level="INFO"/>
 
+    <!-- To see SQL statements set this to DEBUG -->
+    <logger name="org.hibernate.SQL" level="INFO" />
+    <!-- To see the values in SQL statements set this to TRACE -->
+    <logger name="org.hibernate.type" level="INFO" />
+
     <!--
         Logger for capturing Bootstrap logs and NiFi Registry's standard error and standard out.
     -->

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/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 a748f2f..db62768 100644
--- a/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
+++ b/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
@@ -34,4 +34,12 @@ nifi.registry.security.needClientAuth=${nifi.registry.security.needClientAuth}
 nifi.registry.security.authorized.users=${nifi.registry.security.authorized.users}
 
 # providers properties #
-nifi.registry.providers.configuration.file=${nifi.registry.providers.configuration.file}
\ No newline at end of file
+nifi.registry.providers.configuration.file=${nifi.registry.providers.configuration.file}
+
+# database properties
+nifi.registry.db.directory=${nifi.registry.db.directory}
+nifi.registry.db.url.append=${nifi.registry.db.url.append}
+
+#spring.h2.console.enabled=true
+#spring.jpa.hibernate.ddl-auto=validate
+#spring.jpa.show-sql=true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-resources/src/main/resources/conf/providers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-resources/src/main/resources/conf/providers.xml b/nifi-registry-resources/src/main/resources/conf/providers.xml
index 450eecb..40bf012 100644
--- a/nifi-registry-resources/src/main/resources/conf/providers.xml
+++ b/nifi-registry-resources/src/main/resources/conf/providers.xml
@@ -15,13 +15,8 @@
 -->
 <providers>
 
-    <metadataProvider>
-        <class>org.apache.nifi.registry.metadata.FileSystemMetadataProvider</class>
-        <property name="Metadata File">./metadata.xml</property>
-    </metadataProvider>
-
     <flowPersistenceProvider>
-        <class>org.apache.nifi.registry.flow.FileSystemFlowPersistenceProvider</class>
+        <class>org.apache.nifi.registry.provider.flow.FileSystemFlowPersistenceProvider</class>
         <property name="Flow Storage Directory">./flow_storage</property>
     </flowPersistenceProvider>
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/pom.xml b/nifi-registry-web-api/pom.xml
index 7253e0c..603cb68 100644
--- a/nifi-registry-web-api/pom.xml
+++ b/nifi-registry-web-api/pom.xml
@@ -122,19 +122,19 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-properties</artifactId>
+            <artifactId>nifi-registry-framework</artifactId>
             <version>0.0.1-SNAPSHOT</version>
-            <scope>provided</scope>
         </dependency>
+        <!-- This will directly in lib so mark it as provided -->
         <dependency>
             <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-framework</artifactId>
+            <artifactId>nifi-registry-properties</artifactId>
             <version>0.0.1-SNAPSHOT</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-provider-impl</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
         </dependency>
         <dependency>
             <groupId>javax.servlet</groupId>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
new file mode 100644
index 0000000..e8e36a7
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
@@ -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.
+ */
+package org.apache.nifi.registry;
+
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.boot.web.support.SpringBootServletInitializer;
+
+import java.util.Properties;
+
+/**
+ * Main class for starting the NiFi Registry Web API as a Spring Boot application.
+ *
+ * This class is purposely in the org.apache.nifi.registry package since that is the common base
+ * package across other modules. This is done because spring-boot will use the package of this
+ * class to automatically scan for beans/config/entities/etc. and would otherwise require
+ * configuring custom packages to scan in several different places.
+ */
+@SpringBootApplication
+public class NiFiRegistryApiApplication extends SpringBootServletInitializer {
+
+    public static final String NIFI_REGISTRY_PROPERTIES_ATTRIBUTE = "nifi-registry.properties";
+
+    @Override
+    protected SpringApplicationBuilder configure(SpringApplicationBuilder application) {
+        final Properties fixedProps = new Properties();
+        fixedProps.setProperty("spring.jpa.hibernate.ddl-auto", "none");
+        fixedProps.setProperty("spring.jpa.hibernate.naming.physical-strategy", "org.hibernate.boot.model.naming.PhysicalNamingStrategyStandardImpl");
+
+        return application
+                .sources(NiFiRegistryApiApplication.class)
+                .properties(fixedProps);
+    }
+
+    public static void main(String[] args) {
+        SpringApplication.run(NiFiRegistryApiApplication.class, args);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryPropertiesFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryPropertiesFactory.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryPropertiesFactory.java
new file mode 100644
index 0000000..b7865bb
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryPropertiesFactory.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;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.web.context.ServletContextAware;
+
+import javax.servlet.ServletContext;
+
+/**
+ * The JettyServer puts an instance of NiFiRegistryProperties into the ServletContext, this class
+ * obtains that instance and makes it available to inject to all other places.
+ *
+ */
+@Configuration
+public class NiFiRegistryPropertiesFactory implements ServletContextAware {
+
+    private NiFiRegistryProperties properties;
+
+    @Override
+    public void setServletContext(ServletContext servletContext) {
+        properties = (NiFiRegistryProperties) servletContext.getAttribute(
+                NiFiRegistryApiApplication.NIFI_REGISTRY_PROPERTIES_ATTRIBUTE);
+    }
+
+    @Bean
+    public NiFiRegistryProperties getNiFiRegistryProperties() {
+        return properties;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryApiApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryApiApplication.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryApiApplication.java
deleted file mode 100644
index 9d2f5a4..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryApiApplication.java
+++ /dev/null
@@ -1,44 +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.web;
-
-import org.springframework.boot.SpringApplication;
-import org.springframework.boot.autoconfigure.SpringBootApplication;
-import org.springframework.boot.builder.SpringApplicationBuilder;
-import org.springframework.boot.web.support.SpringBootServletInitializer;
-import org.springframework.context.annotation.ComponentScan;
-
-/**
- * Main class for starting the NiFi Registry Web API as a Spring Boot application.
- *
- * By default, Spring Boot will only scan in the package this class is located in, so we set
- * @ComponentScan to the common parent package to find beans in other packages.
- */
-@SpringBootApplication
-@ComponentScan("org.apache.nifi.registry")
-public class NiFiRegistryApiApplication extends SpringBootServletInitializer {
-
-    @Override
-    protected SpringApplicationBuilder configure(SpringApplicationBuilder application) {
-        return application.sources(NiFiRegistryApiApplication.class);
-    }
-
-    public static void main(String[] args) {
-        SpringApplication.run(NiFiRegistryApiApplication.class, args);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryPropertiesFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryPropertiesFactory.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryPropertiesFactory.java
deleted file mode 100644
index 85e9784..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryPropertiesFactory.java
+++ /dev/null
@@ -1,46 +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.web;
-
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.springframework.context.annotation.Bean;
-import org.springframework.context.annotation.Configuration;
-import org.springframework.web.context.ServletContextAware;
-
-import javax.servlet.ServletContext;
-
-/**
- * The JettyServer puts an instance of NiFiRegistryProperties into the ServletContext, so this class
- * obtains that instance and makes it available to inject into all other places.
- *
- */
-@Configuration
-public class NiFiRegistryPropertiesFactory implements ServletContextAware {
-
-    private NiFiRegistryProperties properties;
-
-    @Override
-    public void setServletContext(ServletContext servletContext) {
-        properties = (NiFiRegistryProperties) servletContext.getAttribute("nifi-registry.properties");
-    }
-
-    @Bean
-    public NiFiRegistryProperties getNiFiRegistryProperties() {
-        return properties;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
index 7a6ac13..969f849 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
@@ -19,7 +19,7 @@ package org.apache.nifi.registry.web;
 import org.apache.nifi.registry.web.api.BucketFlowResource;
 import org.apache.nifi.registry.web.api.BucketResource;
 import org.apache.nifi.registry.web.api.FlowResource;
-import org.apache.nifi.registry.web.api.TestResource;
+import org.apache.nifi.registry.web.api.ItemResource;
 import org.apache.nifi.registry.web.mapper.IllegalArgumentExceptionMapper;
 import org.apache.nifi.registry.web.mapper.IllegalStateExceptionMapper;
 import org.apache.nifi.registry.web.mapper.ResourceNotFoundExceptionMapper;
@@ -58,9 +58,7 @@ public class NiFiRegistryResourceConfig extends ResourceConfig {
         register(BucketResource.class);
         register(BucketFlowResource.class);
         register(FlowResource.class);
-
-        // test endpoint to exercise spring dependency injection
-        register(TestResource.class);
+        register(ItemResource.class);
 
         property(ServerProperties.BV_SEND_ERROR_IN_RESPONSE, true);
     }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
index b5f2d93..1272522 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
@@ -23,6 +23,10 @@ import io.swagger.annotations.ApiResponses;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.bucket.Bucket;
 import org.apache.nifi.registry.service.RegistryService;
+import org.apache.nifi.registry.service.params.QueryParameters;
+import org.apache.nifi.registry.service.params.SortParameter;
+import org.apache.nifi.registry.web.link.LinkService;
+import org.apache.nifi.registry.web.response.FieldsEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -30,16 +34,19 @@ import org.springframework.stereotype.Component;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriInfo;
+import java.util.List;
 import java.util.Set;
 
 @Component
@@ -56,10 +63,14 @@ public class BucketResource {
     @Context
     UriInfo uriInfo;
 
+    private final LinkService linkService;
+
     private final RegistryService registryService;
 
-    public BucketResource(@Autowired final RegistryService registryService) {
+    @Autowired
+    public BucketResource(final RegistryService registryService, final LinkService linkService) {
         this.registryService = registryService;
+        this.linkService = linkService;
     }
 
     @POST
@@ -78,12 +89,20 @@ public class BucketResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Get metadata for all buckets in the registry for which the client is authorized. TODO: Will add some search parameters as well.",
+            value = "Get metadata for all buckets in the registry for which the client is authorized. Information about the items stored in each " +
+                    "bucket should be obtained by requesting and individual bucket by id.",
             response = Bucket.class,
             responseContainer = "List"
     )
-    public Response getBuckets() {
-        final Set<Bucket> buckets = registryService.getBuckets();
+    public Response getBuckets(@QueryParam("sort") final List<SortParameter> sortParameters) {
+
+        final QueryParameters params = new QueryParameters.Builder()
+                .addSorts(sortParameters)
+                .build();
+
+        final List<Bucket> buckets = registryService.getBuckets(params);
+        linkService.populateBucketLinks(buckets);
+
         return Response.status(Response.Status.OK).entity(buckets).build();
     }
 
@@ -92,7 +111,8 @@ public class BucketResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Get metadata for an existing bucket in the registry.",
+            value = "Get metadata for an existing bucket in the registry. If verbose is set to true, then each bucket will be returned " +
+                    "with the set of items in the bucket, but any further children of those items will not be included.",
             response = Bucket.class
     )
     @ApiResponses(
@@ -100,8 +120,15 @@ public class BucketResource {
                     @ApiResponse(code = 404, message = "The specified resource could not be found."),
             }
     )
-    public Response getBucket(@PathParam("bucketId") final String bucketId) {
-        final Bucket bucket = registryService.getBucket(bucketId);
+    public Response getBucket(@PathParam("bucketId") final String bucketId,
+                              @QueryParam("verbose") @DefaultValue("false") boolean verbose) {
+        final Bucket bucket = registryService.getBucket(bucketId, verbose);
+        linkService.populateBucketLinks(bucket);
+
+        if (bucket.getVersionedFlows() != null) {
+            linkService.populateFlowLinks(bucket.getVersionedFlows());
+        }
+
         return Response.status(Response.Status.OK).entity(bucket).build();
     }
 
@@ -157,4 +184,18 @@ public class BucketResource {
         return Response.status(Response.Status.OK).entity(deletedBucket).build();
     }
 
+    @GET
+    @Path("fields")
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @ApiOperation(
+            value = "Retrieves the available field names that can be used for searching or sorting on buckets.",
+            response = FieldsEntity.class
+    )
+    public Response getAvailableBucketFields() {
+        final Set<String> bucketFields = registryService.getBucketFields();
+        final FieldsEntity fieldsEntity = new FieldsEntity(bucketFields);
+        return Response.status(Response.Status.OK).entity(fieldsEntity).build();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
index 6298aa5..abd0d69 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
@@ -25,6 +25,10 @@ import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.service.RegistryService;
+import org.apache.nifi.registry.service.params.QueryParameters;
+import org.apache.nifi.registry.service.params.SortParameter;
+import org.apache.nifi.registry.web.link.LinkService;
+import org.apache.nifi.registry.web.response.FieldsEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -32,14 +36,19 @@ import org.springframework.stereotype.Component;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 
@@ -53,22 +62,37 @@ public class FlowResource {
 
     private static final Logger logger = LoggerFactory.getLogger(FlowResource.class);
 
+    @Context
+    UriInfo uriInfo;
+
+    private final LinkService linkService;
+
     private final RegistryService registryService;
 
-    public FlowResource(@Autowired final RegistryService registryService) {
+    @Autowired
+    public FlowResource(final RegistryService registryService, final LinkService linkService) {
         this.registryService = registryService;
+        this.linkService = linkService;
     }
 
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Get metadata for all flows in all buckets that the registry has stored for which the client is authorized. TODO: Will add some search parameters as well.",
+            value = "Get metadata for all flows in all buckets that the registry has stored for which the client is authorized. The information about " +
+                    "the versions of each flow should be obtained by requesting a specific flow by id.",
             response = VersionedFlow.class,
             responseContainer = "List"
     )
-    public Response getFlows() {
-        final Set<VersionedFlow> flows = registryService.getFlows();
+    public Response getFlows(@QueryParam("sort") final List<SortParameter> sortParameters) {
+
+        final QueryParameters params = new QueryParameters.Builder()
+                .addSorts(sortParameters)
+                .build();
+
+        final List<VersionedFlow> flows = registryService.getFlows(params);
+        linkService.populateFlowLinks(flows);
+
         return Response.status(Response.Status.OK).entity(flows).build();
     }
 
@@ -77,7 +101,8 @@ public class FlowResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Get metadata for an existing flow the registry has stored.",
+            value = "Get metadata for an existing flow the registry has stored. If verbose is true, then the metadata " +
+                    "about all snapshots for the flow will also be returned.",
             response = VersionedFlow.class
     )
     @ApiResponses(
@@ -85,8 +110,16 @@ public class FlowResource {
                     @ApiResponse(code = 404, message = "The specified resource could not be found."),
             }
     )
-    public Response getFlow(@PathParam("flowId") final String flowId) {
-        final VersionedFlow flow = registryService.getFlow(flowId);
+    public Response getFlow(@PathParam("flowId") final String flowId,
+                            @QueryParam("verbose") @DefaultValue("false") boolean verbose) {
+
+        final VersionedFlow flow = registryService.getFlow(flowId, verbose);
+        linkService.populateFlowLinks(flow);
+
+        if (flow.getSnapshotMetadata() != null) {
+            linkService.populateSnapshotLinks(flow.getSnapshotMetadata());
+        }
+
         return Response.status(Response.Status.OK).entity(flow).build();
     }
 
@@ -188,7 +221,12 @@ public class FlowResource {
             }
     )
     public Response getFlowVersions(@PathParam("flowId") final String flowId) {
-        final VersionedFlow flow = registryService.getFlow(flowId);
+        final VersionedFlow flow = registryService.getFlow(flowId, true);
+
+        if (flow.getSnapshotMetadata() != null) {
+            linkService.populateSnapshotLinks(flow.getSnapshotMetadata());
+        }
+
         return Response.status(Response.Status.OK).entity(flow.getSnapshotMetadata()).build();
     }
 
@@ -206,7 +244,7 @@ public class FlowResource {
             }
     )
     public Response getLatestFlowVersion(@PathParam("flowId") final String flowId) {
-        final VersionedFlow flow = registryService.getFlow(flowId);
+        final VersionedFlow flow = registryService.getFlow(flowId, true);
 
         final SortedSet<VersionedFlowSnapshotMetadata> snapshots = flow.getSnapshotMetadata();
         if (snapshots == null || snapshots.size() == 0) {
@@ -241,4 +279,18 @@ public class FlowResource {
         return Response.status(Response.Status.OK).entity(snapshot).build();
     }
 
+    @GET
+    @Path("fields")
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @ApiOperation(
+            value = "Retrieves the available field names that can be used for searching or sorting on flows.",
+            response = FieldsEntity.class
+    )
+    public Response getAvailableFlowFields() {
+        final Set<String> flowFields = registryService.getFlowFields();
+        final FieldsEntity fieldsEntity = new FieldsEntity(flowFields);
+        return Response.status(Response.Status.OK).entity(fieldsEntity).build();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
new file mode 100644
index 0000000..1867c83
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
@@ -0,0 +1,125 @@
+/*
+ * 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.web.api;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import org.apache.nifi.registry.bucket.BucketItem;
+import org.apache.nifi.registry.service.RegistryService;
+import org.apache.nifi.registry.service.params.QueryParameters;
+import org.apache.nifi.registry.service.params.SortParameter;
+import org.apache.nifi.registry.web.link.LinkService;
+import org.apache.nifi.registry.web.response.FieldsEntity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.util.List;
+import java.util.Set;
+
+@Component
+@Path("/items")
+@Api(
+        value = "/items",
+        description = "Retrieve items across all buckets for which the user is authorized."
+)
+public class ItemResource {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ItemResource.class);
+
+    @Context
+    UriInfo uriInfo;
+
+    private final LinkService linkService;
+
+    private final RegistryService registryService;
+
+    @Autowired
+    public ItemResource(final RegistryService registryService, final LinkService linkService) {
+        this.registryService = registryService;
+        this.linkService = linkService;
+    }
+
+
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @ApiOperation(
+            value = "Get metadata for items across all buckets for which the user is authorized.",
+            response = BucketItem.class,
+            responseContainer = "List"
+    )
+    public Response getItems(@QueryParam("sort") final List<SortParameter> sortParameters) {
+
+        final QueryParameters params = new QueryParameters.Builder()
+                .addSorts(sortParameters)
+                .build();
+
+        final List<BucketItem> items = registryService.getBucketItems(params);
+        linkService.populateItemLinks(items);
+
+        return Response.status(Response.Status.OK).entity(items).build();
+    }
+
+    @GET
+    @Path("{bucketId}")
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @ApiOperation(
+            value = "Get metadata for items of the given bucket.",
+            response = BucketItem.class,
+            responseContainer = "List"
+    )
+    public Response getItems(@PathParam("bucketId") final String bucketId,
+                             @QueryParam("sort") final List<SortParameter> sortParameters) {
+
+        final QueryParameters params = new QueryParameters.Builder()
+                .addSorts(sortParameters)
+                .build();
+
+        final List<BucketItem> items = registryService.getBucketItems(params, bucketId);
+        linkService.populateItemLinks(items);
+
+        return Response.status(Response.Status.OK).entity(items).build();
+    }
+
+    @GET
+    @Path("fields")
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @ApiOperation(
+            value = "Retrieves the available field names that can be used for searching or sorting on bucket items.",
+            response = FieldsEntity.class
+    )
+    public Response getAvailableBucketFields() {
+        final Set<String> bucketFields = registryService.getBucketItemFields();
+        final FieldsEntity fieldsEntity = new FieldsEntity(bucketFields);
+        return Response.status(Response.Status.OK).entity(fieldsEntity).build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TestResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TestResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TestResource.java
deleted file mode 100644
index 1f6e1ba..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TestResource.java
+++ /dev/null
@@ -1,55 +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.web.api;
-
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.service.TestService;
-import org.apache.nifi.registry.web.response.TestEntity;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-import javax.ws.rs.GET;
-import javax.ws.rs.Path;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-
-@Component
-@Path("/test")
-public class TestResource {
-
-    private TestService testService;
-
-    private NiFiRegistryProperties properties;
-
-    public TestResource(@Autowired TestService testService, @Autowired NiFiRegistryProperties properties) {
-        this.testService = testService;
-        this.properties = properties;
-
-        if (this.properties == null) {
-            throw new IllegalStateException("Properties cannot be null");
-        }
-    }
-
-    @GET
-    @Produces(MediaType.APPLICATION_JSON)
-    public Response getTest() {
-        final TestEntity testEntity = new TestEntity(testService.test());
-        return Response.ok(testEntity).build();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/LinkService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/LinkService.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/LinkService.java
new file mode 100644
index 0000000..19e2168
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/LinkService.java
@@ -0,0 +1,110 @@
+/*
+ * 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.web.link;
+
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.bucket.BucketItem;
+import org.apache.nifi.registry.flow.VersionedFlow;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+import org.apache.nifi.registry.web.link.builder.BucketLinkBuilder;
+import org.apache.nifi.registry.web.link.builder.LinkBuilder;
+import org.apache.nifi.registry.web.link.builder.VersionedFlowLinkBuilder;
+import org.apache.nifi.registry.web.link.builder.VersionedFlowSnapshotLinkBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Service;
+
+import javax.ws.rs.core.Link;
+
+@Service
+public class LinkService {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(LinkService.class);
+
+    private final LinkBuilder<Bucket> bucketLinkBuilder = new BucketLinkBuilder();
+
+    private final LinkBuilder<VersionedFlow> versionedFlowLinkBuilder = new VersionedFlowLinkBuilder();
+
+    private final LinkBuilder<VersionedFlowSnapshotMetadata> snapshotMetadataLinkBuilder = new VersionedFlowSnapshotLinkBuilder();
+
+    // ---- Bucket Links
+
+    public void populateBucketLinks(final Iterable<Bucket> buckets) {
+        if (buckets == null) {
+            return;
+        }
+
+        buckets.forEach(b -> populateBucketLinks(b));
+    }
+
+    public void populateBucketLinks(final Bucket bucket) {
+        final Link bucketLink = bucketLinkBuilder.createLink(bucket);
+        bucket.setLink(bucketLink);
+    }
+
+    // ---- Flow Links
+
+    public void populateFlowLinks(final Iterable<VersionedFlow> versionedFlows) {
+        if (versionedFlows == null) {
+            return;
+        }
+
+        versionedFlows.forEach(f  -> populateFlowLinks(f));
+    }
+
+    public void populateFlowLinks(final VersionedFlow versionedFlow) {
+        final Link flowLink = versionedFlowLinkBuilder.createLink(versionedFlow);
+        versionedFlow.setLink(flowLink);
+    }
+
+    // ---- Flow Snapshot Links
+
+    public void populateSnapshotLinks(final Iterable<VersionedFlowSnapshotMetadata> snapshotMetadatas) {
+        if (snapshotMetadatas == null) {
+            return;
+        }
+
+        snapshotMetadatas.forEach(s -> populateSnapshotLinks(s));
+    }
+
+    public void populateSnapshotLinks(final VersionedFlowSnapshotMetadata snapshotMetadata) {
+        final Link snapshotLink = snapshotMetadataLinkBuilder.createLink(snapshotMetadata);
+        snapshotMetadata.setLink(snapshotLink);
+    }
+
+    // ---- BucketItem Links
+
+    public void populateItemLinks(final Iterable<BucketItem> items) {
+        if (items == null) {
+            return;
+        }
+
+        items.forEach(i -> populateItemLinks(i));
+    }
+
+    public void populateItemLinks(final BucketItem bucketItem) {
+        if (bucketItem == null) {
+            return;
+        }
+
+        if (bucketItem instanceof VersionedFlow) {
+            populateFlowLinks((VersionedFlow)bucketItem);
+        } else {
+            LOGGER.error("Unable to create link for BucketItem with type: " + bucketItem.getClass().getCanonicalName());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/BucketLinkBuilder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/BucketLinkBuilder.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/BucketLinkBuilder.java
new file mode 100644
index 0000000..f0409c7
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/BucketLinkBuilder.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.link.builder;
+
+import org.apache.nifi.registry.bucket.Bucket;
+
+import javax.ws.rs.core.Link;
+import javax.ws.rs.core.UriBuilder;
+import java.net.URI;
+
+/**
+ * LinkBuilder that builds "self" links for Buckets.
+ */
+public class BucketLinkBuilder implements LinkBuilder<Bucket> {
+
+    private static final String PATH = "buckets/{id}";
+
+    @Override
+    public Link createLink(final Bucket bucket) {
+        if (bucket == null) {
+            return null;
+        }
+
+        final URI uri = UriBuilder.fromPath(PATH)
+                .resolveTemplate("id", bucket.getIdentifier())
+                .build();
+
+        return Link.fromUri(uri).rel("self").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/LinkBuilder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/LinkBuilder.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/LinkBuilder.java
new file mode 100644
index 0000000..ec356fd
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/LinkBuilder.java
@@ -0,0 +1,30 @@
+/*
+ * 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.web.link.builder;
+
+import javax.ws.rs.core.Link;
+
+/**
+ * Creates a Link for a given type.
+ *
+ * @param <T> the type to create a link for
+ */
+public interface LinkBuilder<T> {
+
+    Link createLink(T t);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowLinkBuilder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowLinkBuilder.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowLinkBuilder.java
new file mode 100644
index 0000000..e18d583
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowLinkBuilder.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.link.builder;
+
+import org.apache.nifi.registry.flow.VersionedFlow;
+
+import javax.ws.rs.core.Link;
+import javax.ws.rs.core.UriBuilder;
+import java.net.URI;
+
+/**
+ * LinkBuilder that builds "self" links for VersionedFlows.
+ */
+public class VersionedFlowLinkBuilder implements LinkBuilder<VersionedFlow> {
+
+    private static final String PATH = "flows/{id}";
+
+    @Override
+    public Link createLink(final VersionedFlow versionedFlow) {
+        if (versionedFlow == null) {
+            return null;
+        }
+
+        final URI uri = UriBuilder.fromPath(PATH)
+                .resolveTemplate("id", versionedFlow.getIdentifier())
+                .build();
+
+        return Link.fromUri(uri).rel("self").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowSnapshotLinkBuilder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowSnapshotLinkBuilder.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowSnapshotLinkBuilder.java
new file mode 100644
index 0000000..47eb15f
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/link/builder/VersionedFlowSnapshotLinkBuilder.java
@@ -0,0 +1,46 @@
+/*
+ * 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.web.link.builder;
+
+import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+
+import javax.ws.rs.core.Link;
+import javax.ws.rs.core.UriBuilder;
+import java.net.URI;
+
+/**
+ * LinkBuilder that builds "self" links for VersionedFlowSnapshotMetadata.
+ */
+public class VersionedFlowSnapshotLinkBuilder implements LinkBuilder<VersionedFlowSnapshotMetadata> {
+
+    private static final String PATH = "flows/{flowId}/versions/{versionNumber}";
+
+    @Override
+    public Link createLink(final VersionedFlowSnapshotMetadata snapshotMetadata) {
+        if (snapshotMetadata == null) {
+            return null;
+        }
+
+        final URI uri = UriBuilder.fromPath(PATH)
+                .resolveTemplate("flowId", snapshotMetadata.getFlowIdentifier())
+                .resolveTemplate("versionNumber", snapshotMetadata.getVersion())
+                .build();
+
+        return Link.fromUri(uri).rel("content").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/FieldsEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/FieldsEntity.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/FieldsEntity.java
new file mode 100644
index 0000000..3dec2b0
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/FieldsEntity.java
@@ -0,0 +1,37 @@
+/*
+ * 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.web.response;
+
+import java.util.Set;
+
+public class FieldsEntity {
+
+    private Set<String> fields;
+
+    public FieldsEntity(Set<String> fields) {
+        this.fields = fields;
+    }
+
+    public Set<String> getFields() {
+        return fields;
+    }
+
+    public void setFields(Set<String> fields) {
+        this.fields = fields;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/TestEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/TestEntity.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/TestEntity.java
deleted file mode 100644
index d72899d..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/response/TestEntity.java
+++ /dev/null
@@ -1,31 +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.web.response;
-
-public class TestEntity {
-
-    private final  String message;
-
-    public TestEntity(String message) {
-        this.message = message;
-    }
-
-    public String getMessage() {
-        return message;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/TestRestAPI.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/TestRestAPI.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/TestRestAPI.java
index fc980f9..485c10a 100644
--- a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/TestRestAPI.java
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/TestRestAPI.java
@@ -30,6 +30,8 @@ import javax.ws.rs.client.ClientBuilder;
 import javax.ws.rs.client.Entity;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.util.ArrayList;
+import java.util.List;
 
 public class TestRestAPI {
 
@@ -43,37 +45,29 @@ public class TestRestAPI {
         try {
             final Client client = ClientBuilder.newClient();
 
-            // Create a bucket
+            // create some buckets
+            final int numBuckets = 20;
+            final List<Bucket> createdBuckets = new ArrayList<>();
 
-            final Bucket bucket = new Bucket();
-            bucket.setName("First Bucket");
-            bucket.setDescription("This is the first bucket.");
+            for (int i=0; i < numBuckets; i++) {
+                final Bucket createdBucket = createBucket(client, i);
+                System.out.println("Created bucket # " + i + " with id " + createdBucket.getIdentifier());
+                createdBuckets.add(createdBucket);
+            }
 
-            final Bucket createdBucket = client.target(REGISTRY_API_BUCKETS_URL)
-                    .request()
-                    .post(
-                            Entity.entity(bucket, MediaType.APPLICATION_JSON),
-                            Bucket.class
-                    );
-
-            System.out.println("Created bucket with id: " + createdBucket.getName());
+            final Bucket createdBucket = createdBuckets.get(0);
 
-            // Create a flow
+            // create some flows
+            final int numFlows = 20;
+            final List<VersionedFlow> createdFlows = new ArrayList<>();
 
-            final VersionedFlow versionedFlow = new VersionedFlow();
-            versionedFlow.setName("First Flow");
-            versionedFlow.setDescription("This is the first flow.");
-
-            final VersionedFlow createdFlow = client.target(REGISTRY_API_BUCKETS_URL)
-                    .path("/{bucketId}/flows")
-                    .resolveTemplate("bucketId", createdBucket.getIdentifier())
-                    .request()
-                    .post(
-                            Entity.entity(versionedFlow, MediaType.APPLICATION_JSON),
-                            VersionedFlow.class
-                    );
+            for (int i=0; i < numFlows; i++) {
+                final VersionedFlow createdFlow = createFlow(client, createdBucket, i);
+                System.out.println("Created flow # " + i + " with id " + createdFlow.getIdentifier());
+                createdFlows.add(createdFlow);
+            }
 
-            System.out.println("Created flow with id: " + createdFlow.getIdentifier());
+            final VersionedFlow createdFlow = createdFlows.get(0);
 
             // Create first snapshot for the flow
 
@@ -149,4 +143,36 @@ public class TestRestAPI {
         }
     }
 
+    private static Bucket createBucket(Client client, int num) {
+        final Bucket bucket = new Bucket();
+        bucket.setName("Bucket #" + num);
+        bucket.setDescription("This is bucket #" + num);
+
+        final Bucket createdBucket = client.target(REGISTRY_API_BUCKETS_URL)
+                .request()
+                .post(
+                        Entity.entity(bucket, MediaType.APPLICATION_JSON),
+                        Bucket.class
+                );
+
+        return createdBucket;
+    }
+
+    private static VersionedFlow createFlow(Client client, Bucket bucket, int num) {
+        final VersionedFlow versionedFlow = new VersionedFlow();
+        versionedFlow.setName("Flow #" + num);
+        versionedFlow.setDescription("This is flow #" + num);
+
+        final VersionedFlow createdFlow = client.target(REGISTRY_API_BUCKETS_URL)
+                .path("/{bucketId}/flows")
+                .resolveTemplate("bucketId", bucket.getIdentifier())
+                .request()
+                .post(
+                        Entity.entity(versionedFlow, MediaType.APPLICATION_JSON),
+                        VersionedFlow.class
+                );
+
+        return createdFlow;
+    }
+
 }


[5/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketRepository.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketRepository.java
new file mode 100644
index 0000000..6a0a3ed
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestBucketRepository.java
@@ -0,0 +1,177 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.data.domain.Page;
+import org.springframework.data.domain.PageRequest;
+import org.springframework.data.domain.Pageable;
+import org.springframework.data.domain.Sort;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestBucketRepository extends RepositoryBaseTest {
+
+    @Autowired
+    private BucketRepository bucketRepository;
+
+    @Test
+    public void testCreate() {
+        final BucketEntity bucket = new BucketEntity();
+        bucket.setId(UUID.randomUUID().toString());
+        bucket.setName("Some new bucket");
+        bucket.setDescription("This is some new bucket");
+        bucket.setCreated(new Date());
+
+        final BucketEntity createdBucket = bucketRepository.save(bucket);
+        assertNotNull(createdBucket);
+        assertEquals(bucket.getId(), createdBucket.getId());
+        assertEquals(bucket.getName(), createdBucket.getName());
+        assertEquals(bucket.getDescription(), createdBucket.getDescription());
+        assertEquals(bucket.getCreated(), createdBucket.getCreated());
+    }
+
+    @Test
+    public void testUpdate() {
+        final String id = "1";
+
+        final BucketEntity existingBucket = bucketRepository.findOne(id);
+        assertNotNull(existingBucket);
+
+        final String updatedDescription = existingBucket.getDescription() + " UPDATED";
+        existingBucket.setDescription(updatedDescription);
+
+        bucketRepository.save(existingBucket);
+
+        final BucketEntity updatedBucket = bucketRepository.findOne(id);
+        assertNotNull(updatedBucket);
+        assertEquals(updatedDescription, updatedBucket.getDescription());
+
+        // create date should not have changed
+        assertEquals(existingBucket.getCreated(), updatedBucket.getCreated());
+    }
+
+    @Test
+    public void testDelete() {
+        final String id = "6";
+
+        final BucketEntity existingBucket = bucketRepository.findOne(id);
+        assertNotNull(existingBucket);
+
+        bucketRepository.delete(existingBucket);
+
+        final BucketEntity updatedBucket = bucketRepository.findOne(id);
+        assertNull(updatedBucket);
+    }
+
+    @Test
+    public void testOneToManyWithBucketItems() {
+        final String id = "1";
+
+        final BucketEntity existingBucket = bucketRepository.findOne(id);
+        assertNotNull(existingBucket);
+
+        final Set<BucketItemEntity> items = existingBucket.getItems();
+        assertNotNull(items);
+        assertEquals(2, items.size());
+    }
+
+    @Test
+    public void testFindByNameCaseInsensitive() {
+        final String bucketName = "bUcKEt 1";
+
+        final List<BucketEntity> buckets = bucketRepository.findByNameIgnoreCase(bucketName);
+        assertNotNull(buckets);
+        assertEquals(1, buckets.size());
+
+        final BucketEntity bucket = buckets.get(0);
+        assertEquals(bucketName.toLowerCase(), bucket.getName().toLowerCase());
+    }
+
+    @Test
+    public void testFindAllWithPaging() {
+        final Sort sort = new Sort(Sort.Direction.ASC, "id");
+
+        int pageIndex = 0;
+        int pageSize = 2;
+
+        // query for first page
+        Pageable pageable = new PageRequest(pageIndex, pageSize, sort);
+
+        Page<BucketEntity> page = bucketRepository.findAll(pageable);
+        assertNotNull(page);
+        assertEquals(6, page.getTotalElements());
+        assertEquals(3, page.getTotalPages());
+
+        Iterable<BucketEntity> buckets = page.getContent();
+        assertNotNull(buckets);
+
+        List<String> ids = getIds(buckets);
+        assertEquals(2, ids.size());
+        assertEquals("1", ids.get(0));
+        assertEquals("2", ids.get(1));
+
+        // query for second page
+        pageIndex++;
+        pageable = new PageRequest(pageIndex, pageSize, sort);
+        buckets = bucketRepository.findAll(pageable);
+        assertNotNull(buckets);
+
+        ids = getIds(buckets);
+        assertEquals(2, ids.size());
+        assertEquals("3", ids.get(0));
+        assertEquals("4", ids.get(1));
+
+        // query for third page
+        pageIndex++;
+        pageable = new PageRequest(pageIndex, pageSize, sort);
+        buckets = bucketRepository.findAll(pageable);
+        assertNotNull(buckets);
+
+        ids = getIds(buckets);
+        assertEquals(2, ids.size());
+        assertEquals("5", ids.get(0));
+        assertEquals("6", ids.get(1));
+
+        // query for fourth page
+        pageIndex++;
+        pageable = new PageRequest(pageIndex, pageSize, sort);
+        buckets = bucketRepository.findAll(pageable);
+        assertNotNull(buckets);
+
+        ids = getIds(buckets);
+        assertEquals(0, ids.size());
+    }
+
+    private List<String> getIds(final Iterable<BucketEntity> buckets) {
+        List<String> ids = new ArrayList<>();
+        buckets.forEach(b -> ids.add(b.getId()));
+        return ids;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowRepository.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowRepository.java
new file mode 100644
index 0000000..b176f1e
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowRepository.java
@@ -0,0 +1,125 @@
+/*
+ * 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.repository;
+
+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.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+import java.util.Date;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestFlowRepository extends RepositoryBaseTest {
+
+    @Autowired
+    private BucketRepository bucketRepository;
+
+    @Autowired
+    private FlowRepository flowRepository;
+
+
+    @Test
+    public void testCreateFlow() {
+        final String bucketId = "1";
+
+        final BucketEntity existingBucket = bucketRepository.findOne(bucketId);
+        assertNotNull(existingBucket);
+        assertNotNull(existingBucket.getItems());
+        assertEquals(2, existingBucket.getItems().size());
+
+        // created, and modified should get set automatically
+        final FlowEntity flow = new FlowEntity();
+        flow.setId(UUID.randomUUID().toString());
+        flow.setName("Flow 4");
+        flow.setDescription("This is flow 4");
+        flow.setBucket(existingBucket);
+        flow.setCreated(new Date());
+        flow.setModified(new Date());
+
+        final FlowEntity createdFlow = flowRepository.save(flow);
+        assertEquals(flow.getId(), createdFlow.getId());
+        assertEquals(flow.getName(), createdFlow.getName());
+        assertEquals(flow.getDescription(), createdFlow.getDescription());
+        assertEquals(flow.getCreated(), createdFlow.getCreated());
+        assertEquals(flow.getModified(), createdFlow.getModified());
+        assertEquals(BucketItemEntityType.FLOW, flow.getType());
+    }
+
+    @Test
+    public void testUpdateFlow() {
+        final String flowId = "1";
+
+        final FlowEntity flow = flowRepository.findOne(flowId);
+        assertNotNull(flow);
+        assertEquals(flowId, flow.getId());
+
+        flow.setName(flow.getName() + " UPDATED");
+        flow.setDescription(flow.getDescription() + " UPDATED");
+
+        flowRepository.save(flow);
+
+        final FlowEntity updatedFlow = flowRepository.findOne(flowId);
+        assertEquals(flow.getName(), updatedFlow.getName());
+        assertEquals(flow.getDescription(), updatedFlow.getDescription());
+        assertEquals(flow.getCreated(), updatedFlow.getCreated());
+        assertEquals(flow.getModified(), updatedFlow.getModified());
+        assertEquals(BucketItemEntityType.FLOW, updatedFlow.getType());
+    }
+
+    @Test
+    public void testDeleteFlow() {
+        final String flowId = "1";
+
+        final FlowEntity flow = flowRepository.findOne(flowId);
+        assertNotNull(flow);
+
+        flowRepository.delete(flow);
+
+        final FlowEntity deletedFlow = flowRepository.findOne(flowId);
+        assertNull(deletedFlow);
+    }
+
+    @Test
+    public void testOneToManyWithFlowSnapshots() {
+        final String flowId = "1";
+
+        final FlowEntity flow = flowRepository.findOne(flowId);
+        assertNotNull(flow);
+        assertNotNull(flow.getSnapshots());
+        assertEquals(3, flow.getSnapshots().size());
+    }
+
+    @Test
+    public void testFindFlowByNameCaseInsensitive() {
+        final String flowName = "fLoW 1";
+
+        final List<FlowEntity> flows = flowRepository.findByNameIgnoreCase(flowName);
+        assertNotNull(flows);
+        assertEquals(1, flows.size());
+
+        final FlowEntity flow = flows.get(0);
+        assertEquals(flowName.toLowerCase(), flow.getName().toLowerCase());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowSnapshotRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowSnapshotRepository.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowSnapshotRepository.java
new file mode 100644
index 0000000..2b30272
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestFlowSnapshotRepository.java
@@ -0,0 +1,109 @@
+/*
+ * 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.repository;
+
+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.apache.nifi.registry.db.entity.FlowSnapshotEntityKey;
+import org.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+import java.util.Date;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestFlowSnapshotRepository extends RepositoryBaseTest {
+
+    @Autowired
+    private BucketRepository bucketRepository;
+
+    @Autowired
+    private FlowRepository flowRepository;
+
+    @Autowired
+    private FlowSnapshotRepository flowSnapshotRepository;
+
+
+    @Test
+    public void testCreateFlowSnapshot() {
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId("2");
+        key.setVersion(1);
+
+        final FlowSnapshotEntity flowSnapshot = new FlowSnapshotEntity();
+        flowSnapshot.setId(key);
+        flowSnapshot.setComments("This is snapshot 1 for flow 2");
+        flowSnapshot.setCreated(new Date());
+
+        flowSnapshotRepository.save(flowSnapshot);
+
+        final FlowSnapshotEntity createdFlowSnapshot = flowSnapshotRepository.findOne(key);
+        assertNotNull(createdFlowSnapshot);
+    }
+
+    @Test
+    public void testFindById() {
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId("1");
+        key.setVersion(1);
+
+        final FlowSnapshotEntity flowSnapshot = flowSnapshotRepository.findOne(key);
+        assertNotNull(flowSnapshot);
+        assertEquals(key, flowSnapshot.getId());
+        assertNotNull(flowSnapshot.getFlow());
+    }
+
+    @Test
+    public void testDeleteFlowSnapshot() {
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId("1");
+        key.setVersion(1);
+
+        final FlowSnapshotEntity flowSnapshot = flowSnapshotRepository.findOne(key);
+        assertNotNull(flowSnapshot);
+
+        flowSnapshotRepository.delete(flowSnapshot);
+
+        final FlowSnapshotEntity deletedFlowSnapshot = flowSnapshotRepository.findOne(key);
+        assertNull(deletedFlowSnapshot);
+    }
+
+    @Test
+    public void testDeleteBucketCascadesToSnapshots() {
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId("1");
+        key.setVersion(1);
+
+        final FlowSnapshotEntity flowSnapshot = flowSnapshotRepository.findOne(key);
+        assertNotNull(flowSnapshot);
+
+        final FlowEntity flow = flowSnapshot.getFlow();
+        assertNotNull(flow);
+
+        final BucketEntity bucket = flow.getBucket();
+        assertNotNull(bucket);
+
+        bucketRepository.delete(bucket);
+
+        assertNull(flowRepository.findOne(flow.getId()));
+        assertNull(flowSnapshotRepository.findOne(key));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/flow/TestStandardFlowSnapshotContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/flow/TestStandardFlowSnapshotContext.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/flow/TestStandardFlowSnapshotContext.java
deleted file mode 100644
index e0c7f16..0000000
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/flow/TestStandardFlowSnapshotContext.java
+++ /dev/null
@@ -1,53 +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.flow;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestStandardFlowSnapshotContext {
-
-    @Test
-    public void testBuilder() {
-        final String bucketId = "1234-1234-1234-1234";
-        final String bucketName = "Some Bucket";
-        final String flowId = "2345-2345-2345-2345";
-        final String flowName = "Some Flow";
-        final int version = 2;
-        final String comments = "Some Comments";
-        final long timestamp = System.currentTimeMillis();
-
-        final FlowSnapshotContext context = new StandardFlowSnapshotContext.Builder()
-                .bucketId(bucketId)
-                .bucketName(bucketName)
-                .flowId(flowId)
-                .flowName(flowName)
-                .version(version)
-                .comments(comments)
-                .snapshotTimestamp(timestamp)
-                .build();
-
-        Assert.assertEquals(bucketId, context.getBucketId());
-        Assert.assertEquals(bucketName, context.getBucketName());
-        Assert.assertEquals(flowId, context.getFlowId());
-        Assert.assertEquals(flowName, context.getFlowName());
-        Assert.assertEquals(version, context.getVersion());
-        Assert.assertEquals(comments, context.getComments());
-        Assert.assertEquals(timestamp, context.getSnapshotTimestamp());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/MockMetadataProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/MockMetadataProvider.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/MockMetadataProvider.java
deleted file mode 100644
index 176a9a5..0000000
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/MockMetadataProvider.java
+++ /dev/null
@@ -1,119 +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.provider;
-
-import org.apache.nifi.registry.metadata.BucketMetadata;
-import org.apache.nifi.registry.metadata.FlowMetadata;
-import org.apache.nifi.registry.metadata.FlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.MetadataProvider;
-
-import java.util.Map;
-import java.util.Set;
-
-public class MockMetadataProvider implements MetadataProvider {
-
-    private Map<String,String> properties;
-
-    @Override
-    public void onConfigured(ProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        this.properties = configurationContext.getProperties();
-    }
-
-    public Map<String,String> getProperties() {
-        return properties;
-    }
-
-    @Override
-    public BucketMetadata createBucket(BucketMetadata bucket) {
-        return null;
-    }
-
-    @Override
-    public BucketMetadata getBucketById(String bucketIdentifier) {
-        return null;
-    }
-
-    @Override
-    public BucketMetadata getBucketByName(String name) {
-        return null;
-    }
-
-    @Override
-    public BucketMetadata updateBucket(BucketMetadata bucket) {
-        return null;
-    }
-
-    @Override
-    public void deleteBucket(String bucketIdentifier) {
-
-    }
-
-    @Override
-    public Set<BucketMetadata> getBuckets() {
-        return null;
-    }
-
-    @Override
-    public FlowMetadata createFlow(String bucketIdentifier, FlowMetadata flow) {
-        return null;
-    }
-
-    @Override
-    public FlowMetadata getFlowById(String flowIdentifier) {
-        return null;
-    }
-
-    @Override
-    public FlowMetadata getFlowByName(String name) {
-        return null;
-    }
-
-    @Override
-    public FlowMetadata updateFlow(FlowMetadata versionedFlow) {
-        return null;
-    }
-
-    @Override
-    public void deleteFlow(String flowIdentifier) {
-
-    }
-
-    @Override
-    public Set<FlowMetadata> getFlows() {
-        return null;
-    }
-
-    @Override
-    public Set<FlowMetadata> getFlows(String bucketId) {
-        return null;
-    }
-
-    @Override
-    public FlowSnapshotMetadata createFlowSnapshot(FlowSnapshotMetadata flowSnapshot) {
-        return null;
-    }
-
-    @Override
-    public FlowSnapshotMetadata getFlowSnapshot(String flowIdentifier, Integer version) {
-        return null;
-    }
-
-    @Override
-    public void deleteFlowSnapshot(String flowIdentifier, Integer version) {
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/TestStandardProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/TestStandardProviderFactory.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/TestStandardProviderFactory.java
index 2bec5ba..f5ad445 100644
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/TestStandardProviderFactory.java
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/TestStandardProviderFactory.java
@@ -17,13 +17,11 @@
 package org.apache.nifi.registry.provider;
 
 import org.apache.nifi.registry.flow.FlowPersistenceProvider;
-import org.apache.nifi.registry.metadata.MetadataProvider;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 
 public class TestStandardProviderFactory {
 
@@ -35,15 +33,6 @@ public class TestStandardProviderFactory {
         final ProviderFactory providerFactory = new StandardProviderFactory(props);
         providerFactory.initialize();
 
-        final MetadataProvider metadataProvider = providerFactory.getMetadataProvider();
-        assertNotNull(metadataProvider);
-        assertTrue(metadataProvider instanceof MockMetadataProvider);
-
-        final MockMetadataProvider mockMetadataProvider = (MockMetadataProvider) metadataProvider;
-        assertNotNull(mockMetadataProvider.getProperties());
-        assertEquals("metadata foo", mockMetadataProvider.getProperties().get("Metadata Property 1"));
-        assertEquals("metadata bar", mockMetadataProvider.getProperties().get("Metadata Property 2"));
-
         final FlowPersistenceProvider flowPersistenceProvider = providerFactory.getFlowPersistenceProvider();
         assertNotNull(flowPersistenceProvider);
 
@@ -54,15 +43,6 @@ public class TestStandardProviderFactory {
     }
 
     @Test(expected = ProviderFactoryException.class)
-    public void testGetMetadataProviderBeforeInitializingShouldThrowException() {
-        final NiFiRegistryProperties props = new NiFiRegistryProperties();
-        props.setProperty(NiFiRegistryProperties.PROVIDERS_CONFIGURATION_FILE, "src/test/resources/provider/providers-good.xml");
-
-        final ProviderFactory providerFactory = new StandardProviderFactory(props);
-        providerFactory.getMetadataProvider();
-    }
-
-    @Test(expected = ProviderFactoryException.class)
     public void testGetFlowProviderBeforeInitializingShouldThrowException() {
         final NiFiRegistryProperties props = new NiFiRegistryProperties();
         props.setProperty(NiFiRegistryProperties.PROVIDERS_CONFIGURATION_FILE, "src/test/resources/provider/providers-good.xml");
@@ -81,17 +61,6 @@ public class TestStandardProviderFactory {
     }
 
     @Test(expected = ProviderFactoryException.class)
-    public void testMetadataProviderClassNotFound() {
-        final NiFiRegistryProperties props = new NiFiRegistryProperties();
-        props.setProperty(NiFiRegistryProperties.PROVIDERS_CONFIGURATION_FILE, "src/test/resources/provider/providers-class-not-found.xml");
-
-        final ProviderFactory providerFactory = new StandardProviderFactory(props);
-        providerFactory.initialize();
-
-        providerFactory.getMetadataProvider();
-    }
-
-    @Test(expected = ProviderFactoryException.class)
     public void testFlowProviderClassNotFound() {
         final NiFiRegistryProperties props = new NiFiRegistryProperties();
         props.setProperty(NiFiRegistryProperties.PROVIDERS_CONFIGURATION_FILE, "src/test/resources/provider/providers-class-not-found.xml");

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestFileSystemFlowPersistenceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestFileSystemFlowPersistenceProvider.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestFileSystemFlowPersistenceProvider.java
new file mode 100644
index 0000000..e51aa7f
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestFileSystemFlowPersistenceProvider.java
@@ -0,0 +1,204 @@
+/*
+ * 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.provider.flow;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.registry.flow.FlowPersistenceProvider;
+import org.apache.nifi.registry.flow.FlowSnapshotContext;
+import org.apache.nifi.registry.provider.ProviderConfigurationContext;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.mockito.Mockito.when;
+
+public class TestFileSystemFlowPersistenceProvider {
+
+    static final String FLOW_STORAGE_DIR = "target/flow_storage";
+
+    static final ProviderConfigurationContext CONFIGURATION_CONTEXT = new ProviderConfigurationContext() {
+        @Override
+        public Map<String, String> getProperties() {
+            final Map<String,String> props = new HashMap<>();
+            props.put(FileSystemFlowPersistenceProvider.FLOW_STORAGE_DIR_PROP, FLOW_STORAGE_DIR);
+            return props;
+        }
+    };
+
+    private File flowStorageDir;
+    private FileSystemFlowPersistenceProvider fileSystemFlowProvider;
+
+    @Before
+    public void setup() throws IOException {
+        flowStorageDir = new File(FLOW_STORAGE_DIR);
+        if (flowStorageDir.exists()) {
+            org.apache.commons.io.FileUtils.cleanDirectory(flowStorageDir);
+            flowStorageDir.delete();
+        }
+
+        Assert.assertFalse(flowStorageDir.exists());
+
+        fileSystemFlowProvider = new FileSystemFlowPersistenceProvider();
+        fileSystemFlowProvider.onConfigured(CONFIGURATION_CONTEXT);
+        Assert.assertTrue(flowStorageDir.exists());
+    }
+
+    @Test
+    public void testSaveSuccessfully() throws IOException {
+        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 1, "flow1v1");
+        verifySnapshot(flowStorageDir, "bucket1", "flow1", 1, "flow1v1");
+
+        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 2, "flow1v2");
+        verifySnapshot(flowStorageDir, "bucket1", "flow1", 2, "flow1v2");
+
+        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow2", 1, "flow2v1");
+        verifySnapshot(flowStorageDir, "bucket1", "flow2", 1, "flow2v1");
+
+        createAndSaveSnapshot(fileSystemFlowProvider,"bucket2", "flow3", 1, "flow3v1");
+        verifySnapshot(flowStorageDir, "bucket2", "flow3", 1, "flow3v1");
+    }
+
+    @Test
+    public void testSaveWithExistingVersion() throws IOException {
+        final FlowSnapshotContext context = Mockito.mock(FlowSnapshotContext.class);
+        when(context.getBucketId()).thenReturn("bucket1");
+        when(context.getFlowId()).thenReturn("flow1");
+        when(context.getVersion()).thenReturn(1);
+
+        final byte[] content = "flow1v1".getBytes(StandardCharsets.UTF_8);
+        fileSystemFlowProvider.saveSnapshot(context, content);
+
+        // save new content for an existing version
+        final byte[] content2 = "XXX".getBytes(StandardCharsets.UTF_8);
+        try {
+            fileSystemFlowProvider.saveSnapshot(context, content2);
+            Assert.fail("Should have thrown exception");
+        } catch (Exception e) {
+
+        }
+
+        // verify the new content wasn't written
+        final File flowSnapshotFile = new File(flowStorageDir, "bucket1/flow1/1/1" + FileSystemFlowPersistenceProvider.SNAPSHOT_EXTENSION);
+        try (InputStream in = new FileInputStream(flowSnapshotFile)) {
+            Assert.assertEquals("flow1v1", IOUtils.toString(in, StandardCharsets.UTF_8));
+        }
+    }
+
+    @Test
+    public void testSaveAndGet() throws IOException {
+        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 1, "flow1v1");
+        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 2, "flow1v2");
+
+        final byte[] flow1v1 = fileSystemFlowProvider.getSnapshot("bucket1", "flow1", 1);
+        Assert.assertEquals("flow1v1", new String(flow1v1, StandardCharsets.UTF_8));
+
+        final byte[] flow1v2 = fileSystemFlowProvider.getSnapshot("bucket1", "flow1", 2);
+        Assert.assertEquals("flow1v2", new String(flow1v2, StandardCharsets.UTF_8));
+    }
+
+    @Test
+    public void testGetWhenDoesNotExist() {
+        final byte[] flow1v1 = fileSystemFlowProvider.getSnapshot("bucket1", "flow1", 1);
+        Assert.assertNull(flow1v1);
+    }
+
+    @Test
+    public void testDeleteSnapshots() throws IOException {
+        final String bucketId = "bucket1";
+        final String flowId = "flow1";
+
+        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 1, "flow1v1");
+        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 2, "flow1v2");
+
+        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
+        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
+
+        fileSystemFlowProvider.deleteSnapshots(bucketId, flowId);
+
+        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
+        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
+
+        // delete a flow that doesn't exist
+        fileSystemFlowProvider.deleteSnapshots(bucketId, "some-other-flow");
+
+        // delete a bucket that doesn't exist
+        fileSystemFlowProvider.deleteSnapshots("some-other-bucket", flowId);
+    }
+
+    @Test
+    public void testDeleteSnapshot() throws IOException {
+        final String bucketId = "bucket1";
+        final String flowId = "flow1";
+
+        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 1, "flow1v1");
+        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 2, "flow1v2");
+
+        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
+        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
+
+        fileSystemFlowProvider.deleteSnapshot(bucketId, flowId, 1);
+
+        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
+        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
+
+        fileSystemFlowProvider.deleteSnapshot(bucketId, flowId, 2);
+
+        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
+        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
+
+        // delete a version that doesn't exist
+        fileSystemFlowProvider.deleteSnapshot(bucketId, flowId, 3);
+
+        // delete a flow that doesn't exist
+        fileSystemFlowProvider.deleteSnapshot(bucketId, "some-other-flow", 1);
+
+        // delete a bucket that doesn't exist
+        fileSystemFlowProvider.deleteSnapshot("some-other-bucket", flowId, 1);
+    }
+
+    private void createAndSaveSnapshot(final FlowPersistenceProvider flowPersistenceProvider, final String bucketId, final String flowId, final int version,
+                                       final String contentString) throws IOException {
+        final FlowSnapshotContext context = Mockito.mock(FlowSnapshotContext.class);
+        when(context.getBucketId()).thenReturn(bucketId);
+        when(context.getFlowId()).thenReturn(flowId);
+        when(context.getVersion()).thenReturn(version);
+
+        final byte[] content = contentString.getBytes(StandardCharsets.UTF_8);
+        flowPersistenceProvider.saveSnapshot(context, content);
+    }
+
+    private void verifySnapshot(final File flowStorageDir, final String bucketId, final String flowId, final int version,
+                                final String contentString) throws IOException {
+        // verify the correct snapshot file was created
+        final File flowSnapshotFile = new File(flowStorageDir,
+                bucketId + "/" + flowId + "/" + version + "/" + version + FileSystemFlowPersistenceProvider.SNAPSHOT_EXTENSION);
+        Assert.assertTrue(flowSnapshotFile.exists());
+
+        try (InputStream in = new FileInputStream(flowSnapshotFile)) {
+            Assert.assertEquals(contentString, IOUtils.toString(in, StandardCharsets.UTF_8));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestStandardFlowSnapshotContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestStandardFlowSnapshotContext.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestStandardFlowSnapshotContext.java
new file mode 100644
index 0000000..3e0a106
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/provider/flow/TestStandardFlowSnapshotContext.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.provider.flow;
+
+import org.apache.nifi.registry.flow.FlowSnapshotContext;
+import org.apache.nifi.registry.provider.flow.StandardFlowSnapshotContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestStandardFlowSnapshotContext {
+
+    @Test
+    public void testBuilder() {
+        final String bucketId = "1234-1234-1234-1234";
+        final String bucketName = "Some Bucket";
+        final String flowId = "2345-2345-2345-2345";
+        final String flowName = "Some Flow";
+        final int version = 2;
+        final String comments = "Some Comments";
+        final long timestamp = System.currentTimeMillis();
+
+        final FlowSnapshotContext context = new StandardFlowSnapshotContext.Builder()
+                .bucketId(bucketId)
+                .bucketName(bucketName)
+                .flowId(flowId)
+                .flowName(flowName)
+                .version(version)
+                .comments(comments)
+                .snapshotTimestamp(timestamp)
+                .build();
+
+        Assert.assertEquals(bucketId, context.getBucketId());
+        Assert.assertEquals(bucketName, context.getBucketName());
+        Assert.assertEquals(flowId, context.getFlowId());
+        Assert.assertEquals(flowName, context.getFlowName());
+        Assert.assertEquals(version, context.getVersion());
+        Assert.assertEquals(comments, context.getComments());
+        Assert.assertEquals(timestamp, context.getSnapshotTimestamp());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestDataModelMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestDataModelMapper.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestDataModelMapper.java
deleted file mode 100644
index 1a6f3c9..0000000
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestDataModelMapper.java
+++ /dev/null
@@ -1,174 +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.service;
-
-import org.apache.nifi.registry.bucket.Bucket;
-import org.apache.nifi.registry.flow.VersionedFlow;
-import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.BucketMetadata;
-import org.apache.nifi.registry.metadata.FlowMetadata;
-import org.apache.nifi.registry.metadata.FlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.StandardBucketMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowSnapshotMetadata;
-import org.junit.Test;
-
-import java.util.LinkedHashSet;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-public class TestDataModelMapper {
-
-    @Test
-    public void testMapBucketToBucketMetadata() {
-        // create a bucket
-        final Bucket bucket = new Bucket();
-        bucket.setIdentifier("bucket1");
-        bucket.setName("Bucket 1");
-        bucket.setDescription("This is bucket 1.");
-        bucket.setCreatedTimestamp(System.currentTimeMillis());
-
-        // create a flow
-        final VersionedFlow versionedFlow = new VersionedFlow();
-        versionedFlow.setIdentifier("flow1");
-        versionedFlow.setName("Flow 1");
-        versionedFlow.setDescription("This is flow 1");
-        versionedFlow.setBucketIdentifier(bucket.getIdentifier());
-        versionedFlow.setCreatedTimestamp(System.currentTimeMillis());
-        versionedFlow.setModifiedTimestamp(System.currentTimeMillis());
-
-        // create a snapshot for the flow
-        final VersionedFlowSnapshotMetadata versionedFlowSnapshotMetadata = new VersionedFlowSnapshotMetadata();
-        versionedFlowSnapshotMetadata.setBucketIdentifier(bucket.getIdentifier());
-        versionedFlowSnapshotMetadata.setFlowIdentifier(versionedFlow.getIdentifier());
-        versionedFlowSnapshotMetadata.setFlowName(versionedFlow.getName());
-        versionedFlowSnapshotMetadata.setVersion(1);
-        versionedFlowSnapshotMetadata.setTimestamp(System.currentTimeMillis());
-        versionedFlowSnapshotMetadata.setComments("This is snapshot 1 of flow 1");
-
-        // add the snapshot to the flow
-        final SortedSet<VersionedFlowSnapshotMetadata> versionedFlowSnapshotMetadataSet = new TreeSet<>();
-        versionedFlowSnapshotMetadataSet.add(versionedFlowSnapshotMetadata);
-        versionedFlow.setSnapshotMetadata(versionedFlowSnapshotMetadataSet);
-
-        // add the flow to the bucket
-        final Set<VersionedFlow> versionedFlows = new LinkedHashSet<>();
-        versionedFlows.add(versionedFlow);
-        bucket.setVersionedFlows(versionedFlows);
-
-        // test the mapping from bucket to bucket metadata
-
-        final BucketMetadata bucketMetadata = DataModelMapper.map(bucket);
-        assertEquals(bucket.getIdentifier(), bucketMetadata.getIdentifier());
-        assertEquals(bucket.getName(), bucketMetadata.getName());
-        assertEquals(bucket.getDescription(), bucketMetadata.getDescription());
-        assertEquals(bucket.getCreatedTimestamp(), bucketMetadata.getCreatedTimestamp());
-
-        assertNotNull(bucketMetadata.getFlowMetadata());
-        assertEquals(1, bucketMetadata.getFlowMetadata().size());
-
-        final FlowMetadata flowMetadata = bucketMetadata.getFlowMetadata().iterator().next();
-        assertNotNull(flowMetadata);
-        assertEquals(versionedFlow.getIdentifier(), flowMetadata.getIdentifier());
-        assertEquals(versionedFlow.getName(), flowMetadata.getName());
-        assertEquals(versionedFlow.getDescription(), flowMetadata.getDescription());
-        assertEquals(versionedFlow.getBucketIdentifier(), flowMetadata.getBucketIdentifier());
-        assertEquals(versionedFlow.getCreatedTimestamp(), flowMetadata.getCreatedTimestamp());
-        assertEquals(versionedFlow.getModifiedTimestamp(), flowMetadata.getModifiedTimestamp());
-
-        assertNotNull(flowMetadata.getSnapshotMetadata());
-        assertEquals(1, flowMetadata.getSnapshotMetadata().size());
-
-        final FlowSnapshotMetadata flowSnapshotMetadata = flowMetadata.getSnapshotMetadata().iterator().next();
-        assertNotNull(flowSnapshotMetadata);
-        assertEquals(versionedFlowSnapshotMetadata.getFlowIdentifier(), flowSnapshotMetadata.getFlowIdentifier());
-        assertEquals(versionedFlowSnapshotMetadata.getFlowName(), flowSnapshotMetadata.getFlowName());
-        assertEquals(versionedFlowSnapshotMetadata.getBucketIdentifier(), flowSnapshotMetadata.getBucketIdentifier());
-        assertEquals(versionedFlowSnapshotMetadata.getVersion(), flowSnapshotMetadata.getVersion());
-        assertEquals(versionedFlowSnapshotMetadata.getComments(), flowSnapshotMetadata.getComments());
-        assertEquals(versionedFlowSnapshotMetadata.getTimestamp(), flowSnapshotMetadata.getCreatedTimestamp());
-    }
-
-    @Test
-    public void testMapBucketMetadataToBucket() {
-        // create snapshot metadata
-        final FlowSnapshotMetadata snapshotMetadata = new StandardFlowSnapshotMetadata.Builder()
-                .flowIdentifier("flow1")
-                .flowName("Flow 1")
-                .bucketIdentifier("bucket1")
-                .version(1)
-                .comments("This is snapshot 1 of flow 1.")
-                .created(System.currentTimeMillis())
-                .build();
-
-        // create flow metadata
-        final FlowMetadata flowMetadata = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("Flow 1")
-                .bucketIdentifier("bucket1")
-                .description("This flow 1.")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .addSnapshot(snapshotMetadata)
-                .build();
-
-        // create bucket metadata
-        final BucketMetadata bucketMetadata = new StandardBucketMetadata.Builder()
-                .identifier("bucket1")
-                .name("Bucket 1")
-                .description("This is bucket 1.")
-                .created(System.currentTimeMillis())
-                .addFlow(flowMetadata)
-                .build();
-
-        // test the mapping from bucket metadata to bucket
-
-        final Bucket bucket = DataModelMapper.map(bucketMetadata);
-        assertEquals(bucketMetadata.getIdentifier(), bucket.getIdentifier());
-        assertEquals(bucketMetadata.getName(), bucket.getName());
-        assertEquals(bucketMetadata.getDescription(), bucket.getDescription());
-        assertEquals(bucketMetadata.getCreatedTimestamp(), bucket.getCreatedTimestamp());
-
-        assertNotNull(bucket.getVersionedFlows());
-        assertEquals(1, bucket.getVersionedFlows().size());
-
-        final VersionedFlow versionedFlow = bucket.getVersionedFlows().iterator().next();
-        assertNotNull(versionedFlow);
-        assertEquals(flowMetadata.getIdentifier(), versionedFlow.getIdentifier());
-        assertEquals(flowMetadata.getName(), versionedFlow.getName());
-        assertEquals(flowMetadata.getBucketIdentifier(), versionedFlow.getBucketIdentifier());
-        assertEquals(flowMetadata.getDescription(), versionedFlow.getDescription());
-        assertEquals(flowMetadata.getCreatedTimestamp(), versionedFlow.getCreatedTimestamp());
-        assertEquals(flowMetadata.getModifiedTimestamp(), versionedFlow.getModifiedTimestamp());
-
-        assertNotNull(versionedFlow.getSnapshotMetadata());
-        assertEquals(1, versionedFlow.getSnapshotMetadata().size());
-
-        final VersionedFlowSnapshotMetadata versionedFlowSnapshotMetadata = versionedFlow.getSnapshotMetadata().first();
-        assertEquals(snapshotMetadata.getFlowIdentifier(), versionedFlowSnapshotMetadata.getFlowIdentifier());
-        assertEquals(snapshotMetadata.getFlowName(), versionedFlowSnapshotMetadata.getFlowName());
-        assertEquals(snapshotMetadata.getBucketIdentifier(), versionedFlowSnapshotMetadata.getBucketIdentifier());
-        assertEquals(snapshotMetadata.getVersion(), versionedFlowSnapshotMetadata.getVersion());
-        assertEquals(snapshotMetadata.getComments(), versionedFlowSnapshotMetadata.getComments());
-        assertEquals(snapshotMetadata.getCreatedTimestamp(), versionedFlowSnapshotMetadata.getTimestamp());
-    }
-
-}


[7/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Posted by bb...@apache.org.
NIFIREG-18 Initial plumbling for H2 database
- Setup Flyway with initial migration to define tables
- Setup entity classes with repositories
- Setup unit testing for repositories
- Removed existing MetadataProvider concept
- Removed provider impl module and moved remaining pieces into framework
- Added MetadataService with DatabaseMetadataService implementation
- Refactored RegistryService to use MetadataService
- Introduced verbose flag on some end-points to control loading nested objects
- Added ability to pass down paging/sorting params
- Added endpoints for available fields
- Adding ItemResource and ability to retrieve all items, or items by bucket
- Changing from Set to List on retrieval methods
- Added web linking in entities returned from REST end-points

This closes #10.


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

Branch: refs/heads/master
Commit: d478c20ee26ae6d021573dd842bb9098ed8c752d
Parents: d93eab3
Author: Bryan Bende <bb...@apache.org>
Authored: Tue Sep 19 10:01:24 2017 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Thu Sep 21 09:50:04 2017 -0400

----------------------------------------------------------------------
 nifi-registry-assembly/pom.xml                  |   3 +
 nifi-registry-data-model/pom.xml                |   4 +
 .../org/apache/nifi/registry/bucket/Bucket.java |   9 +-
 .../apache/nifi/registry/bucket/BucketItem.java |  21 +-
 .../nifi/registry/flow/VersionedFlow.java       |  11 -
 .../flow/VersionedFlowSnapshotMetadata.java     |   3 +-
 .../apache/nifi/registry/link/LinkAdapter.java  |  67 ++
 .../nifi/registry/link/LinkableEntity.java      |  45 +
 nifi-registry-framework/pom.xml                 |  50 +-
 .../nifi/registry/db/DataSourceFactory.java     |  85 ++
 .../registry/db/DatabaseMetadataService.java    | 394 +++++++++
 .../nifi/registry/db/entity/BucketEntity.java   | 123 +++
 .../registry/db/entity/BucketItemEntity.java    | 135 +++
 .../db/entity/BucketItemEntityType.java         |  42 +
 .../nifi/registry/db/entity/FlowEntity.java     |  57 ++
 .../registry/db/entity/FlowSnapshotEntity.java  |  93 +++
 .../db/entity/FlowSnapshotEntityKey.java        |  82 ++
 .../db/repository/BucketItemRepository.java     |  84 ++
 .../db/repository/BucketRepository.java         |  31 +
 .../registry/db/repository/FlowRepository.java  |  31 +
 .../db/repository/FlowSnapshotRepository.java   |  28 +
 .../flow/StandardFlowSnapshotContext.java       | 155 ----
 .../nifi/registry/provider/ProviderFactory.java |   6 -
 .../provider/StandardProviderFactory.java       |  38 +-
 .../flow/FileSystemFlowPersistenceProvider.java | 169 ++++
 .../flow/StandardFlowSnapshotContext.java       | 157 ++++
 .../serialization/jaxb/JAXBSerializer.java      |   2 +-
 .../nifi/registry/service/DataModelMapper.java  | 131 +--
 .../nifi/registry/service/MetadataService.java  | 186 +++++
 .../nifi/registry/service/RegistryService.java  | 293 ++++---
 .../nifi/registry/service/TestService.java      |  31 -
 .../service/params/QueryParameters.java         | 109 +++
 .../nifi/registry/service/params/SortOrder.java |  47 ++
 .../registry/service/params/SortParameter.java  |  79 ++
 .../main/resources/db/migration/V1__Initial.sql |  46 ++
 .../src/main/xsd/providers.xsd                  |   1 -
 .../registry/db/RepositoryTestApplication.java  |  45 +
 .../db/repository/RepositoryBaseTest.java       |  34 +
 .../db/repository/TestBucketItemRepository.java |  98 +++
 .../db/repository/TestBucketRepository.java     | 177 ++++
 .../db/repository/TestFlowRepository.java       | 125 +++
 .../repository/TestFlowSnapshotRepository.java  | 109 +++
 .../flow/TestStandardFlowSnapshotContext.java   |  53 --
 .../registry/provider/MockMetadataProvider.java | 119 ---
 .../provider/TestStandardProviderFactory.java   |  31 -
 .../TestFileSystemFlowPersistenceProvider.java  | 204 +++++
 .../flow/TestStandardFlowSnapshotContext.java   |  55 ++
 .../registry/service/TestDataModelMapper.java   | 174 ----
 .../registry/service/TestRegistryService.java   | 828 ++++++++++---------
 .../src/test/resources/application.properties   |  19 +
 .../db/migration/V999999.1__test-setup.sql      |  64 ++
 .../provider/providers-class-not-found.xml      |   6 -
 .../test/resources/provider/providers-good.xml  |   6 -
 .../properties/NiFiRegistryProperties.java      |  12 +
 .../nifi/registry/metadata/BucketMetadata.java  |  51 --
 .../nifi/registry/metadata/FlowMetadata.java    |  69 --
 .../registry/metadata/FlowSnapshotMetadata.java |  54 --
 .../registry/metadata/MetadataProvider.java     | 158 ----
 .../metadata/MetadataProviderException.java     |  35 -
 .../metadata/StandardBucketMetadata.java        | 131 ---
 .../registry/metadata/StandardFlowMetadata.java | 174 ----
 .../metadata/StandardFlowSnapshotMetadata.java  | 136 ---
 nifi-registry-provider-impl/pom.xml             |  94 ---
 .../flow/FileSystemFlowPersistenceProvider.java | 166 ----
 .../metadata/FileSystemMetadataProvider.java    | 469 -----------
 .../nifi/registry/metadata/MetadataHolder.java  | 176 ----
 .../src/main/xsd/metadata.xsd                   | 139 ----
 .../TestFileSystemFlowPersistenceProvider.java  | 202 -----
 .../TestFileSystemMetadataProvider.java         | 549 ------------
 .../metadata/metadata-empty-concise.xml         |  16 -
 .../test/resources/metadata/metadata-empty.xml  |  17 -
 .../resources/metadata/metadata-existing.xml    |  35 -
 .../src/main/resources/conf/logback.xml         |   5 +
 .../resources/conf/nifi-registry.properties     |  10 +-
 .../src/main/resources/conf/providers.xml       |   7 +-
 nifi-registry-web-api/pom.xml                   |  12 +-
 .../registry/NiFiRegistryApiApplication.java    |  54 ++
 .../registry/NiFiRegistryPropertiesFactory.java |  47 ++
 .../web/NiFiRegistryApiApplication.java         |  44 -
 .../web/NiFiRegistryPropertiesFactory.java      |  46 --
 .../web/NiFiRegistryResourceConfig.java         |   6 +-
 .../nifi/registry/web/api/BucketResource.java   |  55 +-
 .../nifi/registry/web/api/FlowResource.java     |  70 +-
 .../nifi/registry/web/api/ItemResource.java     | 125 +++
 .../nifi/registry/web/api/TestResource.java     |  55 --
 .../nifi/registry/web/link/LinkService.java     | 110 +++
 .../web/link/builder/BucketLinkBuilder.java     |  45 +
 .../registry/web/link/builder/LinkBuilder.java  |  30 +
 .../link/builder/VersionedFlowLinkBuilder.java  |  45 +
 .../VersionedFlowSnapshotLinkBuilder.java       |  46 ++
 .../registry/web/response/FieldsEntity.java     |  37 +
 .../nifi/registry/web/response/TestEntity.java  |  31 -
 .../apache/nifi/registry/web/TestRestAPI.java   |  78 +-
 .../nifi/registry/web/link/TestLinkService.java | 119 +++
 .../src/main/webapp/WEB-INF/web.xml             |   1 +
 pom.xml                                         |  30 +-
 96 files changed, 4631 insertions(+), 4155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-assembly/pom.xml b/nifi-registry-assembly/pom.xml
index b2403de..7ed4ff0 100644
--- a/nifi-registry-assembly/pom.xml
+++ b/nifi-registry-assembly/pom.xml
@@ -140,5 +140,8 @@
         <!-- nifi-registry.properties: provider properties -->
         <nifi.registry.providers.configuration.file>./conf/providers.xml</nifi.registry.providers.configuration.file>
 
+        <!-- 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>
     </properties>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/pom.xml b/nifi-registry-data-model/pom.xml
index 438fe01..3b6d4b1 100644
--- a/nifi-registry-data-model/pom.xml
+++ b/nifi-registry-data-model/pom.xml
@@ -30,5 +30,9 @@
             <groupId>javax.validation</groupId>
             <artifactId>validation-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/Bucket.java
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/Bucket.java b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/Bucket.java
index 9928cd8..314598e 100644
--- a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/Bucket.java
+++ b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/Bucket.java
@@ -18,6 +18,7 @@ package org.apache.nifi.registry.bucket;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.registry.link.LinkableEntity;
 import org.apache.nifi.registry.flow.VersionedFlow;
 
 import javax.validation.Valid;
@@ -29,7 +30,7 @@ import java.util.Set;
 
 @XmlRootElement
 @ApiModel(value = "bucket")
-public class Bucket {
+public class Bucket extends LinkableEntity {
 
     @NotBlank
     private String identifier;
@@ -46,7 +47,7 @@ public class Bucket {
     private Set<VersionedFlow> versionedFlows;
 
 
-    @ApiModelProperty("The id of the bucket. This is set by the server at creation time.")
+    @ApiModelProperty("An ID to uniquely identify this object.")
     public String getIdentifier() {
         return identifier;
     }
@@ -93,7 +94,7 @@ public class Bucket {
 
     @Override
     public int hashCode() {
-        return Objects.hashCode(this.identifier);
+        return Objects.hashCode(this.getIdentifier());
     }
 
     @Override
@@ -106,7 +107,7 @@ public class Bucket {
         }
 
         final Bucket other = (Bucket) obj;
-        return Objects.equals(this.identifier, other.identifier);
+        return Objects.equals(this.getIdentifier(), other.getIdentifier());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/BucketItem.java
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/BucketItem.java b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/BucketItem.java
index d57f07b..cb55664 100644
--- a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/BucketItem.java
+++ b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/bucket/BucketItem.java
@@ -18,6 +18,7 @@ package org.apache.nifi.registry.bucket;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.registry.link.LinkableEntity;
 
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotBlank;
@@ -25,7 +26,7 @@ import javax.validation.constraints.NotNull;
 import java.util.Objects;
 
 @ApiModel("bucketItem")
-public abstract class BucketItem {
+public abstract class BucketItem extends LinkableEntity {
 
     @NotBlank
     private String identifier;
@@ -33,6 +34,8 @@ public abstract class BucketItem {
     @NotBlank
     private String name;
 
+    private String description;
+
     @NotBlank
     private String bucketIdentifier;
 
@@ -45,11 +48,11 @@ public abstract class BucketItem {
     @NotNull
     private final BucketItemType type;
 
+
     public BucketItem(final BucketItemType type) {
         this.type = type;
     }
 
-
     @ApiModelProperty("An ID to uniquely identify this object.")
     public String getIdentifier() {
         return identifier;
@@ -68,6 +71,16 @@ public abstract class BucketItem {
         this.name = name;
     }
 
+    @ApiModelProperty("A description of the item.")
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+
     @ApiModelProperty("The identifier of the bucket this items belongs to.")
     public String getBucketIdentifier() {
         return bucketIdentifier;
@@ -102,7 +115,7 @@ public abstract class BucketItem {
 
     @Override
     public int hashCode() {
-        return Objects.hashCode(this.identifier);
+        return Objects.hashCode(this.getIdentifier());
     }
 
     @Override
@@ -115,6 +128,6 @@ public abstract class BucketItem {
         }
 
         final BucketItem other = (BucketItem) obj;
-        return Objects.equals(this.identifier, other.identifier);
+        return Objects.equals(this.getIdentifier(), other.getIdentifier());
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlow.java
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlow.java b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlow.java
index 8bbb040..b70f6be 100644
--- a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlow.java
+++ b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlow.java
@@ -38,8 +38,6 @@ import java.util.SortedSet;
 @ApiModel(value = "versionedFlow")
 public class VersionedFlow extends BucketItem {
 
-    private String description;
-
     @Valid
     private SortedSet<VersionedFlowSnapshotMetadata> snapshotMetadata;
 
@@ -47,15 +45,6 @@ public class VersionedFlow extends BucketItem {
         super(BucketItemType.FLOW);
     }
 
-    @ApiModelProperty("A description of the flow.")
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
     @ApiModelProperty(value = "The metadata for each snapshot of this flow.", readOnly = true)
     public SortedSet<VersionedFlowSnapshotMetadata> getSnapshotMetadata() {
         return snapshotMetadata;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlowSnapshotMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlowSnapshotMetadata.java b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlowSnapshotMetadata.java
index e60dcfd..36a4cf9 100644
--- a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlowSnapshotMetadata.java
+++ b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedFlowSnapshotMetadata.java
@@ -18,6 +18,7 @@ package org.apache.nifi.registry.flow;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.registry.link.LinkableEntity;
 
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotBlank;
@@ -28,7 +29,7 @@ import java.util.Objects;
  * to sort based on the snapshot version in ascending order.
  */
 @ApiModel(value = "versionedFlowSnapshot")
-public class VersionedFlowSnapshotMetadata implements Comparable<VersionedFlowSnapshotMetadata> {
+public class VersionedFlowSnapshotMetadata extends LinkableEntity implements Comparable<VersionedFlowSnapshotMetadata> {
 
     @NotBlank
     private String bucketIdentifier;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java
new file mode 100644
index 0000000..c3dae90
--- /dev/null
+++ b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.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.link;
+
+import javax.ws.rs.core.Link;
+import javax.xml.bind.annotation.adapters.XmlAdapter;
+import javax.xml.namespace.QName;
+import java.util.Map;
+
+/**
+ * This class is a modified version of Jersey's Link.JaxbAdapter that adds protection against nulls.
+ */
+public class LinkAdapter extends XmlAdapter<Link.JaxbLink, Link> {
+
+    /**
+     * Convert a {@link Link.JaxbLink} into a {@link Link}.
+     *
+     * @param v instance of type {@link Link.JaxbLink}.
+     * @return mapped instance of type {@link Link.JaxbLink}
+     */
+    @Override
+    public Link unmarshal(Link.JaxbLink v) {
+        if (v == null) {
+            return null;
+        }
+
+        Link.Builder lb = Link.fromUri(v.getUri());
+        for (Map.Entry<QName, Object> e : v.getParams().entrySet()) {
+            lb.param(e.getKey().getLocalPart(), e.getValue().toString());
+        }
+        return lb.build();
+    }
+
+    /**
+     * Convert a {@link Link} into a {@link Link.JaxbLink}.
+     *
+     * @param v instance of type {@link Link}.
+     * @return mapped instance of type {@link Link.JaxbLink}.
+     */
+    @Override
+    public Link.JaxbLink marshal(Link v) {
+        if (v == null) {
+           return null;
+        }
+
+        Link.JaxbLink jl = new Link.JaxbLink(v.getUri());
+        for (Map.Entry<String, String> e : v.getParams().entrySet()) {
+            final String name = e.getKey();
+            jl.getParams().put(new QName("", name), e.getValue());
+        }
+        return jl;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkableEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkableEntity.java b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkableEntity.java
new file mode 100644
index 0000000..896e9d3
--- /dev/null
+++ b/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkableEntity.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.link;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.ws.rs.core.Link;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+/**
+ * Base classes for domain objects that want to provide a hypermedia link.
+ */
+@ApiModel("linkableEntity")
+public abstract class LinkableEntity {
+
+    private Link link;
+
+    @XmlElement
+    @XmlJavaTypeAdapter(LinkAdapter.class)
+    @ApiModelProperty(value = "An WebLink to this entity.", readOnly = true)
+    public Link getLink() {
+        return link;
+    }
+
+    public void setLink(Link link) {
+        this.link = link;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/pom.xml b/nifi-registry-framework/pom.xml
index 7ad7468..8b708d2 100644
--- a/nifi-registry-framework/pom.xml
+++ b/nifi-registry-framework/pom.xml
@@ -72,20 +72,17 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-properties</artifactId>
+            <artifactId>nifi-registry-utils</artifactId>
             <version>0.0.1-SNAPSHOT</version>
         </dependency>
         <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-beans</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-core</artifactId>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
         </dependency>
         <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-context</artifactId>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
         </dependency>
         <dependency>
             <groupId>org.hibernate</groupId>
@@ -95,8 +92,43 @@
             <groupId>org.glassfish</groupId>
             <artifactId>javax.el</artifactId>
         </dependency>
+        <!-- Database related dependencies, spring-boot-starter-data-jpa also brings in the standard Spring
+             dependencies for the backend to define beans -->
+        <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-data-jpa</artifactId>
+            <version>${spring.boot.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.tomcat</groupId>
+                    <artifactId>tomcat-jdbc</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.flywaydb</groupId>
+            <artifactId>flyway-core</artifactId>
+            <version>${flyway.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+            <version>1.4.196</version>
+        </dependency>
         <!-- Test Dependencies -->
         <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-test</artifactId>
+            <version>${spring.boot.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.flywaydb.flyway-test-extensions</groupId>
+            <artifactId>flyway-spring-test</artifactId>
+            <version>${flyway.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <version>4.12</version>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/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
new file mode 100644
index 0000000..5ec2043
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DataSourceFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.properties.NiFiRegistryProperties;
+import org.h2.jdbcx.JdbcConnectionPool;
+import org.springframework.beans.factory.annotation.Autowired;
+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
+ * from NiFiRegistryProperties rather than the standard application.properties/yaml.
+ */
+@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 final NiFiRegistryProperties properties;
+
+    private JdbcConnectionPool connectionPool;
+
+    @Autowired
+    public DataSourceFactory(final NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    @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);
+        }
+
+        return connectionPool;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java
new file mode 100644
index 0000000..35c6b55
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseMetadataService.java
@@ -0,0 +1,394 @@
+/*
+ * 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.Validate;
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntityKey;
+import org.apache.nifi.registry.db.repository.BucketItemRepository;
+import org.apache.nifi.registry.db.repository.BucketRepository;
+import org.apache.nifi.registry.db.repository.FlowRepository;
+import org.apache.nifi.registry.db.repository.FlowSnapshotRepository;
+import org.apache.nifi.registry.service.MetadataService;
+import org.apache.nifi.registry.service.params.QueryParameters;
+import org.apache.nifi.registry.service.params.SortOrder;
+import org.apache.nifi.registry.service.params.SortParameter;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.data.domain.PageRequest;
+import org.springframework.data.domain.Pageable;
+import org.springframework.data.domain.Sort;
+import org.springframework.stereotype.Service;
+
+import javax.persistence.EntityManager;
+import javax.persistence.metamodel.ManagedType;
+import javax.persistence.metamodel.Metamodel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * A MetadataProvider backed by the embedded relational database. All database access should occur through this class,
+ * other services should not directly use the repositories.
+ */
+@Service
+public class DatabaseMetadataService implements MetadataService {
+
+    private final BucketRepository bucketRepository;
+    private final FlowRepository flowRepository;
+    private final FlowSnapshotRepository flowSnapshotRepository;
+    private final BucketItemRepository itemRepository;
+    private final EntityManager entityManager;
+
+    private final Set<String> bucketFields;
+    private final Set<String> bucketItemFields;
+    private final Set<String> flowFields;
+
+    @Autowired
+    public DatabaseMetadataService(final BucketRepository bucketRepository,
+                                   final FlowRepository flowRepository,
+                                   final FlowSnapshotRepository flowSnapshotRepository,
+                                   final BucketItemRepository itemRepository,
+                                   final EntityManager entityManager) {
+        this.bucketRepository = bucketRepository;
+        this.flowRepository = flowRepository;
+        this.flowSnapshotRepository = flowSnapshotRepository;
+        this.itemRepository = itemRepository;
+        this.entityManager = entityManager;
+
+        Validate.notNull(this.bucketRepository);
+        Validate.notNull(this.flowRepository);
+        Validate.notNull(this.flowSnapshotRepository);
+        Validate.notNull(this.itemRepository);
+        Validate.notNull(this.entityManager);
+
+        this.bucketFields = Collections.unmodifiableSet(getEntityFields(BucketEntity.class));
+        this.bucketItemFields = Collections.unmodifiableSet(getEntityFields(BucketItemEntity.class));
+        this.flowFields = Collections.unmodifiableSet(getEntityFields(FlowEntity.class));
+    }
+
+    private Set<String> getEntityFields(Class<?> clazz) {
+        final Metamodel metamodel = entityManager.getMetamodel();
+        final ManagedType<?> bucketEntityManagedType = metamodel.managedType(clazz);
+        return bucketEntityManagedType.getSingularAttributes().stream().map(s -> s.getName()).collect(Collectors.toSet());
+    }
+
+    // ------------------------------------------------------------------------------------
+
+    @Override
+    public BucketEntity createBucket(final BucketEntity bucket) {
+        bucket.setCreated(new Date());
+        return bucketRepository.save(bucket);
+    }
+
+    @Override
+    public BucketEntity getBucketById(final String bucketIdentifier) {
+        return bucketRepository.findOne(bucketIdentifier);
+    }
+
+    @Override
+    public List<BucketEntity> getBucketsByName(final String name) {
+        List<BucketEntity> buckets = new ArrayList<>();
+
+        final Iterable<BucketEntity> retrievedBuckets = bucketRepository.findByNameIgnoreCase(name);
+        if (retrievedBuckets != null) {
+            for (BucketEntity bucket : retrievedBuckets) {
+                buckets.add(bucket);
+            }
+        }
+
+        return buckets;
+    }
+
+    @Override
+    public BucketEntity updateBucket(final BucketEntity bucket) {
+        return bucketRepository.save(bucket);
+    }
+
+    @Override
+    public void deleteBucket(final BucketEntity bucketEntity) {
+        bucketRepository.delete(bucketEntity);
+    }
+
+    @Override
+    public List<BucketEntity> getBuckets(final QueryParameters params) {
+        if (params.getNumRows() != null && params.getPageNum() != null) {
+            return getPagedBuckets(params);
+        } else if (params.getSortParameters() != null && params.getSortParameters().size() > 0) {
+            return getSortedBuckets(params);
+        } else {
+            return getAllBuckets();
+        }
+    }
+
+    private List<BucketEntity> getAllBuckets() {
+        final List<BucketEntity> buckets = new ArrayList<>();
+        for (BucketEntity bucket : bucketRepository.findAll()) {
+            buckets.add(bucket);
+        }
+        return buckets;
+    }
+
+    private List<BucketEntity> getPagedBuckets(final QueryParameters params) {
+        final Pageable pageable = getPageRequest(params);
+        final List<BucketEntity> buckets = new ArrayList<>();
+        for (BucketEntity bucket : bucketRepository.findAll(pageable)) {
+            buckets.add(bucket);
+        }
+        return buckets;
+    }
+
+    private List<BucketEntity> getSortedBuckets(final QueryParameters params) {
+        final Sort sort = getSort(params);
+        final List<BucketEntity> buckets = new ArrayList<>();
+        for (BucketEntity bucket : bucketRepository.findAll(sort)) {
+            buckets.add(bucket);
+        }
+        return buckets;
+    }
+
+    // ------------------------------------------------------------------------------------
+
+    @Override
+    public List<BucketItemEntity> getBucketItems(final QueryParameters params) {
+        if (params.getNumRows() != null && params.getPageNum() != null) {
+            return getPagedBucketItems(params);
+        } else if (params.getSortParameters() != null && params.getSortParameters().size() > 0) {
+            return getSortedBucketItems(params);
+        } else {
+            return getAllBucketItems();
+        }
+    }
+
+    private List<BucketItemEntity> getAllBucketItems() {
+        final List<BucketItemEntity> bucketItems = new ArrayList<>();
+        for (BucketItemEntity item : itemRepository.findAll()) {
+            bucketItems.add(item);
+        }
+        return bucketItems;
+    }
+
+    private List<BucketItemEntity> getPagedBucketItems(final QueryParameters params) {
+        final Pageable pageable = getPageRequest(params);
+        final List<BucketItemEntity> items = new ArrayList<>();
+        for (BucketItemEntity item : itemRepository.findAll(pageable)) {
+            items.add(item);
+        }
+        return items;
+    }
+
+    private List<BucketItemEntity> getSortedBucketItems(final QueryParameters params) {
+        final Sort sort = getSort(params);
+        final List<BucketItemEntity> items = new ArrayList<>();
+        for (BucketItemEntity item : itemRepository.findAll(sort)) {
+            items.add(item);
+        }
+        return items;
+    }
+
+    // ------------------------------------------------------------------------------------
+
+    @Override
+    public List<BucketItemEntity> getBucketItems(final QueryParameters params, final BucketEntity bucket) {
+        if (params.getNumRows() != null && params.getPageNum() != null) {
+            return getPagedBucketItems(params, bucket);
+        } else if (params.getSortParameters() != null && params.getSortParameters().size() > 0) {
+            return getSortedBucketItems(params, bucket);
+        } else {
+            return getBucketItems(bucket);
+        }
+    }
+
+    private List<BucketItemEntity> getBucketItems(final BucketEntity bucket) {
+        final List<BucketItemEntity> bucketItems = new ArrayList<>();
+        for (BucketItemEntity item : itemRepository.findByBucket(bucket)) {
+            bucketItems.add(item);
+        }
+        return bucketItems;
+    }
+
+    private List<BucketItemEntity> getPagedBucketItems(final QueryParameters params, final BucketEntity bucket) {
+        final Pageable pageable = getPageRequest(params);
+        final List<BucketItemEntity> items = new ArrayList<>();
+        for (BucketItemEntity item : itemRepository.findByBucket(bucket, pageable)) {
+            items.add(item);
+        }
+        return items;
+    }
+
+    private List<BucketItemEntity> getSortedBucketItems(final QueryParameters params, final BucketEntity bucket) {
+        final Sort sort = getSort(params);
+        final List<BucketItemEntity> items = new ArrayList<>();
+        for (BucketItemEntity item : itemRepository.findByBucket(bucket, sort)) {
+            items.add(item);
+        }
+        return items;
+    }
+
+    // ------------------------------------------------------------------------------------
+
+    @Override
+    public FlowEntity createFlow(final FlowEntity flow) {
+        flow.setCreated(new Date());
+        flow.setModified(new Date());
+        return flowRepository.save(flow);
+    }
+
+    @Override
+    public FlowEntity getFlowById(final String flowIdentifier) {
+        return flowRepository.findOne(flowIdentifier);
+    }
+
+    @Override
+    public List<FlowEntity> getFlowsByName(final String name) {
+        List<FlowEntity> flows = new ArrayList<>();
+
+        final Iterable<FlowEntity> retrievedFlows = flowRepository.findByNameIgnoreCase(name);
+        if (retrievedFlows != null) {
+            for (FlowEntity flow : retrievedFlows) {
+                flows.add(flow);
+            }
+        }
+
+        return flows;
+    }
+
+    @Override
+    public FlowEntity updateFlow(final FlowEntity flow) {
+        flow.setModified(new Date());
+        return flowRepository.save(flow);
+    }
+
+    @Override
+    public void deleteFlow(final FlowEntity flow) {
+        flowRepository.delete(flow);
+    }
+
+    @Override
+    public List<FlowEntity> getFlows(final QueryParameters params) {
+        if (params.getNumRows() != null && params.getPageNum() != null) {
+            return getPagedFlows(params);
+        } else if (params.getSortParameters() != null && params.getSortParameters().size() > 0) {
+            return getSortedFlows(params);
+        } else {
+            return getAllFlows();
+        }
+    }
+
+    private List<FlowEntity> getAllFlows() {
+        final List<FlowEntity> flows = new ArrayList<>();
+        for (FlowEntity flowEntity : flowRepository.findAll()) {
+            flows.add(flowEntity);
+        }
+        return flows;
+    }
+
+    private List<FlowEntity> getPagedFlows(final QueryParameters params) {
+        final Pageable pageable = getPageRequest(params);
+        final List<FlowEntity> flows = new ArrayList<>();
+        for (FlowEntity flowEntity : flowRepository.findAll(pageable)) {
+            flows.add(flowEntity);
+        }
+        return flows;
+    }
+
+    private List<FlowEntity> getSortedFlows(final QueryParameters params) {
+        final Sort sort = getSort(params);
+        final List<FlowEntity> flows = new ArrayList<>();
+        for (FlowEntity flowEntity : flowRepository.findAll(sort)) {
+            flows.add(flowEntity);
+        }
+        return flows;
+    }
+
+    // ------------------------------------------------------------------------------------
+
+    @Override
+    public FlowSnapshotEntity createFlowSnapshot(final FlowSnapshotEntity flowSnapshot) {
+        flowSnapshot.setCreated(new Date());
+        return flowSnapshotRepository.save(flowSnapshot);
+    }
+
+    @Override
+    public FlowSnapshotEntity getFlowSnapshot(final String flowIdentifier, final Integer version) {
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey(flowIdentifier, version);
+        return flowSnapshotRepository.findOne(key);
+    }
+
+    @Override
+    public void deleteFlowSnapshot(final FlowSnapshotEntity flowSnapshot) {
+        flowSnapshotRepository.delete(flowSnapshot);
+    }
+
+    @Override
+    public Set<String> getBucketFields() {
+        return bucketFields;
+    }
+
+    @Override
+    public Set<String> getBucketItemFields() {
+        return bucketItemFields;
+    }
+
+    @Override
+    public Set<String> getFlowFields() {
+        return flowFields;
+    }
+
+    /**
+     * Converts the registry query parameters to Spring Data's PageRequest.
+     *
+     * @param parameters the registry query parameters
+     * @return the equivalent Pageable
+     */
+    private Pageable getPageRequest(final QueryParameters parameters) {
+        final Sort sort = getSort(parameters);
+        if (sort == null) {
+            return new PageRequest(parameters.getPageNum(), parameters.getNumRows());
+        } else {
+            return new PageRequest(parameters.getPageNum(), parameters.getNumRows(), sort);
+        }
+    }
+
+    /**
+     * Converts the registry sort parameters to Spring Data's Sort.
+     *
+     * @param parameters the registry query parameters
+     * @return the equivalent Sort
+     */
+    private Sort getSort(final QueryParameters parameters) {
+        final List<Sort.Order> orders = new ArrayList<>();
+
+        for (SortParameter sortParameter : parameters.getSortParameters()) {
+            final Sort.Direction direction = sortParameter.getOrder() == SortOrder.ASC ? Sort.Direction.ASC : Sort.Direction.DESC;
+            final Sort.Order order = new Sort.Order(direction, sortParameter.getFieldName());
+            orders.add(order);
+        }
+
+        if (orders.isEmpty()) {
+            return null;
+        } else {
+            return new Sort(orders);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketEntity.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketEntity.java
new file mode 100644
index 0000000..539dc65
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketEntity.java
@@ -0,0 +1,123 @@
+/*
+ * 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.entity;
+
+import javax.persistence.CascadeType;
+import javax.persistence.Entity;
+import javax.persistence.Id;
+import javax.persistence.OneToMany;
+import javax.persistence.Table;
+import java.util.Date;
+import java.util.LinkedHashSet;
+import java.util.Objects;
+import java.util.Set;
+
+@Entity
+@Table(name = "BUCKET")
+public class BucketEntity {
+
+    @Id
+    private String id;
+
+    private String name;
+
+    private String description;
+
+    private Date created;
+
+    @OneToMany(
+            mappedBy = "bucket",
+            cascade = CascadeType.ALL,
+            orphanRemoval = true
+    )
+    private Set<BucketItemEntity> items;
+
+
+    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;
+    }
+
+    public Set<BucketItemEntity> getItems() {
+        return items;
+    }
+
+    public void setItems(Set<BucketItemEntity> items) {
+        this.items = items;
+    }
+
+    /**
+     * Convenience method to obtain all FlowEntity objects from the items set.
+     *
+     * @return a set of FlowEntities
+     */
+    public Set<FlowEntity> getFlows() {
+        final Set<FlowEntity> flows = new LinkedHashSet<>();
+        for (final BucketItemEntity itemEntity : getItems()) {
+            if (BucketItemEntityType.FLOW == itemEntity.getType()) {
+                flows.add((FlowEntity) itemEntity);
+            }
+        }
+        return flows;
+    }
+
+    @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 BucketEntity other = (BucketEntity) obj;
+        return Objects.equals(this.id, other.id);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntity.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntity.java
new file mode 100644
index 0000000..7bdf4cf
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntity.java
@@ -0,0 +1,135 @@
+/*
+ * 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.entity;
+
+import javax.persistence.Column;
+import javax.persistence.DiscriminatorColumn;
+import javax.persistence.Entity;
+import javax.persistence.EnumType;
+import javax.persistence.Enumerated;
+import javax.persistence.FetchType;
+import javax.persistence.Id;
+import javax.persistence.Inheritance;
+import javax.persistence.InheritanceType;
+import javax.persistence.JoinColumn;
+import javax.persistence.ManyToOne;
+import javax.persistence.Table;
+import java.util.Date;
+import java.util.Objects;
+
+@Entity
+@Table(name = "BUCKET_ITEM")
+@Inheritance(strategy = InheritanceType.JOINED)
+@DiscriminatorColumn(name = "ITEM_TYPE")
+public class BucketItemEntity {
+
+    @Id
+    private String id;
+
+    private String name;
+
+    private String description;
+
+    private Date created;
+
+    private Date modified;
+
+    // NOTE: sub-classes should ensure that the type is set appropriately by overriding the getter/setter
+
+    @Column(name = "ITEM_TYPE", nullable = false, insertable = false, updatable = false)
+    @Enumerated(EnumType.STRING)
+    private BucketItemEntityType type;
+
+    @ManyToOne(fetch = FetchType.LAZY)
+    @JoinColumn(name = "BUCKET_ID")
+    private BucketEntity bucket;
+
+
+    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 BucketItemEntityType getType() {
+        return type;
+    }
+
+    public void setType(BucketItemEntityType type) {
+        this.type = type;
+    }
+
+    public BucketEntity getBucket() {
+        return bucket;
+    }
+
+    public void setBucket(BucketEntity bucket) {
+        this.bucket = bucket;
+    }
+
+    @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 BucketItemEntity other = (BucketItemEntity) obj;
+        return Objects.equals(this.id, other.id);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntityType.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntityType.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntityType.java
new file mode 100644
index 0000000..e78b2b1
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/BucketItemEntityType.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.db.entity;
+
+/**
+ * Possible types of BucketItemEntity.
+ */
+public enum BucketItemEntityType {
+
+    FLOW(Values.FLOW);
+
+    private final String value;
+
+    BucketItemEntityType(final String value) {
+        this.value = value;
+    }
+
+    @Override
+    public String toString() {
+        return value;
+    }
+
+    // need these constants to reference from @DiscriminatorValue
+    public static class Values {
+        public static final String FLOW = "FLOW";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowEntity.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowEntity.java
new file mode 100644
index 0000000..84d2ab4
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowEntity.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.db.entity;
+
+import javax.persistence.CascadeType;
+import javax.persistence.DiscriminatorValue;
+import javax.persistence.Entity;
+import javax.persistence.OneToMany;
+import javax.persistence.Table;
+import java.util.Set;
+
+@Entity
+@Table(name = "FLOW")
+@DiscriminatorValue(value = BucketItemEntityType.Values.FLOW)
+public class FlowEntity extends BucketItemEntity {
+
+    @OneToMany(
+            mappedBy = "flow",
+            cascade = CascadeType.ALL,
+            orphanRemoval = true
+    )
+    private Set<FlowSnapshotEntity> snapshots;
+
+    public FlowEntity() {
+        setType(BucketItemEntityType.FLOW);
+    }
+
+    public Set<FlowSnapshotEntity> getSnapshots() {
+        return snapshots;
+    }
+
+    public void setSnapshots(Set<FlowSnapshotEntity> snapshots) {
+        this.snapshots = snapshots;
+    }
+
+    @Override
+    public void setType(BucketItemEntityType type) {
+        if (BucketItemEntityType.FLOW != type) {
+            throw new IllegalStateException("Must set type to FLOW");
+        }
+        super.setType(type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntity.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntity.java
new file mode 100644
index 0000000..de30845
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntity.java
@@ -0,0 +1,93 @@
+/*
+ * 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.entity;
+
+import javax.persistence.EmbeddedId;
+import javax.persistence.Entity;
+import javax.persistence.FetchType;
+import javax.persistence.JoinColumn;
+import javax.persistence.ManyToOne;
+import javax.persistence.Table;
+import java.util.Date;
+import java.util.Objects;
+
+@Entity
+@Table(name = "FLOW_SNAPSHOT")
+public class FlowSnapshotEntity {
+
+    @EmbeddedId
+    private FlowSnapshotEntityKey id;
+
+    private Date created;
+
+    private String comments;
+
+    @ManyToOne(fetch = FetchType.LAZY)
+    @JoinColumn(name = "FLOW_ID", insertable = false, updatable = false)
+    private FlowEntity flow;
+
+    public FlowSnapshotEntityKey getId() {
+        return id;
+    }
+
+    public void setId(FlowSnapshotEntityKey id) {
+        this.id = id;
+    }
+
+    public Date getCreated() {
+        return created;
+    }
+
+    public void setCreated(Date created) {
+        this.created = created;
+    }
+
+    public String getComments() {
+        return comments;
+    }
+
+    public void setComments(String comments) {
+        this.comments = comments;
+    }
+
+    public FlowEntity getFlow() {
+        return flow;
+    }
+
+    public void setFlow(FlowEntity flow) {
+        this.flow = flow;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(this.id);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+
+        if (!(obj instanceof FlowSnapshotEntity)) {
+            return false;
+        }
+
+        final FlowSnapshotEntity other = (FlowSnapshotEntity) obj;
+        return Objects.equals(this.id, other.id);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntityKey.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntityKey.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntityKey.java
new file mode 100644
index 0000000..679932c
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/FlowSnapshotEntityKey.java
@@ -0,0 +1,82 @@
+/*
+ * 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.entity;
+
+import javax.persistence.Column;
+import javax.persistence.Embeddable;
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Composite Key for FlowSnapshotEntity made up of the flow id and the snapshot version.
+ */
+@Embeddable
+public class FlowSnapshotEntityKey implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    @Column(name = "FLOW_ID")
+    private String flowId;
+
+    @Column(name = "VERSION")
+    private Integer version;
+
+    public FlowSnapshotEntityKey() {
+
+    }
+
+    public FlowSnapshotEntityKey(String flowId, Integer version) {
+        this.flowId = flowId;
+        this.version = version;
+    }
+
+    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;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(flowId, version);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+
+        if (!(obj instanceof FlowSnapshotEntityKey)) {
+            return false;
+        }
+
+        final FlowSnapshotEntityKey other = (FlowSnapshotEntityKey) 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/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketItemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketItemRepository.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketItemRepository.java
new file mode 100644
index 0000000..85d98ad
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketItemRepository.java
@@ -0,0 +1,84 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.springframework.data.domain.Page;
+import org.springframework.data.domain.Pageable;
+import org.springframework.data.domain.Sort;
+import org.springframework.data.repository.Repository;
+
+import java.util.List;
+
+/**
+ * Repository for BucketItems that exposes only the methods from PagingAndSortingRepository.
+ *
+ * There should be no CRUD methods performed directly on BucketItems, only general retrieval.
+ */
+public interface BucketItemRepository extends Repository<BucketItemEntity,String> {
+
+    /**
+     * Returns all instances of the type.
+     *
+     * @return all entities
+     */
+    Iterable<BucketItemEntity> findAll();
+
+    /**
+     * Returns all entities sorted by the given options.
+     *
+     * @param sort the sort params
+     * @return all entities sorted by the given options
+     */
+    Iterable<BucketItemEntity> findAll(Sort sort);
+
+    /**
+     * Returns a {@link Page} of entities meeting the paging restriction provided in the {@code Pageable} object.
+     *
+     * @param pageable the pageable params
+     * @return a page of entities
+     */
+    Page<BucketItemEntity> findAll(Pageable pageable);
+
+    /**
+     * Find all items by bucket.
+     *
+     * @param bucket the bucket to find items for
+     * @return the list of items for the bucket
+     */
+    List<BucketItemEntity> findByBucket(BucketEntity bucket);
+
+    /**
+     * Find all items by bucket with sorting.
+     *
+     * @param bucket the bucket to find items for
+     * @param sort the sort params
+     * @return the list of items for the bucket
+     */
+    List<BucketItemEntity> findByBucket(BucketEntity bucket, Sort sort);
+
+    /**
+     * Find all items by bucket with paging/sorting.
+     *
+     * @param bucket the bucket to find items for
+     * @param pageable the pageable params
+     * @return the list of items for the bucket based on the pageable params
+     */
+    List<BucketItemEntity> findByBucket(BucketEntity bucket, Pageable pageable);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketRepository.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketRepository.java
new file mode 100644
index 0000000..e917c2d
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/BucketRepository.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.db.repository;
+
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.springframework.data.repository.PagingAndSortingRepository;
+
+import java.util.List;
+
+/**
+ * Spring Data Repository for BucketEntity.
+ */
+public interface BucketRepository extends PagingAndSortingRepository<BucketEntity,String> {
+
+    List<BucketEntity> findByNameIgnoreCase(String name);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowRepository.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowRepository.java
new file mode 100644
index 0000000..ac20f81
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowRepository.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.db.repository;
+
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.springframework.data.repository.PagingAndSortingRepository;
+
+import java.util.List;
+
+/**
+ * Spring Data Repository for FlowEntity.
+ */
+public interface FlowRepository extends PagingAndSortingRepository<FlowEntity,String> {
+
+    List<FlowEntity> findByNameIgnoreCase(String name);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowSnapshotRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowSnapshotRepository.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowSnapshotRepository.java
new file mode 100644
index 0000000..1b3917d
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/FlowSnapshotRepository.java
@@ -0,0 +1,28 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntityKey;
+import org.springframework.data.repository.PagingAndSortingRepository;
+
+/**
+ * Repository for FlowSnapshotEntity.
+ */
+public interface FlowSnapshotRepository extends PagingAndSortingRepository<FlowSnapshotEntity, FlowSnapshotEntityKey> {
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/flow/StandardFlowSnapshotContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/flow/StandardFlowSnapshotContext.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/flow/StandardFlowSnapshotContext.java
deleted file mode 100644
index 3527355..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/flow/StandardFlowSnapshotContext.java
+++ /dev/null
@@ -1,155 +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.flow;
-
-import org.apache.commons.lang3.Validate;
-import org.apache.nifi.registry.bucket.Bucket;
-
-/**
- * Standard implementation of FlowSnapshotContext.
- */
-public class StandardFlowSnapshotContext implements FlowSnapshotContext {
-
-    private final String bucketId;
-    private final String bucketName;
-    private final String flowId;
-    private final String flowName;
-    private final int version;
-    private final String comments;
-    private final long snapshotTimestamp;
-
-    private StandardFlowSnapshotContext(final Builder builder) {
-        this.bucketId = builder.bucketId;
-        this.bucketName = builder.bucketName;
-        this.flowId = builder.flowId;
-        this.flowName = builder.flowName;
-        this.version = builder.version;
-        this.comments = builder.comments;
-        this.snapshotTimestamp = builder.snapshotTimestamp;
-
-        Validate.notBlank(bucketId);
-        Validate.notBlank(bucketName);
-        Validate.notBlank(flowId);
-        Validate.notBlank(flowName);
-        Validate.isTrue(version > 0);
-        Validate.isTrue(snapshotTimestamp > 0);
-    }
-
-    @Override
-    public String getBucketId() {
-        return bucketId;
-    }
-
-    @Override
-    public String getBucketName() {
-        return bucketName;
-    }
-
-    @Override
-    public String getFlowId() {
-        return flowId;
-    }
-
-    @Override
-    public String getFlowName() {
-        return flowName;
-    }
-
-    @Override
-    public int getVersion() {
-        return version;
-    }
-
-    @Override
-    public String getComments() {
-        return comments;
-    }
-
-    @Override
-    public long getSnapshotTimestamp() {
-        return snapshotTimestamp;
-    }
-
-    /**
-     * Builder for creating instances of StandardFlowSnapshotContext.
-     */
-    public static class Builder {
-
-        private String bucketId;
-        private String bucketName;
-        private String flowId;
-        private String flowName;
-        private int version;
-        private String comments;
-        private long snapshotTimestamp;
-
-        public Builder() {
-
-        }
-
-        public Builder(final Bucket bucket, final VersionedFlowSnapshotMetadata snapshotMetadata) {
-            bucketId(bucket.getIdentifier());
-            bucketName(bucket.getName());
-            flowId(snapshotMetadata.getFlowIdentifier());
-            flowName(snapshotMetadata.getFlowName());
-            version(snapshotMetadata.getVersion());
-            comments(snapshotMetadata.getComments());
-            snapshotTimestamp(snapshotMetadata.getTimestamp());
-        }
-
-        public Builder bucketId(final String bucketId) {
-            this.bucketId = bucketId;
-            return this;
-        }
-
-        public Builder bucketName(final String bucketName) {
-            this.bucketName = bucketName;
-            return this;
-        }
-
-        public Builder flowId(final String flowId) {
-            this.flowId = flowId;
-            return this;
-        }
-
-        public Builder flowName(final String flowName) {
-            this.flowName = flowName;
-            return this;
-        }
-
-        public Builder version(final int version) {
-            this.version = version;
-            return this;
-        }
-
-        public Builder comments(final String comments) {
-            this.comments = comments;
-            return this;
-        }
-
-        public Builder snapshotTimestamp(final long snapshotTimestamp) {
-            this.snapshotTimestamp = snapshotTimestamp;
-            return this;
-        }
-
-        public StandardFlowSnapshotContext build() {
-            return new StandardFlowSnapshotContext(this);
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/ProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/ProviderFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/ProviderFactory.java
index 0e6229c..d5b40a2 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/ProviderFactory.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/ProviderFactory.java
@@ -17,7 +17,6 @@
 package org.apache.nifi.registry.provider;
 
 import org.apache.nifi.registry.flow.FlowPersistenceProvider;
-import org.apache.nifi.registry.metadata.MetadataProvider;
 
 /**
  * A factory for obtaining the configured providers.
@@ -32,11 +31,6 @@ public interface ProviderFactory {
     void initialize() throws ProviderFactoryException;
 
     /**
-     * @return the configured MetadataProvider
-     */
-    MetadataProvider getMetadataProvider();
-
-    /**
      * @return the configured FlowPersistenceProvider
      */
     FlowPersistenceProvider getFlowPersistenceProvider();

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/StandardProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/StandardProviderFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/StandardProviderFactory.java
index 918ab79..01d8a8a 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/StandardProviderFactory.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/StandardProviderFactory.java
@@ -17,7 +17,6 @@
 package org.apache.nifi.registry.provider;
 
 import org.apache.nifi.registry.flow.FlowPersistenceProvider;
-import org.apache.nifi.registry.metadata.MetadataProvider;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
 import org.apache.nifi.registry.provider.generated.Property;
 import org.apache.nifi.registry.provider.generated.Providers;
@@ -71,9 +70,9 @@ public class StandardProviderFactory implements ProviderFactory {
     private final AtomicReference<Providers> providersHolder = new AtomicReference<>(null);
 
     private FlowPersistenceProvider flowPersistenceProvider;
-    private MetadataProvider metadataProvider;
 
-    public StandardProviderFactory(@Autowired final NiFiRegistryProperties properties) {
+    @Autowired
+    public StandardProviderFactory(final NiFiRegistryProperties properties) {
         this.properties = properties;
 
         if (this.properties == null) {
@@ -110,39 +109,6 @@ public class StandardProviderFactory implements ProviderFactory {
 
     @Bean
     @Override
-    public synchronized MetadataProvider getMetadataProvider() {
-        if (metadataProvider == null) {
-            if (providersHolder.get() == null) {
-                throw new ProviderFactoryException("ProviderFactory must be initialized before obtaining a Provider");
-            }
-
-            final Providers providers = providersHolder.get();
-            final org.apache.nifi.registry.provider.generated.Provider jaxbMetadataProvider = providers.getMetadataProvider();
-            final String metadataProviderClassName = jaxbMetadataProvider.getClazz();
-
-            try {
-                final Class<?> rawMetadataProviderClass = Class.forName(metadataProviderClassName, true, StandardProviderFactory.class.getClassLoader());
-                final Class<? extends MetadataProvider> metadataProviderClass = rawMetadataProviderClass.asSubclass(MetadataProvider.class);
-
-                // otherwise create a new instance
-                final Constructor constructor = metadataProviderClass.getConstructor();
-                metadataProvider = (MetadataProvider) constructor.newInstance();
-
-                LOGGER.info("Instantiated MetadataProvider with class name {}", new Object[] {metadataProviderClassName});
-            } catch (Exception e) {
-                throw new ProviderFactoryException("Error creating MetadataProvider with class name: " + metadataProviderClassName, e);
-            }
-
-            final ProviderConfigurationContext configurationContext = createConfigurationContext(jaxbMetadataProvider.getProperty());
-            metadataProvider.onConfigured(configurationContext);
-            LOGGER.info("Configured MetadataProvider with class name {}", new Object[] {metadataProviderClassName});
-        }
-
-        return metadataProvider;
-    }
-
-    @Bean
-    @Override
     public synchronized FlowPersistenceProvider getFlowPersistenceProvider() {
         if (flowPersistenceProvider == null) {
             if (providersHolder.get() == null) {


[4/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
index 8762706..d4b42cf 100644
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
@@ -17,21 +17,20 @@
 package org.apache.nifi.registry.service;
 
 import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.db.entity.BucketEntity;
+import org.apache.nifi.registry.db.entity.BucketItemEntity;
+import org.apache.nifi.registry.db.entity.FlowEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
+import org.apache.nifi.registry.db.entity.FlowSnapshotEntityKey;
 import org.apache.nifi.registry.exception.ResourceNotFoundException;
 import org.apache.nifi.registry.flow.FlowPersistenceProvider;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
-import org.apache.nifi.registry.metadata.BucketMetadata;
-import org.apache.nifi.registry.metadata.FlowMetadata;
-import org.apache.nifi.registry.metadata.FlowSnapshotMetadata;
-import org.apache.nifi.registry.metadata.MetadataProvider;
-import org.apache.nifi.registry.metadata.StandardBucketMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowMetadata;
-import org.apache.nifi.registry.metadata.StandardFlowSnapshotMetadata;
 import org.apache.nifi.registry.serialization.FlowSnapshotSerializer;
 import org.apache.nifi.registry.serialization.Serializer;
+import org.apache.nifi.registry.service.params.QueryParameters;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
@@ -43,8 +42,12 @@ import javax.validation.Validator;
 import javax.validation.ValidatorFactory;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
@@ -60,7 +63,7 @@ import static org.mockito.Mockito.when;
 
 public class TestRegistryService {
 
-    private MetadataProvider metadataProvider;
+    private MetadataService metadataService;
     private FlowPersistenceProvider flowPersistenceProvider;
     private Serializer<VersionedFlowSnapshot> snapshotSerializer;
     private Validator validator;
@@ -69,14 +72,14 @@ public class TestRegistryService {
 
     @Before
     public void setup() {
-        metadataProvider = mock(MetadataProvider.class);
+        metadataService = mock(MetadataService.class);
         flowPersistenceProvider = mock(FlowPersistenceProvider.class);
         snapshotSerializer = mock(FlowSnapshotSerializer.class);
 
         final ValidatorFactory validatorFactory = Validation.buildDefaultValidatorFactory();
         validator = validatorFactory.getValidator();
 
-        registryService = new RegistryService(metadataProvider, flowPersistenceProvider, snapshotSerializer, validator);
+        registryService = new RegistryService(metadataService, flowPersistenceProvider, snapshotSerializer, validator);
     }
 
     // ---------------------- Test Bucket methods ---------------------------------------------
@@ -87,9 +90,9 @@ public class TestRegistryService {
         bucket.setName("My Bucket");
         bucket.setDescription("This is my bucket.");
 
-        when(metadataProvider.getBucketByName(bucket.getName())).thenReturn(null);
+        when(metadataService.getBucketsByName(bucket.getName())).thenReturn(Collections.emptyList());
 
-        doAnswer(createBucketAnswer()).when(metadataProvider).createBucket(any(BucketMetadata.class));
+        doAnswer(createBucketAnswer()).when(metadataService).createBucket(any(BucketEntity.class));
 
         final Bucket createdBucket = registryService.createBucket(bucket);
         assertNotNull(createdBucket);
@@ -106,13 +109,12 @@ public class TestRegistryService {
         bucket.setName("My Bucket");
         bucket.setDescription("This is my bucket.");
 
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketByName(bucket.getName())).thenReturn(existingBucket);
+        when(metadataService.getBucketsByName(bucket.getName())).thenReturn(Collections.singletonList(existingBucket));
 
         // should throw exception since a bucket with the same name exists
         registryService.createBucket(bucket);
@@ -121,33 +123,32 @@ public class TestRegistryService {
     @Test(expected = ConstraintViolationException.class)
     public void testCreateBucketWithMissingName() {
         final Bucket bucket = new Bucket();
-        when(metadataProvider.getBucketByName(bucket.getName())).thenReturn(null);
+        when(metadataService.getBucketsByName(bucket.getName())).thenReturn(Collections.emptyList());
         registryService.createBucket(bucket);
     }
 
     @Test
     public void testGetExistingBucket() {
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
-        final Bucket bucket = registryService.getBucket(existingBucket.getIdentifier());
+        final Bucket bucket = registryService.getBucket(existingBucket.getId(), true);
         assertNotNull(bucket);
-        assertEquals(existingBucket.getIdentifier(), bucket.getIdentifier());
+        assertEquals(existingBucket.getId(), bucket.getIdentifier());
         assertEquals(existingBucket.getName(), bucket.getName());
         assertEquals(existingBucket.getDescription(), bucket.getDescription());
-        assertEquals(existingBucket.getCreatedTimestamp(), bucket.getCreatedTimestamp());
+        assertEquals(existingBucket.getCreated().getTime(), bucket.getCreatedTimestamp());
     }
 
     @Test(expected = ResourceNotFoundException.class)
     public void testGetBucketDoesNotExist() {
-        when(metadataProvider.getBucketById(any(String.class))).thenReturn(null);
-        registryService.getBucket("does-not-exist");
+        when(metadataService.getBucketById(any(String.class))).thenReturn(null);
+        registryService.getBucket("does-not-exist", true);
     }
 
     @Test(expected = IllegalArgumentException.class)
@@ -166,33 +167,31 @@ public class TestRegistryService {
         bucket.setDescription("This is my bucket.");
         registryService.updateBucket(bucket);
 
-        when(metadataProvider.getBucketById(any(String.class))).thenReturn(null);
+        when(metadataService.getBucketById(any(String.class))).thenReturn(null);
         registryService.updateBucket(bucket);
     }
 
     @Test(expected = IllegalStateException.class)
     public void testUpdateBucketWithSameNameAsExistingBucket() {
-        final BucketMetadata bucketToUpdate = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity bucketToUpdate = new BucketEntity();
+        bucketToUpdate.setId("b1");
+        bucketToUpdate.setName("My Bucket");
+        bucketToUpdate.setDescription("This is my bucket");
+        bucketToUpdate.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(bucketToUpdate.getIdentifier())).thenReturn(bucketToUpdate);
+        when(metadataService.getBucketById(bucketToUpdate.getId())).thenReturn(bucketToUpdate);
 
-        final BucketMetadata otherBucket = new StandardBucketMetadata.Builder()
-                .identifier("b2")
-                .name("My Bucket #2")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity otherBucket = new BucketEntity();
+        otherBucket.setId("b2");
+        otherBucket.setName("My Bucket #2");
+        otherBucket.setDescription("This is my bucket");
+        otherBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketByName(otherBucket.getName())).thenReturn(otherBucket);
+        when(metadataService.getBucketsByName(otherBucket.getName())).thenReturn(Collections.singletonList(otherBucket));
 
         // should fail because other bucket has the same name
         final Bucket updatedBucket = new Bucket();
-        updatedBucket.setIdentifier(bucketToUpdate.getIdentifier());
+        updatedBucket.setIdentifier(bucketToUpdate.getId());
         updatedBucket.setName("My Bucket #2");
         updatedBucket.setDescription(bucketToUpdate.getDescription());
 
@@ -201,19 +200,18 @@ public class TestRegistryService {
 
     @Test
     public void testUpdateBucket() {
-        final BucketMetadata bucketToUpdate = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity bucketToUpdate = new BucketEntity();
+        bucketToUpdate.setId("b1");
+        bucketToUpdate.setName("My Bucket");
+        bucketToUpdate.setDescription("This is my bucket");
+        bucketToUpdate.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(bucketToUpdate.getIdentifier())).thenReturn(bucketToUpdate);
+        when(metadataService.getBucketById(bucketToUpdate.getId())).thenReturn(bucketToUpdate);
 
-        doAnswer(updateBucketAnswer()).when(metadataProvider).updateBucket(any(BucketMetadata.class));
+        doAnswer(updateBucketAnswer()).when(metadataService).updateBucket(any(BucketEntity.class));
 
         final Bucket updatedBucket = new Bucket();
-        updatedBucket.setIdentifier(bucketToUpdate.getIdentifier());
+        updatedBucket.setIdentifier(bucketToUpdate.getId());
         updatedBucket.setName("Updated Name");
         updatedBucket.setDescription("Updated Description");
 
@@ -225,19 +223,18 @@ public class TestRegistryService {
 
     @Test
     public void testUpdateBucketPartial() {
-        final BucketMetadata bucketToUpdate = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity bucketToUpdate = new BucketEntity();
+        bucketToUpdate.setId("b1");
+        bucketToUpdate.setName("My Bucket");
+        bucketToUpdate.setDescription("This is my bucket");
+        bucketToUpdate.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(bucketToUpdate.getIdentifier())).thenReturn(bucketToUpdate);
+        when(metadataService.getBucketById(bucketToUpdate.getId())).thenReturn(bucketToUpdate);
 
-        doAnswer(updateBucketAnswer()).when(metadataProvider).updateBucket(any(BucketMetadata.class));
+        doAnswer(updateBucketAnswer()).when(metadataService).updateBucket(any(BucketEntity.class));
 
         final Bucket updatedBucket = new Bucket();
-        updatedBucket.setIdentifier(bucketToUpdate.getIdentifier());
+        updatedBucket.setIdentifier(bucketToUpdate.getId());
         updatedBucket.setName("Updated Name");
         updatedBucket.setDescription(null);
 
@@ -251,39 +248,36 @@ public class TestRegistryService {
     @Test(expected = ResourceNotFoundException.class)
     public void testDeleteBucketDoesNotExist() {
         final String bucketId = "b1";
-        when(metadataProvider.getBucketById(bucketId)).thenReturn(null);
+        when(metadataService.getBucketById(bucketId)).thenReturn(null);
         registryService.deleteBucket(bucketId);
     }
 
     @Test
     public void testDeleteBucketWithFlows() {
-        final BucketMetadata bucketToDelete = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity bucketToDelete = new BucketEntity();
+        bucketToDelete.setId("b1");
+        bucketToDelete.setName("My Bucket");
+        bucketToDelete.setDescription("This is my bucket");
+        bucketToDelete.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(bucketToDelete.getIdentifier())).thenReturn(bucketToDelete);
+        final FlowEntity flowToDelete = new FlowEntity();
+        flowToDelete.setId("flow1");
+        flowToDelete.setName("Flow 1");
+        flowToDelete.setDescription("This is flow 1");
+        flowToDelete.setCreated(new Date());
 
-        final FlowMetadata flowToDelete = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("Flow 1")
-                .description("This is flow 1")
-                .created(System.currentTimeMillis())
-                .build();
+        final Set<BucketItemEntity> items = new HashSet<>();
+        items.add(flowToDelete);
+        bucketToDelete.setItems(items);
 
-        final Set<FlowMetadata> flowsToDelete = new HashSet<>();
-        flowsToDelete.add(flowToDelete);
+        when(metadataService.getBucketById(bucketToDelete.getId())).thenReturn(bucketToDelete);
 
-        when(metadataProvider.getFlows(bucketToDelete.getIdentifier())).thenReturn(flowsToDelete);
-
-        final Bucket deletedBucket = registryService.deleteBucket(bucketToDelete.getIdentifier());
+        final Bucket deletedBucket = registryService.deleteBucket(bucketToDelete.getId());
         assertNotNull(deletedBucket);
-        assertEquals(bucketToDelete.getIdentifier(), deletedBucket.getIdentifier());
+        assertEquals(bucketToDelete.getId(), deletedBucket.getIdentifier());
 
         verify(flowPersistenceProvider, times(1))
-                .deleteSnapshots(eq(bucketToDelete.getIdentifier()), eq(flowToDelete.getIdentifier()));
+                .deleteSnapshots(eq(bucketToDelete.getId()), eq(flowToDelete.getId()));
     }
 
     // ---------------------- Test VersionedFlow methods ---------------------------------------------
@@ -297,7 +291,7 @@ public class TestRegistryService {
     @Test(expected = ResourceNotFoundException.class)
     public void testCreateFlowBucketDoesNotExist() {
 
-        when(metadataProvider.getBucketById(any(String.class))).thenReturn(null);
+        when(metadataService.getBucketById(any(String.class))).thenReturn(null);
 
         final VersionedFlow versionedFlow = new VersionedFlow();
         versionedFlow.setName("My Flow");
@@ -308,30 +302,27 @@ public class TestRegistryService {
 
     @Test(expected = IllegalStateException.class)
     public void testCreateFlowWithSameName() {
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
         // setup a flow with the same name that already exists
 
-        final FlowMetadata flowMetadata = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
+        final FlowEntity flowWithSameName = new FlowEntity();
+        flowWithSameName.setId("flow1");
+        flowWithSameName.setName("Flow 1");
+        flowWithSameName.setDescription("This is flow 1");
+        flowWithSameName.setCreated(new Date());
+        flowWithSameName.setModified(new Date());
 
-        when(metadataProvider.getFlowByName(flowMetadata.getName())).thenReturn(flowMetadata);
+        when(metadataService.getFlowsByName(flowWithSameName.getName())).thenReturn(Collections.singletonList(flowWithSameName));
 
         final VersionedFlow versionedFlow = new VersionedFlow();
-        versionedFlow.setName(flowMetadata.getName());
+        versionedFlow.setName(flowWithSameName.getName());
         versionedFlow.setBucketIdentifier("b1");
 
         registryService.createFlow(versionedFlow.getBucketIdentifier(), versionedFlow);
@@ -339,20 +330,19 @@ public class TestRegistryService {
 
     @Test
     public void testCreateFlowValid() {
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
         final VersionedFlow versionedFlow = new VersionedFlow();
         versionedFlow.setName("My Flow");
         versionedFlow.setBucketIdentifier("b1");
 
-        doAnswer(createFlowAnswer()).when(metadataProvider).createFlow(any(String.class), any(FlowMetadata.class));
+        doAnswer(createFlowAnswer()).when(metadataService).createFlow(any(FlowEntity.class));
 
         final VersionedFlow createdFlow = registryService.createFlow(versionedFlow.getBucketIdentifier(), versionedFlow);
         assertNotNull(createdFlow);
@@ -366,106 +356,112 @@ public class TestRegistryService {
 
     @Test(expected = ResourceNotFoundException.class)
     public void testGetFlowDoesNotExist() {
-        when(metadataProvider.getFlowById(any(String.class))).thenReturn(null);
-        registryService.getFlow("flow1");
+        when(metadataService.getFlowById(any(String.class))).thenReturn(null);
+        registryService.getFlow("flow1", false);
     }
 
     @Test
     public void testGetFlowExists() {
-        final FlowMetadata flowMetadata = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowById(flowMetadata.getIdentifier())).thenReturn(flowMetadata);
-
-        final VersionedFlow versionedFlow = registryService.getFlow(flowMetadata.getIdentifier());
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity flowEntity = new FlowEntity();
+        flowEntity.setId("flow1");
+        flowEntity.setName("My Flow");
+        flowEntity.setDescription("This is my flow.");
+        flowEntity.setCreated(new Date());
+        flowEntity.setModified(new Date());
+        flowEntity.setBucket(existingBucket);
+
+        when(metadataService.getFlowById(flowEntity.getId())).thenReturn(flowEntity);
+
+        final VersionedFlow versionedFlow = registryService.getFlow(flowEntity.getId(), false);
         assertNotNull(versionedFlow);
-        assertEquals(flowMetadata.getIdentifier(), versionedFlow.getIdentifier());
-        assertEquals(flowMetadata.getName(), versionedFlow.getName());
-        assertEquals(flowMetadata.getDescription(), versionedFlow.getDescription());
-        assertEquals(flowMetadata.getBucketIdentifier(), versionedFlow.getBucketIdentifier());
-        assertEquals(flowMetadata.getCreatedTimestamp(), versionedFlow.getCreatedTimestamp());
-        assertEquals(flowMetadata.getModifiedTimestamp(), versionedFlow.getModifiedTimestamp());
+        assertEquals(flowEntity.getId(), versionedFlow.getIdentifier());
+        assertEquals(flowEntity.getName(), versionedFlow.getName());
+        assertEquals(flowEntity.getDescription(), versionedFlow.getDescription());
+        assertEquals(flowEntity.getBucket().getId(), versionedFlow.getBucketIdentifier());
+        assertEquals(flowEntity.getCreated().getTime(), versionedFlow.getCreatedTimestamp());
+        assertEquals(flowEntity.getModified().getTime(), versionedFlow.getModifiedTimestamp());
     }
 
     @Test
     public void testGetFlows() {
-        final FlowMetadata flowMetadata1 = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        final FlowMetadata flowMetadata2 = new StandardFlowMetadata.Builder()
-                .identifier("flow2")
-                .name("My Flow")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        final Set<FlowMetadata> flows = new LinkedHashSet<>();
-        flows.add(flowMetadata1);
-        flows.add(flowMetadata2);
-
-        when(metadataProvider.getFlows()).thenReturn(flows);
-
-        final Set<VersionedFlow> allFlows = registryService.getFlows();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity flowEntity1 = new FlowEntity();
+        flowEntity1.setId("flow1");
+        flowEntity1.setName("My Flow");
+        flowEntity1.setDescription("This is my flow.");
+        flowEntity1.setCreated(new Date());
+        flowEntity1.setModified(new Date());
+        flowEntity1.setBucket(existingBucket);
+
+        final FlowEntity flowEntity2 = new FlowEntity();
+        flowEntity2.setId("flow2");
+        flowEntity2.setName("My Flow 2");
+        flowEntity2.setDescription("This is my flow 2.");
+        flowEntity2.setCreated(new Date());
+        flowEntity2.setModified(new Date());
+        flowEntity2.setBucket(existingBucket);
+
+        final List<FlowEntity> flows = new ArrayList<>();
+        flows.add(flowEntity1);
+        flows.add(flowEntity2);
+
+        when(metadataService.getFlows(any(QueryParameters.class))).thenReturn(flows);
+
+        final QueryParameters queryParameters = new QueryParameters.Builder().build();
+        final List<VersionedFlow> allFlows = registryService.getFlows(queryParameters);
         assertNotNull(allFlows);
         assertEquals(2, allFlows.size());
     }
 
     @Test(expected = ResourceNotFoundException.class)
     public void testGetFlowsByBucketDoesNotExist() {
-        when(metadataProvider.getBucketById(any(String.class))).thenReturn(null);
+        when(metadataService.getBucketById(any(String.class))).thenReturn(null);
         registryService.getFlows("b1");
     }
 
     @Test
     public void testGetFlowsByBucketExists() {
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier("b1")
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
-
-        final FlowMetadata flowMetadata1 = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        final FlowMetadata flowMetadata2 = new StandardFlowMetadata.Builder()
-                .identifier("flow2")
-                .name("My Flow")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        final Set<FlowMetadata> flows = new LinkedHashSet<>();
-        flows.add(flowMetadata1);
-        flows.add(flowMetadata2);
-
-        when(metadataProvider.getFlows(existingBucket.getIdentifier())).thenReturn(flows);
-
-        final Set<VersionedFlow> allFlows = registryService.getFlows(existingBucket.getIdentifier());
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity flowEntity1 = new FlowEntity();
+        flowEntity1.setId("flow1");
+        flowEntity1.setName("My Flow");
+        flowEntity1.setDescription("This is my flow.");
+        flowEntity1.setCreated(new Date());
+        flowEntity1.setModified(new Date());
+        flowEntity1.setBucket(existingBucket);
+
+        final FlowEntity flowEntity2 = new FlowEntity();
+        flowEntity2.setId("flow2");
+        flowEntity2.setName("My Flow 2");
+        flowEntity2.setDescription("This is my flow 2.");
+        flowEntity2.setCreated(new Date());
+        flowEntity2.setModified(new Date());
+        flowEntity2.setBucket(existingBucket);
+
+        final Set<BucketItemEntity> flows = new LinkedHashSet<>();
+        flows.add(flowEntity1);
+        flows.add(flowEntity2);
+        existingBucket.setItems(flows);
+
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
+
+        final List<VersionedFlow> allFlows = registryService.getFlows(existingBucket.getId());
         assertNotNull(allFlows);
         assertEquals(2, allFlows.size());
     }
@@ -481,37 +477,41 @@ public class TestRegistryService {
         final VersionedFlow versionedFlow = new VersionedFlow();
         versionedFlow.setIdentifier("flow1");
 
-        when(metadataProvider.getFlowById(versionedFlow.getIdentifier())).thenReturn(null);
+        when(metadataService.getFlowById(versionedFlow.getIdentifier())).thenReturn(null);
 
         registryService.updateFlow(versionedFlow);
     }
 
     @Test(expected = IllegalStateException.class)
     public void testUpdateFlowWithSameNameAsExistingFlow() {
-        final FlowMetadata flowToUpdate = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowById(flowToUpdate.getIdentifier())).thenReturn(flowToUpdate);
-
-        final FlowMetadata otherFlow = new StandardFlowMetadata.Builder()
-                .identifier("flow2")
-                .name("My Flow 2")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowByName(otherFlow.getName())).thenReturn(otherFlow);
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity flowToUpdate = new FlowEntity();
+        flowToUpdate.setId("flow1");
+        flowToUpdate.setName("My Flow");
+        flowToUpdate.setDescription("This is my flow.");
+        flowToUpdate.setCreated(new Date());
+        flowToUpdate.setModified(new Date());
+        flowToUpdate.setBucket(existingBucket);
+
+        when(metadataService.getFlowById(flowToUpdate.getId())).thenReturn(flowToUpdate);
+
+        final FlowEntity otherFlow = new FlowEntity();
+        otherFlow.setId("flow2");
+        otherFlow.setName("My Flow 2");
+        otherFlow.setDescription("This is my flow 2.");
+        otherFlow.setCreated(new Date());
+        otherFlow.setModified(new Date());
+        otherFlow.setBucket(existingBucket);
+
+        when(metadataService.getFlowsByName(otherFlow.getName())).thenReturn(Collections.singletonList(otherFlow));
 
         final VersionedFlow versionedFlow = new VersionedFlow();
-        versionedFlow.setIdentifier(flowToUpdate.getIdentifier());
+        versionedFlow.setIdentifier(flowToUpdate.getId());
         versionedFlow.setName(otherFlow.getName());
 
         registryService.updateFlow(versionedFlow);
@@ -519,22 +519,27 @@ public class TestRegistryService {
 
     @Test
     public void testUpdateFlow() throws InterruptedException {
-        final FlowMetadata flowToUpdate = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity flowToUpdate = new FlowEntity();
+        flowToUpdate.setId("flow1");
+        flowToUpdate.setName("My Flow");
+        flowToUpdate.setDescription("This is my flow.");
+        flowToUpdate.setCreated(new Date());
+        flowToUpdate.setModified(new Date());
+        flowToUpdate.setBucket(existingBucket);
 
-        when(metadataProvider.getFlowById(flowToUpdate.getIdentifier())).thenReturn(flowToUpdate);
-        when(metadataProvider.getFlowByName(flowToUpdate.getName())).thenReturn(flowToUpdate);
+        when(metadataService.getFlowById(flowToUpdate.getId())).thenReturn(flowToUpdate);
+        when(metadataService.getFlowsByName(flowToUpdate.getName())).thenReturn(Collections.singletonList(flowToUpdate));
 
-        doAnswer(updateFlowAnswer()).when(metadataProvider).updateFlow(any(FlowMetadata.class));
+        doAnswer(updateFlowAnswer()).when(metadataService).updateFlow(any(FlowEntity.class));
 
         final VersionedFlow versionedFlow = new VersionedFlow();
-        versionedFlow.setIdentifier(flowToUpdate.getIdentifier());
+        versionedFlow.setIdentifier(flowToUpdate.getId());
         versionedFlow.setName("New Flow Name");
         versionedFlow.setDescription("This is a new description");
 
@@ -549,42 +554,43 @@ public class TestRegistryService {
         assertEquals(versionedFlow.getDescription(), updatedFlow.getDescription());
 
         // other fields should not be updated
-        assertEquals(flowToUpdate.getBucketIdentifier(), updatedFlow.getBucketIdentifier());
-        assertEquals(flowToUpdate.getCreatedTimestamp(), updatedFlow.getCreatedTimestamp());
-
-        // modified timestamp should be auto updated
-        assertTrue(updatedFlow.getModifiedTimestamp() > flowToUpdate.getModifiedTimestamp());
+        assertEquals(flowToUpdate.getBucket().getId(), updatedFlow.getBucketIdentifier());
+        assertEquals(flowToUpdate.getCreated().getTime(), updatedFlow.getCreatedTimestamp());
     }
 
     @Test(expected = ResourceNotFoundException.class)
     public void testDeleteFlowDoesNotExist() {
-        when(metadataProvider.getFlowById(any(String.class))).thenReturn(null);
+        when(metadataService.getFlowById(any(String.class))).thenReturn(null);
         registryService.deleteFlow("flow1");
     }
 
     @Test
     public void testDeleteFlowWithSnapshots() {
-        final FlowMetadata flowToDelete = new StandardFlowMetadata.Builder()
-                .identifier("flow1")
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier("b1")
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowById(flowToDelete.getIdentifier())).thenReturn(flowToDelete);
-        when(metadataProvider.getFlowByName(flowToDelete.getName())).thenReturn(flowToDelete);
-
-        final VersionedFlow deletedFlow = registryService.deleteFlow(flowToDelete.getIdentifier());
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity flowToDelete = new FlowEntity();
+        flowToDelete.setId("flow1");
+        flowToDelete.setName("My Flow");
+        flowToDelete.setDescription("This is my flow.");
+        flowToDelete.setCreated(new Date());
+        flowToDelete.setModified(new Date());
+        flowToDelete.setBucket(existingBucket);
+
+        when(metadataService.getFlowById(flowToDelete.getId())).thenReturn(flowToDelete);
+        when(metadataService.getFlowsByName(flowToDelete.getName())).thenReturn(Collections.singletonList(flowToDelete));
+
+        final VersionedFlow deletedFlow = registryService.deleteFlow(flowToDelete.getId());
         assertNotNull(deletedFlow);
-        assertEquals(flowToDelete.getIdentifier(), deletedFlow.getIdentifier());
+        assertEquals(flowToDelete.getId(), deletedFlow.getIdentifier());
 
         verify(flowPersistenceProvider, times(1))
-                .deleteSnapshots(flowToDelete.getBucketIdentifier(), flowToDelete.getIdentifier());
+                .deleteSnapshots(flowToDelete.getBucket().getId(), flowToDelete.getId());
 
-        verify(metadataProvider, times(1))
-                .deleteFlow(flowToDelete.getIdentifier());
+        verify(metadataService, times(1)).deleteFlow(flowToDelete);
     }
 
     // ---------------------- Test VersionedFlowSnapshot methods ---------------------------------------------
@@ -638,7 +644,7 @@ public class TestRegistryService {
 
     @Test(expected = ResourceNotFoundException.class)
     public void testCreateSnapshotBucketDoesNotExist() {
-        when(metadataProvider.getBucketById(any(String.class))).thenReturn(null);
+        when(metadataService.getBucketById(any(String.class))).thenReturn(null);
 
         final VersionedFlowSnapshot snapshot = createSnapshot();
         registryService.createFlowSnapshot(snapshot);
@@ -648,16 +654,15 @@ public class TestRegistryService {
     public void testCreateSnapshotFlowDoesNotExist() {
         final VersionedFlowSnapshot snapshot = createSnapshot();
 
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
-        when(metadataProvider.getFlowById(snapshot.getSnapshotMetadata().getFlowIdentifier())).thenReturn(null);
+        when(metadataService.getFlowById(snapshot.getSnapshotMetadata().getFlowIdentifier())).thenReturn(null);
 
         registryService.createFlowSnapshot(snapshot);
     }
@@ -666,37 +671,37 @@ public class TestRegistryService {
     public void testCreateSnapshotVersionAlreadyExists() {
         final VersionedFlowSnapshot snapshot = createSnapshot();
 
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
+
+        // return a flow with the existing snapshot when getFlowById is called
+        final FlowEntity existingFlow = new FlowEntity();
+        existingFlow.setId("flow1");
+        existingFlow.setName("My Flow");
+        existingFlow.setDescription("This is my flow.");
+        existingFlow.setCreated(new Date());
+        existingFlow.setModified(new Date());
+        existingFlow.setBucket(existingBucket);
 
         // make a snapshot that has the same version as the one being created
-        final FlowSnapshotMetadata existingSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .flowIdentifier(snapshot.getSnapshotMetadata().getFlowIdentifier())
-                .flowName(snapshot.getSnapshotMetadata().getFlowName())
-                .bucketIdentifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .version(snapshot.getSnapshotMetadata().getVersion())
-                .comments("This is an existing snapshot")
-                .created(System.currentTimeMillis())
-                .build();
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId(snapshot.getSnapshotMetadata().getFlowIdentifier());
+        key.setVersion(snapshot.getSnapshotMetadata().getVersion());
 
-        // return a flow with the existing snapshot when getFlowById is called
-        final FlowMetadata existingFlow = new StandardFlowMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getFlowIdentifier())
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .addSnapshot(existingSnapshot)
-                .build();
-
-        when(metadataProvider.getFlowById(existingFlow.getIdentifier())).thenReturn(existingFlow);
+        final FlowSnapshotEntity existingSnapshot = new FlowSnapshotEntity();
+        existingSnapshot.setId(key);
+        existingSnapshot.setComments("This is an existing snapshot");
+        existingSnapshot.setCreated(new Date());
+        existingSnapshot.setFlow(existingFlow);
+
+        existingFlow.setSnapshots(Collections.singleton(existingSnapshot));
+
+        when(metadataService.getFlowById(existingFlow.getId())).thenReturn(existingFlow);
 
         registryService.createFlowSnapshot(snapshot);
     }
@@ -705,37 +710,37 @@ public class TestRegistryService {
     public void testCreateSnapshotVersionNotNextVersion() {
         final VersionedFlowSnapshot snapshot = createSnapshot();
 
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
-
-        // make a snapshot for version 1
-        final FlowSnapshotMetadata existingSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .flowIdentifier(snapshot.getSnapshotMetadata().getFlowIdentifier())
-                .flowName(snapshot.getSnapshotMetadata().getFlowName())
-                .bucketIdentifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .version(1)
-                .comments("This is an existing snapshot")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
         // return a flow with the existing snapshot when getFlowById is called
-        final FlowMetadata existingFlow = new StandardFlowMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getFlowIdentifier())
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .addSnapshot(existingSnapshot)
-                .build();
-
-        when(metadataProvider.getFlowById(existingFlow.getIdentifier())).thenReturn(existingFlow);
+        final FlowEntity existingFlow = new FlowEntity();
+        existingFlow.setId("flow1");
+        existingFlow.setName("My Flow");
+        existingFlow.setDescription("This is my flow.");
+        existingFlow.setCreated(new Date());
+        existingFlow.setModified(new Date());
+        existingFlow.setBucket(existingBucket);
+
+        // make a snapshot that has the same version as the one being created
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId(snapshot.getSnapshotMetadata().getFlowIdentifier());
+        key.setVersion(snapshot.getSnapshotMetadata().getVersion());
+
+        final FlowSnapshotEntity existingSnapshot = new FlowSnapshotEntity();
+        existingSnapshot.setId(key);
+        existingSnapshot.setComments("This is an existing snapshot");
+        existingSnapshot.setCreated(new Date());
+        existingSnapshot.setFlow(existingFlow);
+
+        existingFlow.setSnapshots(Collections.singleton(existingSnapshot));
+
+        when(metadataService.getFlowById(existingFlow.getId())).thenReturn(existingFlow);
 
         // set the version to something that is not the next one-up version
         snapshot.getSnapshotMetadata().setVersion(100);
@@ -746,59 +751,55 @@ public class TestRegistryService {
     public void testCreateFirstSnapshot() {
         final VersionedFlowSnapshot snapshot = createSnapshot();
 
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
         // return a flow with the existing snapshot when getFlowById is called
-        final FlowMetadata existingFlow = new StandardFlowMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getFlowIdentifier())
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
+        final FlowEntity existingFlow = new FlowEntity();
+        existingFlow.setId("flow1");
+        existingFlow.setName("My Flow");
+        existingFlow.setDescription("This is my flow.");
+        existingFlow.setCreated(new Date());
+        existingFlow.setModified(new Date());
+        existingFlow.setBucket(existingBucket);
 
-        when(metadataProvider.getFlowById(existingFlow.getIdentifier())).thenReturn(existingFlow);
+        when(metadataService.getFlowById(existingFlow.getId())).thenReturn(existingFlow);
 
         final VersionedFlowSnapshot createdSnapshot = registryService.createFlowSnapshot(snapshot);
         assertNotNull(createdSnapshot);
 
         verify(snapshotSerializer, times(1)).serialize(eq(snapshot), any(OutputStream.class));
         verify(flowPersistenceProvider, times(1)).saveSnapshot(any(), any());
-        verify(metadataProvider, times(1)).createFlowSnapshot(any(FlowSnapshotMetadata.class));
+        verify(metadataService, times(1)).createFlowSnapshot(any(FlowSnapshotEntity.class));
     }
 
     @Test(expected = IllegalStateException.class)
     public void testCreateFirstSnapshotWithBadVersion() {
         final VersionedFlowSnapshot snapshot = createSnapshot();
 
-        final BucketMetadata existingBucket = new StandardBucketMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .name("My Bucket #1")
-                .description("This is my bucket.")
-                .created(System.currentTimeMillis())
-                .build();
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
 
-        when(metadataProvider.getBucketById(existingBucket.getIdentifier())).thenReturn(existingBucket);
+        when(metadataService.getBucketById(existingBucket.getId())).thenReturn(existingBucket);
 
         // return a flow with the existing snapshot when getFlowById is called
-        final FlowMetadata existingFlow = new StandardFlowMetadata.Builder()
-                .identifier(snapshot.getSnapshotMetadata().getFlowIdentifier())
-                .name("My Flow 1")
-                .description("This is my flow.")
-                .bucketIdentifier(snapshot.getSnapshotMetadata().getBucketIdentifier())
-                .created(System.currentTimeMillis())
-                .modified(System.currentTimeMillis())
-                .build();
+        final FlowEntity existingFlow = new FlowEntity();
+        existingFlow.setId("flow1");
+        existingFlow.setName("My Flow");
+        existingFlow.setDescription("This is my flow.");
+        existingFlow.setCreated(new Date());
+        existingFlow.setModified(new Date());
+        existingFlow.setBucket(existingBucket);
 
-        when(metadataProvider.getFlowById(existingFlow.getIdentifier())).thenReturn(existingFlow);
+        when(metadataService.getFlowById(existingFlow.getId())).thenReturn(existingFlow);
 
         // set the first version to something other than 1
         snapshot.getSnapshotMetadata().setVersion(100);
@@ -809,59 +810,46 @@ public class TestRegistryService {
     public void testGetSnapshotDoesNotExistInMetadataProvider() {
         final String flowId = "flow1";
         final Integer version = 1;
-        when(metadataProvider.getFlowSnapshot(flowId, version)).thenReturn(null);
+        when(metadataService.getFlowSnapshot(flowId, version)).thenReturn(null);
         registryService.getFlowSnapshot(flowId, version);
     }
 
     @Test(expected = IllegalStateException.class)
     public void testGetSnapshotDoesNotExistInPersistenceProvider() {
-        final FlowSnapshotMetadata existingSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .bucketIdentifier("b1")
-                .flowIdentifier("flow1")
-                .flowName("Flow 1")
-                .version(1)
-                .comments("This is snapshot 1")
-                .created(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowSnapshot(existingSnapshot.getFlowIdentifier(), existingSnapshot.getVersion()))
-                .thenReturn(existingSnapshot);
+        final FlowSnapshotEntity existingSnapshot = createFlowSnapshotEntity();
+        final FlowSnapshotEntityKey key = existingSnapshot.getId();
+
+        when(metadataService.getFlowSnapshot(key.getFlowId(), key.getVersion())).thenReturn(existingSnapshot);
 
         when(flowPersistenceProvider.getSnapshot(
-                existingSnapshot.getBucketIdentifier(),
-                existingSnapshot.getFlowIdentifier(),
-                existingSnapshot.getVersion()
+                existingSnapshot.getFlow().getBucket().getId(),
+                existingSnapshot.getFlow().getId(),
+                existingSnapshot.getId().getVersion()
         )).thenReturn(null);
 
-        registryService.getFlowSnapshot(existingSnapshot.getFlowIdentifier(), existingSnapshot.getVersion());
+        registryService.getFlowSnapshot(existingSnapshot.getFlow().getId(), existingSnapshot.getId().getVersion());
     }
 
     @Test
     public void testGetSnapshotExists() {
-        final FlowSnapshotMetadata existingSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .bucketIdentifier("b1")
-                .flowIdentifier("flow1")
-                .flowName("Flow 1")
-                .version(1)
-                .comments("This is snapshot 1")
-                .created(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowSnapshot(existingSnapshot.getFlowIdentifier(), existingSnapshot.getVersion()))
+        final FlowSnapshotEntity existingSnapshot = createFlowSnapshotEntity();
+        final FlowSnapshotEntityKey key = existingSnapshot.getId();
+
+        when(metadataService.getFlowSnapshot(key.getFlowId(), key.getVersion()))
                 .thenReturn(existingSnapshot);
 
         // return a non-null, non-zero-length array so something gets passed to the serializer
         when(flowPersistenceProvider.getSnapshot(
-                existingSnapshot.getBucketIdentifier(),
-                existingSnapshot.getFlowIdentifier(),
-                existingSnapshot.getVersion()
+                existingSnapshot.getFlow().getBucket().getId(),
+                existingSnapshot.getFlow().getId(),
+                existingSnapshot.getId().getVersion()
         )).thenReturn(new byte[10]);
 
         final VersionedFlowSnapshot snapshotToDeserialize = createSnapshot();
         when(snapshotSerializer.deserialize(any(InputStream.class))).thenReturn(snapshotToDeserialize);
 
         final VersionedFlowSnapshot returnedSnapshot = registryService.getFlowSnapshot(
-                existingSnapshot.getFlowIdentifier(), existingSnapshot.getVersion());
+                existingSnapshot.getFlow().getId(), existingSnapshot.getId().getVersion());
         assertNotNull(returnedSnapshot);
     }
 
@@ -869,68 +857,86 @@ public class TestRegistryService {
     public void testDeleteSnapshotDoesNotExist() {
         final String flowId = "flow1";
         final Integer version = 1;
-        when(metadataProvider.getFlowSnapshot(flowId, version)).thenReturn(null);
+        when(metadataService.getFlowSnapshot(flowId, version)).thenReturn(null);
         registryService.deleteFlowSnapshot(flowId, version);
     }
 
     @Test
     public void testDeleteSnapshotExists() {
-        final FlowSnapshotMetadata existingSnapshot = new StandardFlowSnapshotMetadata.Builder()
-                .bucketIdentifier("b1")
-                .flowIdentifier("flow1")
-                .flowName("Flow 1")
-                .version(1)
-                .comments("This is snapshot 1")
-                .created(System.currentTimeMillis())
-                .build();
-
-        when(metadataProvider.getFlowSnapshot(existingSnapshot.getFlowIdentifier(), existingSnapshot.getVersion()))
+        final FlowSnapshotEntity existingSnapshot = createFlowSnapshotEntity();
+        final FlowSnapshotEntityKey key = existingSnapshot.getId();
+
+        when(metadataService.getFlowSnapshot(key.getFlowId(), key.getVersion()))
                 .thenReturn(existingSnapshot);
 
-        final VersionedFlowSnapshotMetadata deletedSnapshot = registryService.deleteFlowSnapshot(
-                existingSnapshot.getFlowIdentifier(), existingSnapshot.getVersion());
+        final VersionedFlowSnapshotMetadata deletedSnapshot = registryService.deleteFlowSnapshot(key.getFlowId(), key.getVersion());
         assertNotNull(deletedSnapshot);
-        assertEquals(existingSnapshot.getFlowIdentifier(), deletedSnapshot.getFlowIdentifier());
+        assertEquals(existingSnapshot.getId().getFlowId(), deletedSnapshot.getFlowIdentifier());
 
         verify(flowPersistenceProvider, times(1)).deleteSnapshot(
-                existingSnapshot.getBucketIdentifier(),
-                existingSnapshot.getFlowIdentifier(),
-                existingSnapshot.getVersion()
+                existingSnapshot.getFlow().getBucket().getId(),
+                existingSnapshot.getFlow().getId(),
+                existingSnapshot.getId().getVersion()
         );
 
-        verify(metadataProvider, times(1)).deleteFlowSnapshot(
-                existingSnapshot.getFlowIdentifier(),
-                existingSnapshot.getVersion()
-        );
+        verify(metadataService, times(1)).deleteFlowSnapshot(existingSnapshot);
+    }
+
+    private FlowSnapshotEntity createFlowSnapshotEntity() {
+        final BucketEntity existingBucket = new BucketEntity();
+        existingBucket.setId("b1");
+        existingBucket.setName("My Bucket");
+        existingBucket.setDescription("This is my bucket");
+        existingBucket.setCreated(new Date());
+
+        final FlowEntity existingFlow = new FlowEntity();
+        existingFlow.setId("flow1");
+        existingFlow.setName("My Flow");
+        existingFlow.setDescription("This is my flow.");
+        existingFlow.setCreated(new Date());
+        existingFlow.setModified(new Date());
+        existingFlow.setBucket(existingBucket);
+
+        final FlowSnapshotEntityKey key = new FlowSnapshotEntityKey();
+        key.setFlowId("flow1");
+        key.setVersion(1);
+
+        final FlowSnapshotEntity existingSnapshot = new FlowSnapshotEntity();
+        existingSnapshot.setId(key);
+        existingSnapshot.setComments("This is an existing snapshot");
+        existingSnapshot.setCreated(new Date());
+        existingSnapshot.setFlow(existingFlow);
+
+        return existingSnapshot;
     }
 
     // -------------------------------------------------------------------
 
-    private Answer<BucketMetadata> createBucketAnswer() {
+    private Answer<BucketEntity> createBucketAnswer() {
         return (InvocationOnMock invocation) -> {
-            BucketMetadata bucketMetadata = (BucketMetadata) invocation.getArguments()[0];
-            return bucketMetadata;
+            BucketEntity bucketEntity = (BucketEntity) invocation.getArguments()[0];
+            return bucketEntity;
         };
     }
 
-    private Answer<BucketMetadata> updateBucketAnswer() {
+    private Answer<BucketEntity> updateBucketAnswer() {
         return (InvocationOnMock invocation) -> {
-            BucketMetadata bucketMetadata = (BucketMetadata) invocation.getArguments()[0];
-            return bucketMetadata;
+            BucketEntity bucketEntity = (BucketEntity) invocation.getArguments()[0];
+            return bucketEntity;
         };
     }
 
-    private Answer<FlowMetadata> createFlowAnswer() {
+    private Answer<FlowEntity> createFlowAnswer() {
         return (InvocationOnMock invocation) -> {
-            final FlowMetadata flowMetadata = (FlowMetadata) invocation.getArguments()[1];
-            return flowMetadata;
+            final FlowEntity flowEntity = (FlowEntity) invocation.getArguments()[0];
+            return flowEntity;
         };
     }
 
-    private Answer<FlowMetadata> updateFlowAnswer() {
+    private Answer<FlowEntity> updateFlowAnswer() {
         return (InvocationOnMock invocation) -> {
-            final FlowMetadata flowMetadata = (FlowMetadata) invocation.getArguments()[0];
-            return flowMetadata;
+            final FlowEntity flowEntity = (FlowEntity) invocation.getArguments()[0];
+            return flowEntity;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/resources/application.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/resources/application.properties b/nifi-registry-framework/src/test/resources/application.properties
new file mode 100644
index 0000000..fb91fb6
--- /dev/null
+++ b/nifi-registry-framework/src/test/resources/application.properties
@@ -0,0 +1,19 @@
+# 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.
+
+# Properties for Spring Boot tests
+
+spring.jpa.hibernate.ddl-auto=validate
+spring.jpa.show-sql=true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql b/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
new file mode 100644
index 0000000..4352e50
--- /dev/null
+++ b/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
@@ -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.
+
+-- test data for buckets
+
+insert into bucket (id, name, description, created)
+  values ('1', 'Bucket 1', 'This is test bucket 1', parsedatetime('2017-09-11 12:51:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'));
+
+insert into bucket (id, name, description, created)
+  values ('2', 'Bucket 2', 'This is test bucket 2', parsedatetime('2017-09-11 12:52:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'));
+
+insert into bucket (id, name, description, created)
+  values ('3', 'Bucket 3', 'This is test bucket 3', parsedatetime('2017-09-11 12:53:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'));
+
+insert into bucket (id, name, description, created)
+  values ('4', 'Bucket 4', 'This is test bucket 4', parsedatetime('2017-09-11 12:54:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'));
+
+insert into bucket (id, name, description, created)
+  values ('5', 'Bucket 5', 'This is test bucket 5', parsedatetime('2017-09-11 12:55:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'));
+
+insert into bucket (id, name, description, created)
+  values ('6', 'Bucket 6', 'This is test bucket 6', parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'));
+
+
+-- test data for flows
+
+insert into bucket_item (id, name, description, created, modified, item_type, bucket_id)
+  values ('1', 'Flow 1', 'This is flow 1', parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), 'FLOW', '1');
+
+insert into flow (id) values ('1');
+
+insert into bucket_item (id, name, description, created, modified, item_type, bucket_id)
+  values ('2', 'Flow 2', 'This is flow 2', parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), 'FLOW', '1');
+
+insert into flow (id) values ('2');
+
+insert into bucket_item (id, name, description, created, modified, item_type, bucket_id)
+  values ('3', 'Flow 3', 'This is flow 3', parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), parsedatetime('2017-09-11 12:56:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), 'FLOW', '2');
+
+insert into flow (id) values ('3');
+
+
+-- test data for flow snapshots
+
+insert into flow_snapshot (flow_id, version, created, comments)
+  values ('1', 1, parsedatetime('2017-09-11 12:57:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), 'This is flow 1 snapshot 1');
+
+insert into flow_snapshot (flow_id, version, created, comments)
+  values ('1', 2, parsedatetime('2017-09-11 12:58:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), 'This is flow 1 snapshot 2');
+
+insert into flow_snapshot (flow_id, version, created, comments)
+  values ('1', 3, parsedatetime('2017-09-11 12:59:00.000', 'yyyy-MM-dd hh:mm:ss.SSS'), 'This is flow 1 snapshot 3');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/resources/provider/providers-class-not-found.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/resources/provider/providers-class-not-found.xml b/nifi-registry-framework/src/test/resources/provider/providers-class-not-found.xml
index 8b5debe..9adba54 100644
--- a/nifi-registry-framework/src/test/resources/provider/providers-class-not-found.xml
+++ b/nifi-registry-framework/src/test/resources/provider/providers-class-not-found.xml
@@ -15,12 +15,6 @@
 -->
 <providers>
 
-    <metadataProvider>
-        <class>org.apache.nifi.registry.provider.MetadataProviderXXX</class>
-        <property name="Metadata Property 1">foo</property>
-        <property name="Metadata Property 2">bar</property>
-    </metadataProvider>
-
     <flowPersistenceProvider>
         <class>org.apache.nifi.registry.provider.FlowProviderXXX</class>
         <property name="Flow Property 1">foo</property>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-framework/src/test/resources/provider/providers-good.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/resources/provider/providers-good.xml b/nifi-registry-framework/src/test/resources/provider/providers-good.xml
index 4ef2a06..32414e5 100644
--- a/nifi-registry-framework/src/test/resources/provider/providers-good.xml
+++ b/nifi-registry-framework/src/test/resources/provider/providers-good.xml
@@ -15,12 +15,6 @@
 -->
 <providers>
 
-    <metadataProvider>
-        <class>org.apache.nifi.registry.provider.MockMetadataProvider</class>
-        <property name="Metadata Property 1">metadata foo</property>
-        <property name="Metadata Property 2">metadata bar</property>
-    </metadataProvider>
-
     <flowPersistenceProvider>
         <class>org.apache.nifi.registry.provider.MockFlowPersistenceProvider</class>
         <property name="Flow Property 1">flow foo</property>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/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 35a1b48..6aac4a6 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
@@ -47,6 +47,9 @@ public class NiFiRegistryProperties extends Properties {
 
     public static final String PROVIDERS_CONFIGURATION_FILE = "nifi.registry.providers.configuration.file";
 
+    public static final String DATABASE_DIRECTORY = "nifi.registry.db.directory";
+    public static final String DATABASE_URL_APPEND = "nifi.registry.db.url.append";
+
     // Defaults
     public static final String DEFAULT_WEB_WORKING_DIR = "./work/jetty";
     public static final String DEFAULT_WAR_DIR = "./lib";
@@ -155,4 +158,13 @@ public class NiFiRegistryProperties extends Properties {
             return new File(value);
         }
     }
+
+    public String getDatabaseDirectory() {
+        return getProperty(DATABASE_DIRECTORY);
+    }
+
+    public String getDatabaseUrlAppend() {
+        return getProperty(DATABASE_URL_APPEND);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/BucketMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/BucketMetadata.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/BucketMetadata.java
deleted file mode 100644
index 7d10da5..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/BucketMetadata.java
+++ /dev/null
@@ -1,51 +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.metadata;
-
-import java.util.Set;
-
-/**
- * The metadata for a bucket, along with the metadata about any objects stored in the bucket, such as flows.
- */
-public interface BucketMetadata {
-
-    /**
-     * @return the identifier of this bucket
-     */
-    String getIdentifier();
-
-    /**
-     * @return the name of this bucket
-     */
-    String getName();
-
-    /**
-     * @return the timestamp of when this bucket was created
-     */
-    long getCreatedTimestamp();
-
-    /**
-     * @return the description of this bucket
-     */
-    String getDescription();
-
-    /**
-     * @return the metadata about the flows that are part of this bucket
-     */
-    Set<FlowMetadata> getFlowMetadata();
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowMetadata.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowMetadata.java
deleted file mode 100644
index 0aebb63..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowMetadata.java
+++ /dev/null
@@ -1,69 +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.metadata;
-
-import java.util.Set;
-
-/**
- * The metadata about a flow, including the metadata about any of it's snapshots.
- */
-public interface FlowMetadata {
-
-    /**
-     * @return the identifier of this flow
-     */
-    String getIdentifier();
-
-    /**
-     * @return the name of this flow
-     */
-    String getName();
-
-    /**
-     * @return the identifier of the bucket this flow belongs to
-     */
-    String getBucketIdentifier();
-
-    /**
-     * @return the timestamp this flow was created
-     */
-    long getCreatedTimestamp();
-
-    /**
-     * @return the timestamp this flow was modified
-     */
-    long getModifiedTimestamp();
-
-    /**
-     * @return the description of this flow
-     */
-    String getDescription();
-
-    /**
-     * @return the metadata for the snapshots of this flow
-     */
-    Set<FlowSnapshotMetadata> getSnapshotMetadata();
-
-    /**
-     * Get the snapshot for the given version.
-     *
-     * @param version the version of a snapshot
-     * @return the snapshot for the given version, or null if one doesn't exist
-     */
-    FlowSnapshotMetadata getSnapshot(int version);
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowSnapshotMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowSnapshotMetadata.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowSnapshotMetadata.java
deleted file mode 100644
index 98cb666..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/FlowSnapshotMetadata.java
+++ /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.
- */
-package org.apache.nifi.registry.metadata;
-
-/**
- * The metadata for a flow snapshot.
- */
-public interface FlowSnapshotMetadata {
-
-    /**
-     * @return the identifier of the bucket this snapshot belongs to
-     */
-    String getBucketIdentifier();
-
-    /**
-     * @return the identifier of the flow this snapshot belongs to
-     */
-    String getFlowIdentifier();
-
-    /**
-     * @return the name of the flow this snapshot belongs to
-     */
-    String getFlowName();
-
-    /**
-     * @return the version of this snapshot
-     */
-    int getVersion();
-
-    /**
-     * @return the timestamp of when this snapshot was created
-     */
-    long getCreatedTimestamp();
-
-    /**
-     * @return the comments for this snapshot
-     */
-    String getComments();
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProvider.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProvider.java
deleted file mode 100644
index 3ddcadc..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProvider.java
+++ /dev/null
@@ -1,158 +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.metadata;
-
-import org.apache.nifi.registry.provider.Provider;
-
-import java.util.Set;
-
-/**
- * A service for managing metadata about all objects stored by the registry.
- *
- * NOTE: Although this interface is intended to be an extension point, it is not yet considered stable and thus may
- * change across releases until the registry matures.
- */
-public interface MetadataProvider extends Provider {
-
-    /**
-     * Creates the given bucket.
-     *
-     * @param bucket the bucket to create
-     * @return the created bucket
-     */
-    BucketMetadata createBucket(BucketMetadata bucket);
-
-    /**
-     * Retrieves the bucket with the given id.
-     *
-     * @param bucketIdentifier the id of the bucket to retrieve
-     * @return the bucket with the given id, or null if it does not exist
-     */
-    BucketMetadata getBucketById(String bucketIdentifier);
-
-    /**
-     * Retrieves the bucket with the given name. The name comparison must be case-insensitive.
-     *
-     * @param name the name of the bucket to retrieve
-     * @return the bucket with the given name, or null if it does not exist
-     */
-    BucketMetadata getBucketByName(String name);
-    /**
-     * Updates the given bucket, only the name and description should be allowed to be updated.
-     *
-     * @param bucket the updated bucket to save
-     * @return the updated bucket, or null if no bucket with the given id exists
-     */
-    BucketMetadata updateBucket(BucketMetadata bucket);
-
-    /**
-     * Deletes the bucket with the given identifier, as well as any objects that reference the bucket.
-     *
-     * @param bucketIdentifier the id of the bucket to delete
-     */
-    void deleteBucket(String bucketIdentifier);
-
-    /**
-     * Retrieves all buckets known to this metadata provider.
-     *
-     * @return the set of all buckets
-     */
-    Set<BucketMetadata> getBuckets();
-
-    /**
-     * Creates a versioned flow in the given bucket.
-     *
-     * @param bucketIdentifier the id of the bucket where the flow is being created
-     * @param flow the versioned flow to create
-     * @return the created versioned flow
-     * @throws IllegalStateException if no bucket with the given identifier exists
-     */
-    FlowMetadata createFlow(String bucketIdentifier, FlowMetadata flow);
-
-    /**
-     * Retrieves the versioned flow with the given id.
-     *
-     * @param flowIdentifier the identifier of the flow to retrieve
-     * @return the versioned flow with the given id, or null if no flow with the given id exists
-     */
-    FlowMetadata getFlowById(String flowIdentifier);
-
-    /**
-     * Retrieves the versioned flow with the given name. The name comparison must be case-insensitive.
-     *
-     * @param name the name of the flow to retrieve
-     * @return the versioned flow with the given name, or null if no flow with the given name exists
-     */
-    FlowMetadata getFlowByName(String name);
-
-    /**
-     * Updates the given versioned flow, only the name and description should be allowed to be updated.
-     *
-     * @param versionedFlow the updated versioned flow to save
-     * @return the updated versioned flow
-     */
-    FlowMetadata updateFlow(FlowMetadata versionedFlow);
-
-    /**
-     * Deletes the versioned flow with the given identifier if one exists.
-     *
-     * @param flowIdentifier the id of the versioned flow to delete
-     */
-    void deleteFlow(String flowIdentifier);
-
-    /**
-     * Retrieves all versioned flows known to this metadata provider.
-     *
-     * @return the set of all versioned flows
-     */
-    Set<FlowMetadata> getFlows();
-
-    /**
-     * Retrieves all the versioned flows for the given bucket.
-     *
-     * @param bucketId the id of the bucket to retrieve flow for
-     * @return the set of versioned flows for the given bucket, or an empty set if none exist
-     */
-    Set<FlowMetadata> getFlows(String bucketId);
-
-    /**
-     * Creates a versioned flow snapshot.
-     *
-     * @param flowSnapshot the snapshot to create
-     * @return the created snapshot
-     * @throws IllegalStateException if the versioned flow specified by flowSnapshot.getFlowIdentifier() does not exist
-     */
-    FlowSnapshotMetadata createFlowSnapshot(FlowSnapshotMetadata flowSnapshot);
-
-    /**
-     * Retrieves the snapshot for the given flow identifier and snapshot version.
-     *
-     * @param flowIdentifier the identifier of the flow the snapshot belongs to
-     * @param version the version of the snapshot
-     * @return the versioned flow snapshot for the given flow identifier and version, or null if none exists
-     */
-    FlowSnapshotMetadata getFlowSnapshot(String flowIdentifier, Integer version);
-
-    /**
-     * Deletes the snapshot for the given flow identifier and version.
-     *
-     * @param flowIdentifier the identifier of the flow the snapshot belongs to
-     * @param version the version of the snapshot
-     */
-    void deleteFlowSnapshot(String flowIdentifier, Integer version);
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProviderException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProviderException.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProviderException.java
deleted file mode 100644
index bce9352..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/MetadataProviderException.java
+++ /dev/null
@@ -1,35 +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.metadata;
-
-/**
- * An exception thrown when an error is encountered by a MetadataProvider.
- */
-public class MetadataProviderException extends RuntimeException {
-
-    public MetadataProviderException(String message) {
-        super(message);
-    }
-
-    public MetadataProviderException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public MetadataProviderException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardBucketMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardBucketMetadata.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardBucketMetadata.java
deleted file mode 100644
index 02568e6..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardBucketMetadata.java
+++ /dev/null
@@ -1,131 +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.metadata;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashSet;
-import java.util.Set;
-
-public class StandardBucketMetadata implements BucketMetadata {
-
-    private final String identifier;
-    private final String name;
-    private final long createdTimestamp;
-    private final String description;
-    private final Set<FlowMetadata> flowMetadata;
-
-    private StandardBucketMetadata(final Builder builder) {
-        this.identifier = builder.identifier;
-        this.name = builder.name;
-        this.createdTimestamp = builder.createdTimestamp;
-        this.description = builder.description;
-        this.flowMetadata = Collections.unmodifiableSet(
-                builder.flowMetadata == null
-                        ? Collections.emptySet() : new LinkedHashSet<>(builder.flowMetadata)
-        );
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public String getName() {
-        return name;
-    }
-
-    @Override
-    public long getCreatedTimestamp() {
-        return createdTimestamp;
-    }
-
-    @Override
-    public String getDescription() {
-        return description;
-    }
-
-    @Override
-    public Set<FlowMetadata> getFlowMetadata() {
-        return flowMetadata;
-    }
-
-    public static class Builder {
-
-        private String identifier;
-        private String name;
-        private long createdTimestamp;
-        private String description;
-        private Set<FlowMetadata> flowMetadata = new LinkedHashSet<>();
-
-        public Builder() {
-
-        }
-
-        public Builder(BucketMetadata bucketMetadata) {
-            identifier(bucketMetadata.getIdentifier());
-            name(bucketMetadata.getName());
-            created(bucketMetadata.getCreatedTimestamp());
-            description(bucketMetadata.getDescription());
-            addFlows(bucketMetadata.getFlowMetadata());
-        }
-
-        public Builder identifier(final String identifier) {
-            this.identifier = identifier;
-            return this;
-        }
-
-        public Builder name(final String name) {
-            this.name = name;
-            return this;
-        }
-
-        public Builder created(final long createdTimestamp) {
-            this.createdTimestamp = createdTimestamp;
-            return this;
-        }
-
-        public Builder description(final String description) {
-            this.description = description;
-            return this;
-        }
-
-        public Builder addFlow(final FlowMetadata flowMetadata) {
-            if (flowMetadata != null) {
-                this.flowMetadata.add(flowMetadata);
-            }
-            return this;
-        }
-
-        public Builder addFlows(final Collection<FlowMetadata> flows) {
-            if (flows != null) {
-                this.flowMetadata.addAll(flows);
-            }
-            return this;
-        }
-
-        public Builder clearFlows() {
-            this.flowMetadata.clear();
-            return this;
-        }
-
-        public StandardBucketMetadata build() {
-            return new StandardBucketMetadata(this);
-        }
-    }
-}


[3/7] nifi-registry git commit: NIFIREG-18 Initial plumbling for H2 database - Setup Flyway with initial migration to define tables - Setup entity classes with repositories - Setup unit testing for repositories - Removed existing MetadataProvider concept

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowMetadata.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowMetadata.java
deleted file mode 100644
index cfef423..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowMetadata.java
+++ /dev/null
@@ -1,174 +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.metadata;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Standard immutable implementation of FlowMetadata.
- */
-public class StandardFlowMetadata implements FlowMetadata {
-
-    private final String identifier;
-    private final String name;
-    private final String bucketIdentifier;
-    private final long createdTimestamp;
-    private final long modifiedTimestamp;
-    private final String description;
-    private final Set<FlowSnapshotMetadata> snapshotMetadata;
-    private final Map<String,FlowSnapshotMetadata> snapshotMetadataByVersion;
-
-    private StandardFlowMetadata(final Builder builder) {
-        this.identifier = builder.identifier;
-        this.name = builder.name;
-        this.bucketIdentifier = builder.bucketIdentifier;
-        this.createdTimestamp = builder.createdTimestamp;
-        this.modifiedTimestamp = builder.modifiedTimestamp;
-        this.description = builder.description;
-        this.snapshotMetadata = Collections.unmodifiableSet(
-                builder.snapshotMetadata == null
-                        ? Collections.emptySet() : new LinkedHashSet<>(builder.snapshotMetadata));
-
-        final Map<String,FlowSnapshotMetadata> tempMetadataMap = new HashMap<>();
-        this.snapshotMetadata.stream().forEach(s -> tempMetadataMap.put(String.valueOf(s.getVersion()), s));
-        this.snapshotMetadataByVersion = Collections.unmodifiableMap(tempMetadataMap);
-    }
-
-    @Override
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    @Override
-    public String getName() {
-        return name;
-    }
-
-    @Override
-    public String getBucketIdentifier() {
-        return bucketIdentifier;
-    }
-
-    @Override
-    public long getCreatedTimestamp() {
-        return createdTimestamp;
-    }
-
-    @Override
-    public long getModifiedTimestamp() {
-        return modifiedTimestamp;
-    }
-
-    @Override
-    public String getDescription() {
-        return description;
-    }
-
-    @Override
-    public Set<FlowSnapshotMetadata> getSnapshotMetadata() {
-        return snapshotMetadata;
-    }
-
-    @Override
-    public FlowSnapshotMetadata getSnapshot(int version) {
-        return snapshotMetadataByVersion.get(String.valueOf(version));
-    }
-
-    public static class Builder {
-
-        private String identifier;
-        private String name;
-        private String bucketIdentifier;
-        private long createdTimestamp;
-        private long modifiedTimestamp;
-        private String description;
-        private Set<FlowSnapshotMetadata> snapshotMetadata = new LinkedHashSet<>();
-
-        public Builder() {
-
-        }
-
-        public Builder(FlowMetadata flowMetadata) {
-            identifier(flowMetadata.getIdentifier());
-            name(flowMetadata.getName());
-            bucketIdentifier(flowMetadata.getBucketIdentifier());
-            created(flowMetadata.getCreatedTimestamp());
-            modified(flowMetadata.getModifiedTimestamp());
-            description(flowMetadata.getDescription());
-            addSnapshots(flowMetadata.getSnapshotMetadata());
-        }
-
-        public Builder identifier(final String identifier) {
-            this.identifier = identifier;
-            return this;
-        }
-
-        public Builder name(final String name) {
-            this.name = name;
-            return this;
-        }
-
-        public Builder bucketIdentifier(final String bucketIdentifier) {
-            this.bucketIdentifier = bucketIdentifier;
-            return this;
-        }
-
-        public Builder created(final long createdTimestamp) {
-            this.createdTimestamp = createdTimestamp;
-            return this;
-        }
-
-        public Builder modified(final long modifiedTimestamp) {
-            this.modifiedTimestamp = modifiedTimestamp;
-            return this;
-        }
-
-        public Builder description(final String description) {
-            this.description = description;
-            return this;
-        }
-
-        public Builder addSnapshot(final FlowSnapshotMetadata snapshotMetadata) {
-            if (snapshotMetadata != null) {
-                this.snapshotMetadata.add(snapshotMetadata);
-            }
-            return this;
-        }
-
-        public Builder addSnapshots(final Collection<FlowSnapshotMetadata> snapshotMetadata) {
-            if (snapshotMetadata != null) {
-                this.snapshotMetadata.addAll(snapshotMetadata);
-            }
-            return this;
-        }
-
-        public Builder clearSnapshots() {
-            this.snapshotMetadata.clear();
-            return this;
-        }
-
-        public StandardFlowMetadata build() {
-            return new StandardFlowMetadata(this);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowSnapshotMetadata.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowSnapshotMetadata.java b/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowSnapshotMetadata.java
deleted file mode 100644
index f8d09ce..0000000
--- a/nifi-registry-provider-api/src/main/java/org/apache/nifi/registry/metadata/StandardFlowSnapshotMetadata.java
+++ /dev/null
@@ -1,136 +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.metadata;
-
-import java.util.Objects;
-
-/**
- * Standard immutable implementation of FlowSnapshotMetadata.
- */
-public class StandardFlowSnapshotMetadata implements FlowSnapshotMetadata {
-
-    private final String bucketIdentifier;
-    private final String flowIdentifier;
-    private final String flowName;
-    private final int version;
-    private final long createdTimestamp;
-    private final String comments;
-
-    private StandardFlowSnapshotMetadata(final Builder builder) {
-        this.bucketIdentifier = builder.bucketIdentifier;
-        this.flowIdentifier = builder.flowIdentifier;
-        this.flowName = builder.flowName;
-        this.version = builder.version;
-        this.createdTimestamp = builder.createdTimestamp;
-        this.comments = builder.comments;
-    }
-
-    @Override
-    public String getBucketIdentifier() {
-        return bucketIdentifier;
-    }
-
-    @Override
-    public String getFlowIdentifier() {
-        return flowIdentifier;
-    }
-
-    @Override
-    public String getFlowName() {
-        return flowName;
-    }
-
-    @Override
-    public int getVersion() {
-        return version;
-    }
-
-    @Override
-    public long getCreatedTimestamp() {
-        return createdTimestamp;
-    }
-
-    @Override
-    public String getComments() {
-        return comments;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(this.bucketIdentifier, this.flowIdentifier, Integer.valueOf(this.version));
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final StandardFlowSnapshotMetadata other = (StandardFlowSnapshotMetadata) obj;
-
-        return Objects.equals(this.bucketIdentifier, other.bucketIdentifier)
-                && Objects.equals(this.flowIdentifier, other.flowIdentifier)
-                && Objects.equals(this.version, other.version);
-    }
-
-    public static class Builder {
-
-        private String bucketIdentifier;
-        private String flowIdentifier;
-        private String flowName;
-        private int version;
-        private long createdTimestamp;
-        private String comments;
-
-        public Builder bucketIdentifier(final String bucketIdentifier) {
-            this.bucketIdentifier = bucketIdentifier;
-            return this;
-        }
-
-        public Builder flowIdentifier(final String flowIdentifier) {
-            this.flowIdentifier = flowIdentifier;
-            return this;
-        }
-
-        public Builder flowName(final String flowName) {
-            this.flowName = flowName;
-            return this;
-        }
-
-        public Builder version(final int version) {
-            this.version = version;
-            return this;
-        }
-
-        public Builder created(long createdTimestamp) {
-            this.createdTimestamp = createdTimestamp;
-            return this;
-        }
-
-        public Builder comments(String comments) {
-            this.comments = comments;
-            return this;
-        }
-
-        public StandardFlowSnapshotMetadata build() {
-            return new StandardFlowSnapshotMetadata(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/pom.xml b/nifi-registry-provider-impl/pom.xml
deleted file mode 100644
index 2405ba9..0000000
--- a/nifi-registry-provider-impl/pom.xml
+++ /dev/null
@@ -1,94 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <groupId>org.apache.nifi.registry</groupId>
-        <artifactId>nifi-registry</artifactId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-    <artifactId>nifi-registry-provider-impl</artifactId>
-    <packaging>jar</packaging>
-
-    <build>
-        <resources>
-            <resource>
-                <directory>src/main/resources</directory>
-            </resource>
-            <resource>
-                <directory>src/main/xsd</directory>
-            </resource>
-        </resources>
-        <plugins>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>jaxb2-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>current</id>
-                        <goals>
-                            <goal>xjc</goal>
-                        </goals>
-                        <configuration>
-                            <packageName>org.apache.nifi.registry.metadata.generated</packageName>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <configuration>
-                    <excludes>**/metadata/generated/*.java,</excludes>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-provider-api</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-utils</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <version>4.12</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-core</artifactId>
-            <version>2.7.22</version>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/flow/FileSystemFlowPersistenceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/flow/FileSystemFlowPersistenceProvider.java b/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/flow/FileSystemFlowPersistenceProvider.java
deleted file mode 100644
index db09a6d..0000000
--- a/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/flow/FileSystemFlowPersistenceProvider.java
+++ /dev/null
@@ -1,166 +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.flow;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.provider.ProviderConfigurationContext;
-import org.apache.nifi.registry.provider.ProviderCreationException;
-import org.apache.nifi.registry.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-/**
- * A FlowPersistenceProvider that uses the local filesystem for storage.
- */
-public class FileSystemFlowPersistenceProvider implements FlowPersistenceProvider {
-
-    static final Logger LOGGER = LoggerFactory.getLogger(FileSystemFlowPersistenceProvider.class);
-
-    static final String FLOW_STORAGE_DIR_PROP = "Flow Storage Directory";
-
-    static final String SNAPSHOT_EXTENSION = ".snapshot";
-
-    private File flowStorageDir;
-
-    @Override
-    public void onConfigured(final ProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        final Map<String,String> props = configurationContext.getProperties();
-        if (!props.containsKey(FLOW_STORAGE_DIR_PROP)) {
-            throw new ProviderCreationException("The property " + FLOW_STORAGE_DIR_PROP + " must be provided");
-        }
-
-        final String flowStorageDirValue = props.get(FLOW_STORAGE_DIR_PROP);
-        if (StringUtils.isBlank(flowStorageDirValue)) {
-            throw new ProviderCreationException("The property " + FLOW_STORAGE_DIR_PROP + " cannot be null or blank");
-        }
-
-        try {
-            flowStorageDir = new File(flowStorageDirValue);
-            FileUtils.ensureDirectoryExistAndCanReadAndWrite(flowStorageDir);
-            LOGGER.info("Configured FileSystemFlowPersistenceProvider with Flow Storage Directory {}", new Object[] {flowStorageDir.getAbsolutePath()});
-        } catch (IOException e) {
-            throw new ProviderCreationException(e);
-        }
-    }
-
-    @Override
-    public synchronized void saveSnapshot(final FlowSnapshotContext context, final byte[] content) throws FlowPersistenceException {
-        final File bucketDir = new File(flowStorageDir, context.getBucketId());
-        try {
-            FileUtils.ensureDirectoryExistAndCanReadAndWrite(bucketDir);
-        } catch (IOException e) {
-            throw new FlowPersistenceException("Error accessing bucket directory at " + bucketDir.getAbsolutePath(), e);
-        }
-
-        final File flowDir = new File(bucketDir, context.getFlowId());
-        try {
-            FileUtils.ensureDirectoryExistAndCanReadAndWrite(flowDir);
-        } catch (IOException e) {
-            throw new FlowPersistenceException("Error accessing flow directory at " + flowDir.getAbsolutePath(), e);
-        }
-
-        final String versionString = String.valueOf(context.getVersion());
-        final File versionDir = new File(flowDir, versionString);
-        try {
-            FileUtils.ensureDirectoryExistAndCanReadAndWrite(versionDir);
-        } catch (IOException e) {
-            throw new FlowPersistenceException("Error accessing version directory at " + versionDir.getAbsolutePath(), e);
-        }
-
-        final File versionFile = new File(versionDir, versionString + SNAPSHOT_EXTENSION);
-        if (versionFile.exists()) {
-            throw new FlowPersistenceException("Unable to save, a snapshot already exists with version " + versionString);
-        }
-
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Saving snapshot with filename {}", new Object[] {versionFile.getAbsolutePath()});
-        }
-
-        try (final OutputStream out = new FileOutputStream(versionFile)) {
-            out.write(content);
-            out.flush();
-        } catch (Exception e) {
-            throw new FlowPersistenceException("Unable to write snapshot to disk due to " + e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public synchronized byte[] getSnapshot(final String bucketId, final String flowId, final int version) throws FlowPersistenceException {
-        final File snapshotFile = getSnapshotFile(bucketId, flowId, version);
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Retrieving snapshot with filename {}", new Object[] {snapshotFile.getAbsolutePath()});
-        }
-
-        if (!snapshotFile.exists()) {
-            return null;
-        }
-
-        try (final InputStream in = new FileInputStream(snapshotFile)){
-            return IOUtils.toByteArray(in);
-        } catch (IOException e) {
-            throw new FlowPersistenceException("Error reading snapshot file: " + snapshotFile.getAbsolutePath(), e);
-        }
-    }
-
-    @Override
-    public synchronized void deleteSnapshots(final String bucketId, final String flowId) throws FlowPersistenceException {
-        final File flowDir = new File(flowStorageDir, bucketId + "/" + flowId);
-        if (!flowDir.exists()) {
-            LOGGER.debug("Snapshot directory does not exist at {}", new Object[] {flowDir.getAbsolutePath()});
-            return;
-        }
-
-        try {
-            org.apache.commons.io.FileUtils.cleanDirectory(flowDir);
-        } catch (IOException e) {
-            throw new FlowPersistenceException("Error deleting snapshots at " + flowDir.getAbsolutePath(), e);
-        }
-    }
-
-    @Override
-    public synchronized void deleteSnapshot(final String bucketId, final String flowId, final int version) throws FlowPersistenceException {
-        final File snapshotFile = getSnapshotFile(bucketId, flowId, version);
-        if (!snapshotFile.exists()) {
-            LOGGER.debug("Snapshot file does not exist at {}", new Object[] {snapshotFile.getAbsolutePath()});
-            return;
-        }
-
-        final boolean deleted = snapshotFile.delete();
-        if (!deleted) {
-            throw new FlowPersistenceException("Unable to delete snapshot at " + snapshotFile.getAbsolutePath());
-        }
-
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Deleted snapshot at {}", new Object[] {snapshotFile.getAbsolutePath()});
-        }
-    }
-
-    protected File getSnapshotFile(final String bucketId, final String flowId, final int version) {
-        final String snapshotFilename = bucketId + "/" + flowId + "/" + version + "/" + version + SNAPSHOT_EXTENSION;
-        return new File(flowStorageDir, snapshotFilename);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/FileSystemMetadataProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/FileSystemMetadataProvider.java b/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/FileSystemMetadataProvider.java
deleted file mode 100644
index dd3e1ec..0000000
--- a/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/FileSystemMetadataProvider.java
+++ /dev/null
@@ -1,469 +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.metadata;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.metadata.generated.Buckets;
-import org.apache.nifi.registry.metadata.generated.Flow;
-import org.apache.nifi.registry.metadata.generated.Flows;
-import org.apache.nifi.registry.metadata.generated.Metadata;
-import org.apache.nifi.registry.provider.ProviderConfigurationContext;
-import org.apache.nifi.registry.provider.ProviderCreationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.File;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * A MetadataProvider that persists metadata to the local filesystem.
- */
-public class FileSystemMetadataProvider implements MetadataProvider {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(FileSystemMetadataProvider.class);
-
-    private static final String METADATA_XSD = "/metadata.xsd";
-    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.registry.metadata.generated";
-    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    /**
-     * Load the JAXBContext.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_GENERATED_PATH, FileSystemMetadataProvider.class.getClassLoader());
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.", e);
-        }
-    }
-
-    static final String METADATA_FILE_PROP = "Metadata File";
-
-    private File metadataFile;
-    private Schema metadataSchema;
-    private final AtomicReference<MetadataHolder> metadataHolder = new AtomicReference<>(null);
-
-    public FileSystemMetadataProvider() throws ProviderCreationException {
-        try {
-            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-            metadataSchema = schemaFactory.newSchema(FileSystemMetadataProvider.class.getResource(METADATA_XSD));
-        } catch (SAXException e) {
-            throw new ProviderCreationException("Unable to create MetadataProvider due to: " + e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public void onConfigured(final ProviderConfigurationContext configurationContext) throws ProviderCreationException {
-        final Map<String,String> config = configurationContext.getProperties();
-        if (!config.containsKey(METADATA_FILE_PROP)) {
-            throw new ProviderCreationException("The property " + METADATA_FILE_PROP + " must be provided");
-        }
-
-        final String metadataFileValue = config.get(METADATA_FILE_PROP);
-        if (StringUtils.isBlank(metadataFileValue)) {
-            throw new ProviderCreationException("The property " + METADATA_FILE_PROP + " cannot be null or blank");
-        }
-
-        try {
-            metadataFile = new File(metadataFileValue);
-            if (metadataFile.exists()) {
-                LOGGER.info("Loading metadata file from {}", new Object[] {metadataFile.getAbsolutePath()});
-                final Metadata metadata = unmarshallMetadata();
-                metadataHolder.set(new MetadataHolder(metadata));
-            } else {
-                LOGGER.info("Creating new metadata file at {}", new Object[] {metadataFile.getAbsolutePath()});
-
-                final Metadata metadata = new Metadata();
-                metadata.setBuckets(new Buckets());
-                metadata.setFlows(new Flows());
-
-                saveMetadata(metadata);
-                metadataHolder.set(new MetadataHolder(metadata));
-            }
-        } catch (Exception e) {
-            throw new ProviderCreationException("Unable to configure MetadataProvider due to: " + e.getMessage(), e);
-        }
-    }
-
-    private Metadata unmarshallMetadata() throws JAXBException {
-        final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-        unmarshaller.setSchema(metadataSchema);
-
-        final JAXBElement<Metadata> element = unmarshaller.unmarshal(new StreamSource(metadataFile), Metadata.class);
-        return element.getValue();
-    }
-
-    private void saveMetadata(final Metadata metadata) throws JAXBException {
-        final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
-        marshaller.setSchema(metadataSchema);
-        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(metadata, metadataFile);
-    }
-
-    private synchronized void saveAndRefresh(final Metadata metadata) {
-        try {
-            saveMetadata(metadata);
-            metadataHolder.set(new MetadataHolder(metadata));
-        } catch (JAXBException e) {
-            throw new MetadataProviderException("Unable to save metadata", e);
-        }
-    }
-
-    @Override
-    public synchronized BucketMetadata createBucket(final BucketMetadata bucket) {
-        if (bucket == null) {
-            throw new IllegalArgumentException("Bucket cannot be null");
-        }
-
-        final org.apache.nifi.registry.metadata.generated.Bucket jaxbBucket = new org.apache.nifi.registry.metadata.generated.Bucket();
-        jaxbBucket.setIdentifier(bucket.getIdentifier());
-        jaxbBucket.setName(bucket.getName());
-        jaxbBucket.setDescription(bucket.getDescription());
-        jaxbBucket.setCreatedTimestamp(bucket.getCreatedTimestamp());
-
-        final MetadataHolder holder = metadataHolder.get();
-
-        final Metadata metadata = holder.getMetadata();
-        metadata.getBuckets().getBucket().add(jaxbBucket);
-
-        saveAndRefresh(metadata);
-        return metadataHolder.get().getBucketsById().get(bucket.getIdentifier());
-    }
-
-    @Override
-    public BucketMetadata getBucketById(final String bucketIdentifier) {
-        if (bucketIdentifier == null) {
-            throw new IllegalArgumentException("Bucket Identifier cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        return holder.getBucketsById().get(bucketIdentifier);
-    }
-
-    @Override
-    public BucketMetadata getBucketByName(String name) {
-        if (name == null) {
-            throw new IllegalArgumentException("Bucket Name cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        return holder.getBucketsByName().get(name.toLowerCase());
-    }
-
-    @Override
-    public Set<BucketMetadata> getBuckets() {
-        final MetadataHolder holder = metadataHolder.get();
-        final Map<String,BucketMetadata> bucketsBydId = holder.getBucketsById();
-        return new HashSet<>(bucketsBydId.values());
-    }
-
-    @Override
-    public synchronized BucketMetadata updateBucket(final BucketMetadata bucket) {
-        if (bucket == null) {
-            throw new IllegalArgumentException("Bucket cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        final Buckets buckets = holder.getMetadata().getBuckets();
-
-        final org.apache.nifi.registry.metadata.generated.Bucket jaxbBucket = buckets.getBucket().stream()
-                .filter(b -> bucket.getIdentifier().equals(b.getIdentifier()))
-                .findFirst()
-                .orElse(null);
-
-        if (jaxbBucket == null) {
-            return null;
-        }
-
-        jaxbBucket.setName(bucket.getName());
-        jaxbBucket.setDescription(bucket.getDescription());
-
-        saveAndRefresh(holder.getMetadata());
-        return metadataHolder.get().getBucketsById().get(bucket.getIdentifier());
-    }
-
-    @Override
-    public synchronized void deleteBucket(final String bucketIdentifier) {
-        if (bucketIdentifier == null) {
-            throw new IllegalArgumentException("Bucket Identifier cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        final Flows flows = holder.getMetadata().getFlows();
-        final Buckets buckets = holder.getMetadata().getBuckets();
-
-        // first remove any flow that reference the bucket
-        boolean deletedFlow = false;
-        final Iterator<Flow> flowIterator = flows.getFlow().iterator();
-        while (flowIterator.hasNext()) {
-            final Flow flow = flowIterator.next();
-            if (flow.getBucketIdentifier().equals(bucketIdentifier)) {
-                flowIterator.remove();
-                deletedFlow = true;
-            }
-        }
-
-        // now delete the actual bucket
-        boolean deleteBucket = false;
-        final Iterator<org.apache.nifi.registry.metadata.generated.Bucket> bucketIterator = buckets.getBucket().iterator();
-        while (bucketIterator.hasNext()) {
-            final org.apache.nifi.registry.metadata.generated.Bucket bucket = bucketIterator.next();
-            if (bucket.getIdentifier().equals(bucketIdentifier)) {
-               bucketIterator.remove();
-               deleteBucket = true;
-               break;
-            }
-        }
-
-        if (deletedFlow || deleteBucket) {
-            saveAndRefresh(holder.getMetadata());
-        }
-    }
-
-    @Override
-    public synchronized FlowMetadata createFlow(final String bucketIdentifier, final FlowMetadata versionedFlow) {
-        if (bucketIdentifier == null) {
-            throw new IllegalArgumentException("Bucket Identifier cannot be blank");
-        }
-
-        if (versionedFlow == null) {
-            throw new IllegalArgumentException("Versioned Flow cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-
-        final BucketMetadata bucket = holder.getBucketsById().get(bucketIdentifier);
-        if (bucket == null) {
-            throw new IllegalStateException("Unable to create Versioned Flow because Bucket does not exist with id " + bucketIdentifier);
-        }
-
-        final Flow jaxbFlow = new Flow();
-        jaxbFlow.setIdentifier(versionedFlow.getIdentifier());
-        jaxbFlow.setName(versionedFlow.getName());
-        jaxbFlow.setDescription(versionedFlow.getDescription());
-        jaxbFlow.setCreatedTimestamp(versionedFlow.getCreatedTimestamp());
-        jaxbFlow.setModifiedTimestamp(versionedFlow.getModifiedTimestamp());
-        jaxbFlow.setBucketIdentifier(bucketIdentifier);
-
-        final Metadata metadata = holder.getMetadata();
-        metadata.getFlows().getFlow().add(jaxbFlow);
-
-        saveAndRefresh(metadata);
-        return metadataHolder.get().getFlowsById().get(versionedFlow.getIdentifier());
-    }
-
-    @Override
-    public FlowMetadata getFlowById(final String flowIdentifier) {
-        if (flowIdentifier == null) {
-            throw new IllegalArgumentException("Flow Identifier cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        return holder.getFlowsById().get(flowIdentifier);
-    }
-
-    @Override
-    public FlowMetadata getFlowByName(final String name) {
-        if (name == null) {
-            throw new IllegalArgumentException("Flow Name cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        return holder.getFlowsByName().get(name.toLowerCase());
-    }
-
-
-    @Override
-    public Set<FlowMetadata> getFlows() {
-        final MetadataHolder holder = metadataHolder.get();
-        final Map<String,FlowMetadata> flowsById = holder.getFlowsById();
-        return new HashSet<>(flowsById.values());
-    }
-
-    @Override
-    public Set<FlowMetadata> getFlows(String bucketId) {
-        final MetadataHolder holder = metadataHolder.get();
-
-        final Map<String,Set<FlowMetadata>> flowsByBucket = holder.getFlowsByBucket();
-        if (flowsByBucket.containsKey(bucketId)) {
-            return new HashSet<>(flowsByBucket.get(bucketId));
-        } else {
-            return Collections.emptySet();
-        }
-    }
-
-    @Override
-    public synchronized FlowMetadata updateFlow(final FlowMetadata versionedFlow) {
-        if (versionedFlow == null) {
-            throw new IllegalArgumentException("Versioned Flow cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        final Flows flows = holder.getMetadata().getFlows();
-
-        final Flow jaxbFlow = flows.getFlow().stream()
-                .filter(f -> versionedFlow.getIdentifier().equals(f.getIdentifier()))
-                .findFirst()
-                .orElse(null);
-
-        if (jaxbFlow == null) {
-            return null;
-        }
-
-        // TODO should we allow changing the bucket id here, if so it needs to be passed in
-        jaxbFlow.setName(versionedFlow.getName());
-        jaxbFlow.setDescription(versionedFlow.getDescription());
-        jaxbFlow.setModifiedTimestamp(System.currentTimeMillis());
-
-        saveAndRefresh(holder.getMetadata());
-        return metadataHolder.get().getFlowsById().get(versionedFlow.getIdentifier());
-    }
-
-    @Override
-    public synchronized void deleteFlow(final String flowIdentifier) {
-        if (flowIdentifier == null) {
-            throw new IllegalArgumentException("Flow Identifier cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        final Flows flows = holder.getMetadata().getFlows();
-
-        boolean deleted = false;
-        final Iterator<Flow> flowIter = flows.getFlow().iterator();
-
-        while (flowIter.hasNext()) {
-            final Flow jaxbFlow = flowIter.next();
-            if (jaxbFlow.getIdentifier().equals(flowIdentifier)) {
-                flowIter.remove();
-                deleted = true;
-                break;
-            }
-        }
-
-        if (deleted) {
-            saveAndRefresh(holder.getMetadata());
-        }
-    }
-
-    @Override
-    public synchronized FlowSnapshotMetadata createFlowSnapshot(final FlowSnapshotMetadata flowSnapshot) {
-        if (flowSnapshot == null) {
-            throw new IllegalArgumentException("Versioned Flow Snapshot cannot be null");
-        }
-
-        final String flowIdentifier = flowSnapshot.getFlowIdentifier();
-        final int snapshotVersion = flowSnapshot.getVersion();
-
-        final MetadataHolder holder = metadataHolder.get();
-        final Flows flows = holder.getMetadata().getFlows();
-
-        final Flow jaxbFlow = flows.getFlow().stream()
-                .filter(f -> flowIdentifier.equals(f.getIdentifier()))
-                .findFirst()
-                .orElse(null);
-
-        if (jaxbFlow == null) {
-            throw new IllegalStateException("Unable to create snapshot because Versioned Flow does not exist for id " + flowIdentifier);
-        }
-
-        final Flow.Snapshot jaxbSnapshot = new Flow.Snapshot();
-        jaxbSnapshot.setVersion(flowSnapshot.getVersion());
-        jaxbSnapshot.setComments(flowSnapshot.getComments());
-        jaxbSnapshot.setCreatedTimestamp(flowSnapshot.getCreatedTimestamp());
-
-        jaxbFlow.getSnapshot().add(jaxbSnapshot);
-        saveAndRefresh(holder.getMetadata());
-
-        final FlowMetadata versionedFlow = metadataHolder.get().getFlowsById().get(flowIdentifier);
-        return versionedFlow.getSnapshot(snapshotVersion);
-    }
-
-    @Override
-    public FlowSnapshotMetadata getFlowSnapshot(final String flowIdentifier, final Integer version) {
-        if (flowIdentifier == null) {
-            throw new IllegalArgumentException("Flow Identifier cannot be null");
-        }
-
-        if (version == null) {
-            throw new IllegalArgumentException("Version cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-
-        final FlowMetadata versionedFlow = holder.getFlowsById().get(flowIdentifier);
-        if (versionedFlow == null) {
-            return null;
-        }
-
-        return versionedFlow.getSnapshot(version);
-    }
-
-    @Override
-    public synchronized void deleteFlowSnapshot(final String flowIdentifier, final Integer version) {
-        if (flowIdentifier == null) {
-            throw new IllegalArgumentException("Flow Identifier cannot be null");
-        }
-
-        if (version == null) {
-            throw new IllegalArgumentException("Version cannot be null");
-        }
-
-        final MetadataHolder holder = metadataHolder.get();
-        final Flows flows = holder.getMetadata().getFlows();
-
-        final Flow jaxbFlow = flows.getFlow().stream()
-                .filter(f -> flowIdentifier.equals(f.getIdentifier()))
-                .findFirst()
-                .orElse(null);
-
-        if (jaxbFlow == null) {
-            return;
-        }
-
-        boolean deletedSnapshot = false;
-        final Iterator<Flow.Snapshot> snapshotIterator = jaxbFlow.getSnapshot().iterator();
-
-        while (snapshotIterator.hasNext()) {
-            final Flow.Snapshot snapshot = snapshotIterator.next();
-            if (snapshot.getVersion().equals(version)) {
-                snapshotIterator.remove();
-                deletedSnapshot = true;
-                break;
-            }
-        }
-
-        if (deletedSnapshot) {
-            saveAndRefresh(holder.getMetadata());
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/MetadataHolder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/MetadataHolder.java b/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/MetadataHolder.java
deleted file mode 100644
index e49a62b..0000000
--- a/nifi-registry-provider-impl/src/main/java/org/apache/nifi/registry/metadata/MetadataHolder.java
+++ /dev/null
@@ -1,176 +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.metadata;
-
-import org.apache.nifi.registry.metadata.generated.Bucket;
-import org.apache.nifi.registry.metadata.generated.Buckets;
-import org.apache.nifi.registry.metadata.generated.Flow;
-import org.apache.nifi.registry.metadata.generated.Flows;
-import org.apache.nifi.registry.metadata.generated.Metadata;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Provides atomic access to the Metadata object.
- */
-public class MetadataHolder {
-
-    private final Metadata metadata;
-    private final Map<String,Set<FlowMetadata>> flowsByBucket;
-    private final Map<String,FlowMetadata> flowsById;
-    private final Map<String,FlowMetadata> flowsByName;
-    private final Map<String,BucketMetadata> bucketsById;
-    private final Map<String,BucketMetadata> bucketsByName;
-
-    public MetadataHolder(final Metadata metadata) {
-        this.metadata = metadata;
-        this.flowsByBucket = Collections.unmodifiableMap(createFlowsByBucket(metadata));
-        this.flowsByName = Collections.unmodifiableMap(createFlowsByName(flowsByBucket));
-        this.flowsById = Collections.unmodifiableMap(createFlowsById(flowsByBucket));
-        this.bucketsById = Collections.unmodifiableMap(createBucketsBydId(metadata, flowsByBucket));
-        this.bucketsByName = Collections.unmodifiableMap(createBucketsByName(bucketsById));
-    }
-
-    private Map<String,BucketMetadata> createBucketsBydId(final Metadata metadata, final Map<String,Set<FlowMetadata>> flowsByBucket) {
-        final Map<String,BucketMetadata> bucketsById = new HashMap<>();
-
-        final Buckets buckets = metadata.getBuckets();
-        if (buckets != null) {
-            buckets.getBucket().stream().forEach(b -> {
-                    final Set<FlowMetadata> bucketFlows = flowsByBucket.get(b.getIdentifier());
-                    final BucketMetadata bucketMetadata = createBucketMetadata(b, bucketFlows);
-                    bucketsById.put(b.getIdentifier(), bucketMetadata);
-            });
-        }
-
-        return bucketsById;
-    }
-
-    private Map<String,BucketMetadata> createBucketsByName(Map<String,BucketMetadata> bucketsById) {
-        final Map<String,BucketMetadata> bucketsByName = new HashMap<>();
-        bucketsById.values().stream().forEach(b -> bucketsByName.put(b.getName().toLowerCase(), b));
-        return bucketsByName;
-    }
-
-    private BucketMetadata createBucketMetadata(final Bucket jaxbBucket, final Set<FlowMetadata> bucketFlows) {
-        return new StandardBucketMetadata.Builder()
-                .identifier(jaxbBucket.getIdentifier())
-                .name(jaxbBucket.getName())
-                .description(jaxbBucket.getDescription())
-                .created(jaxbBucket.getCreatedTimestamp())
-                .addFlows(bucketFlows)
-                .build();
-    }
-
-    private Map<String,Set<FlowMetadata>> createFlowsByBucket(final Metadata metadata) {
-        final Map<String,Set<FlowMetadata>> flowsByBucket = new HashMap<>();
-
-        final Flows flows = metadata.getFlows();
-        if (flows != null) {
-            flows.getFlow().stream().forEach(f -> {
-                Set<FlowMetadata> bucketFLows = flowsByBucket.get(f.getBucketIdentifier());
-                if (bucketFLows == null) {
-                    bucketFLows = new HashSet<>();
-                    flowsByBucket.put(f.getBucketIdentifier(), bucketFLows);
-                }
-                bucketFLows.add(createFlowMetadata(f));
-            });
-        }
-
-        return flowsByBucket;
-    }
-
-    private FlowMetadata createFlowMetadata(final Flow jaxbFlow) {
-        final StandardFlowMetadata.Builder builder = new StandardFlowMetadata.Builder()
-                .identifier(jaxbFlow.getIdentifier())
-                .name(jaxbFlow.getName())
-                .bucketIdentifier(jaxbFlow.getBucketIdentifier())
-                .description(jaxbFlow.getDescription())
-                .created(jaxbFlow.getCreatedTimestamp())
-                .modified(jaxbFlow.getModifiedTimestamp());
-
-        if (jaxbFlow.getSnapshot() != null) {
-            jaxbFlow.getSnapshot().stream().forEach(s -> builder.addSnapshot(createSnapshotMetadata(jaxbFlow, s)));
-        }
-
-        return builder.build();
-    }
-
-    private FlowSnapshotMetadata createSnapshotMetadata(final Flow jaxbFlow, final Flow.Snapshot jaxbSnapshot) {
-        return new StandardFlowSnapshotMetadata.Builder()
-                .bucketIdentifier(jaxbFlow.getBucketIdentifier())
-                .flowIdentifier(jaxbFlow.getIdentifier())
-                .flowName(jaxbFlow.getName())
-                .version(jaxbSnapshot.getVersion())
-                .comments(jaxbSnapshot.getComments())
-                .created(jaxbSnapshot.getCreatedTimestamp())
-                .build();
-    }
-
-    private Map<String,FlowMetadata> createFlowsById(final Map<String,Set<FlowMetadata>> flowsByBucket) {
-        final Map<String,FlowMetadata> flowsById = new HashMap<>();
-
-        for (final Map.Entry<String,Set<FlowMetadata>> entry : flowsByBucket.entrySet()) {
-            for (final FlowMetadata flowMetadata : entry.getValue()) {
-                flowsById.put(flowMetadata.getIdentifier(), flowMetadata);
-            }
-        }
-
-        return flowsById;
-    }
-
-    private Map<String,FlowMetadata> createFlowsByName(final Map<String,Set<FlowMetadata>> flowsByBucket) {
-        final Map<String,FlowMetadata> flowsByName = new HashMap<>();
-
-        for (final Map.Entry<String,Set<FlowMetadata>> entry : flowsByBucket.entrySet()) {
-            for (final FlowMetadata flow : entry.getValue()) {
-                flowsByName.put(flow.getName().toLowerCase(), flow);
-            }
-        }
-
-        return flowsByName;
-    }
-
-    public Metadata getMetadata() {
-        return metadata;
-    }
-
-    public Map<String,BucketMetadata> getBucketsById() {
-        return bucketsById;
-    }
-
-    public Map<String,BucketMetadata> getBucketsByName() {
-        return bucketsByName;
-    }
-
-    public Map<String,FlowMetadata> getFlowsById() {
-        return flowsById;
-    }
-
-    public Map<String,FlowMetadata> getFlowsByName() {
-        return flowsByName;
-    }
-
-    public Map<String,Set<FlowMetadata>> getFlowsByBucket() {
-        return flowsByBucket;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/main/xsd/metadata.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/main/xsd/metadata.xsd b/nifi-registry-provider-impl/src/main/xsd/metadata.xsd
deleted file mode 100644
index 7f71745..0000000
--- a/nifi-registry-provider-impl/src/main/xsd/metadata.xsd
+++ /dev/null
@@ -1,139 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
-
-    <!-- Buckets -->
-
-    <xs:complexType name="Bucket">
-        <xs:sequence>
-            <xs:element name="description" type="xs:string" minOccurs="0" maxOccurs="1"/>
-        </xs:sequence>
-        <xs:attribute name="identifier">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="name">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="createdTimestamp">
-            <xs:simpleType>
-                <xs:restriction base="xs:long">
-                    <xs:minInclusive value="1"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-    </xs:complexType>
-
-    <xs:complexType name="Buckets">
-        <xs:sequence>
-            <xs:element name="bucket" type="Bucket" minOccurs="0" maxOccurs="unbounded"/>
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Flows -->
-
-    <xs:complexType name="Flow">
-        <xs:sequence>
-            <xs:element name="description" type="xs:string" minOccurs="0" maxOccurs="1"/>
-            <xs:element name="snapshot" minOccurs="0" maxOccurs="unbounded" >
-                <xs:complexType>
-                    <xs:sequence>
-                        <xs:element name="comments" type="xs:string" minOccurs="0" maxOccurs="1"/>
-                    </xs:sequence>
-                    <xs:attribute name="version">
-                        <xs:simpleType>
-                            <xs:restriction base="xs:int">
-                                <xs:minInclusive value="1"/>
-                            </xs:restriction>
-                        </xs:simpleType>
-                    </xs:attribute>
-                    <xs:attribute name="createdTimestamp">
-                        <xs:simpleType>
-                            <xs:restriction base="xs:long">
-                                <xs:minInclusive value="1"/>
-                            </xs:restriction>
-                        </xs:simpleType>
-                    </xs:attribute>
-                </xs:complexType>
-            </xs:element>
-        </xs:sequence>
-        <xs:attribute name="identifier">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="name">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="createdTimestamp">
-            <xs:simpleType>
-                <xs:restriction base="xs:long">
-                    <xs:minInclusive value="1"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="modifiedTimestamp">
-            <xs:simpleType>
-                <xs:restriction base="xs:long">
-                    <xs:minInclusive value="1"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="bucketIdentifier">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-    </xs:complexType>
-
-    <xs:complexType name="Flows">
-        <xs:sequence>
-            <xs:element name="flow" type="Flow" minOccurs="0" maxOccurs="unbounded" />
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Metadata -->
-
-    <xs:element name="metadata">
-        <xs:complexType>
-            <xs:sequence>
-                <xs:element name="buckets" type="Buckets" minOccurs="0" maxOccurs="1"/>
-                <xs:element name="flows" type="Flows" minOccurs="0" maxOccurs="1" />
-            </xs:sequence>
-        </xs:complexType>
-    </xs:element>
-
-</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/d478c20e/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/flow/TestFileSystemFlowPersistenceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/flow/TestFileSystemFlowPersistenceProvider.java b/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/flow/TestFileSystemFlowPersistenceProvider.java
deleted file mode 100644
index aa0bf58..0000000
--- a/nifi-registry-provider-impl/src/test/java/org/apache/nifi/registry/flow/TestFileSystemFlowPersistenceProvider.java
+++ /dev/null
@@ -1,202 +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.flow;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.nifi.registry.provider.ProviderConfigurationContext;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.mockito.Mockito.when;
-
-public class TestFileSystemFlowPersistenceProvider {
-
-    static final String FLOW_STORAGE_DIR = "target/flow_storage";
-
-    static final ProviderConfigurationContext CONFIGURATION_CONTEXT = new ProviderConfigurationContext() {
-        @Override
-        public Map<String, String> getProperties() {
-            final Map<String,String> props = new HashMap<>();
-            props.put(FileSystemFlowPersistenceProvider.FLOW_STORAGE_DIR_PROP, FLOW_STORAGE_DIR);
-            return props;
-        }
-    };
-
-    private File flowStorageDir;
-    private FileSystemFlowPersistenceProvider fileSystemFlowProvider;
-
-    @Before
-    public void setup() throws IOException {
-        flowStorageDir = new File(FLOW_STORAGE_DIR);
-        if (flowStorageDir.exists()) {
-            org.apache.commons.io.FileUtils.cleanDirectory(flowStorageDir);
-            flowStorageDir.delete();
-        }
-
-        Assert.assertFalse(flowStorageDir.exists());
-
-        fileSystemFlowProvider = new FileSystemFlowPersistenceProvider();
-        fileSystemFlowProvider.onConfigured(CONFIGURATION_CONTEXT);
-        Assert.assertTrue(flowStorageDir.exists());
-    }
-
-    @Test
-    public void testSaveSuccessfully() throws IOException {
-        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 1, "flow1v1");
-        verifySnapshot(flowStorageDir, "bucket1", "flow1", 1, "flow1v1");
-
-        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 2, "flow1v2");
-        verifySnapshot(flowStorageDir, "bucket1", "flow1", 2, "flow1v2");
-
-        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow2", 1, "flow2v1");
-        verifySnapshot(flowStorageDir, "bucket1", "flow2", 1, "flow2v1");
-
-        createAndSaveSnapshot(fileSystemFlowProvider,"bucket2", "flow3", 1, "flow3v1");
-        verifySnapshot(flowStorageDir, "bucket2", "flow3", 1, "flow3v1");
-    }
-
-    @Test
-    public void testSaveWithExistingVersion() throws IOException {
-        final FlowSnapshotContext context = Mockito.mock(FlowSnapshotContext.class);
-        when(context.getBucketId()).thenReturn("bucket1");
-        when(context.getFlowId()).thenReturn("flow1");
-        when(context.getVersion()).thenReturn(1);
-
-        final byte[] content = "flow1v1".getBytes(StandardCharsets.UTF_8);
-        fileSystemFlowProvider.saveSnapshot(context, content);
-
-        // save new content for an existing version
-        final byte[] content2 = "XXX".getBytes(StandardCharsets.UTF_8);
-        try {
-            fileSystemFlowProvider.saveSnapshot(context, content2);
-            Assert.fail("Should have thrown exception");
-        } catch (Exception e) {
-
-        }
-
-        // verify the new content wasn't written
-        final File flowSnapshotFile = new File(flowStorageDir, "bucket1/flow1/1/1" + FileSystemFlowPersistenceProvider.SNAPSHOT_EXTENSION);
-        try (InputStream in = new FileInputStream(flowSnapshotFile)) {
-            Assert.assertEquals("flow1v1", IOUtils.toString(in, StandardCharsets.UTF_8));
-        }
-    }
-
-    @Test
-    public void testSaveAndGet() throws IOException {
-        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 1, "flow1v1");
-        createAndSaveSnapshot(fileSystemFlowProvider,"bucket1", "flow1", 2, "flow1v2");
-
-        final byte[] flow1v1 = fileSystemFlowProvider.getSnapshot("bucket1", "flow1", 1);
-        Assert.assertEquals("flow1v1", new String(flow1v1, StandardCharsets.UTF_8));
-
-        final byte[] flow1v2 = fileSystemFlowProvider.getSnapshot("bucket1", "flow1", 2);
-        Assert.assertEquals("flow1v2", new String(flow1v2, StandardCharsets.UTF_8));
-    }
-
-    @Test
-    public void testGetWhenDoesNotExist() {
-        final byte[] flow1v1 = fileSystemFlowProvider.getSnapshot("bucket1", "flow1", 1);
-        Assert.assertNull(flow1v1);
-    }
-
-    @Test
-    public void testDeleteSnapshots() throws IOException {
-        final String bucketId = "bucket1";
-        final String flowId = "flow1";
-
-        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 1, "flow1v1");
-        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 2, "flow1v2");
-
-        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
-        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
-
-        fileSystemFlowProvider.deleteSnapshots(bucketId, flowId);
-
-        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
-        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
-
-        // delete a flow that doesn't exist
-        fileSystemFlowProvider.deleteSnapshots(bucketId, "some-other-flow");
-
-        // delete a bucket that doesn't exist
-        fileSystemFlowProvider.deleteSnapshots("some-other-bucket", flowId);
-    }
-
-    @Test
-    public void testDeleteSnapshot() throws IOException {
-        final String bucketId = "bucket1";
-        final String flowId = "flow1";
-
-        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 1, "flow1v1");
-        createAndSaveSnapshot(fileSystemFlowProvider, bucketId, flowId, 2, "flow1v2");
-
-        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
-        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
-
-        fileSystemFlowProvider.deleteSnapshot(bucketId, flowId, 1);
-
-        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
-        Assert.assertNotNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
-
-        fileSystemFlowProvider.deleteSnapshot(bucketId, flowId, 2);
-
-        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 1));
-        Assert.assertNull(fileSystemFlowProvider.getSnapshot(bucketId, flowId, 2));
-
-        // delete a version that doesn't exist
-        fileSystemFlowProvider.deleteSnapshot(bucketId, flowId, 3);
-
-        // delete a flow that doesn't exist
-        fileSystemFlowProvider.deleteSnapshot(bucketId, "some-other-flow", 1);
-
-        // delete a bucket that doesn't exist
-        fileSystemFlowProvider.deleteSnapshot("some-other-bucket", flowId, 1);
-    }
-
-    private void createAndSaveSnapshot(final FlowPersistenceProvider flowPersistenceProvider, final String bucketId, final String flowId, final int version,
-                                       final String contentString) throws IOException {
-        final FlowSnapshotContext context = Mockito.mock(FlowSnapshotContext.class);
-        when(context.getBucketId()).thenReturn(bucketId);
-        when(context.getFlowId()).thenReturn(flowId);
-        when(context.getVersion()).thenReturn(version);
-
-        final byte[] content = contentString.getBytes(StandardCharsets.UTF_8);
-        flowPersistenceProvider.saveSnapshot(context, content);
-    }
-
-    private void verifySnapshot(final File flowStorageDir, final String bucketId, final String flowId, final int version,
-                                final String contentString) throws IOException {
-        // verify the correct snapshot file was created
-        final File flowSnapshotFile = new File(flowStorageDir,
-                bucketId + "/" + flowId + "/" + version + "/" + version + FileSystemFlowPersistenceProvider.SNAPSHOT_EXTENSION);
-        Assert.assertTrue(flowSnapshotFile.exists());
-
-        try (InputStream in = new FileInputStream(flowSnapshotFile)) {
-            Assert.assertEquals(contentString, IOUtils.toString(in, StandardCharsets.UTF_8));
-        }
-    }
-}