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:36 UTC

[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

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;
+    }
+
 }