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 2018/01/08 18:13:55 UTC

[07/50] nifi git commit: NIFI-4436: - Adding support to save a version of a flow based on a selected Process Group. - Adding support for revert changes back to the most recent version. - Adding support to disconnect from version control. - Moving the ver

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
index d3a11dc..9fbd5e8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
@@ -17,8 +17,6 @@
 
 package org.apache.nifi.web.api;
 
-import com.sun.jersey.api.client.ClientResponse.Status;
-import com.sun.jersey.api.core.ResourceContext;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
@@ -84,9 +82,9 @@ 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.Response.Status;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -109,8 +107,6 @@ import java.util.stream.Collectors;
 public class VersionsResource extends ApplicationResource {
     private static final Logger logger = LoggerFactory.getLogger(VersionsResource.class);
 
-    @Context
-    private ResourceContext resourceContext;
     private NiFiServiceFacade serviceFacade;
     private Authorizer authorizer;
     private ComponentLifecycle clusterComponentLifecycle;
@@ -128,12 +124,11 @@ public class VersionsResource extends ApplicationResource {
     private ActiveRequest activeRequest = null;
     private final Object activeRequestMonitor = new Object();
 
-
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("process-groups/{id}")
-    @ApiOperation(value = "Gets the Version Control information for a process group", response = VersionControlInformationEntity.class, authorizations = {
+    @ApiOperation(value = "Gets the Version Control information for a process group", response = VersionControlInformationEntity.class, notes = NON_GUARANTEED_ENDPOINT, authorizations = {
         @Authorization(value = "Read - /process-groups/{uuid}")
     })
     @ApiResponses(value = {
@@ -144,6 +139,7 @@ public class VersionsResource extends ApplicationResource {
         @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
     })
     public Response getVersionInformation(@ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId) {
+
         if (isReplicateRequest()) {
             return replicate(HttpMethod.GET);
         }
@@ -155,9 +151,11 @@ public class VersionsResource extends ApplicationResource {
         });
 
         // get the version control information for this process group
-        final VersionControlInformationEntity entity = serviceFacade.getVersionControlInformation(groupId);
+        VersionControlInformationEntity entity = serviceFacade.getVersionControlInformation(groupId);
         if (entity == null) {
-            throw new ResourceNotFoundException("Process Group with ID " + groupId + " is not currently under Version Control");
+            final ProcessGroupEntity processGroup = serviceFacade.getProcessGroup(groupId);
+            entity = new VersionControlInformationEntity();
+            entity.setProcessGroupRevision(processGroup.getRevision());
         }
 
         return generateOkResponse(entity).build();
@@ -168,7 +166,10 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("start-requests")
-    @ApiOperation(value = "Creates a request so that a Process Group can be placed under Version Control or have its Version Control configuration changed", response = VersionControlInformationEntity.class)
+    @ApiOperation(
+            value = "Creates a request so that a Process Group can be placed under Version Control or have its Version Control configuration changed",
+            response = VersionControlInformationEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT)
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -186,6 +187,7 @@ public class VersionsResource extends ApplicationResource {
             serviceFacade,
             /* entity */ null,
             lookup -> {
+                // TODO - pass in PG ID to authorize
             },
             /* verifier */ null,
             requestEntity -> {
@@ -213,9 +215,13 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("start-requests/{id}")
-    @ApiOperation(value = "Updates the request with the given ID", response = VersionControlInformationEntity.class, authorizations = {
-        @Authorization(value = "Write - /process-groups/{uuid}")
-    })
+    @ApiOperation(
+            value = "Updates the request with the given ID",
+            response = VersionControlInformationEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+                @Authorization(value = "Write - /process-groups/{uuid}")
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -301,7 +307,9 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("start-requests/{id}")
-    @ApiOperation(value = "Deletes the request with the given ID")
+    @ApiOperation(
+            value = "Deletes the request with the given ID",
+            notes = NON_GUARANTEED_ENDPOINT)
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -343,9 +351,13 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("process-groups/{id}")
-    @ApiOperation(value = "Begins version controlling the Process Group with the given ID", response = VersionControlInformationEntity.class, authorizations = {
-        @Authorization(value = "Read - /process-groups/{uuid}")
-    })
+    @ApiOperation(
+            value = "Begins version controlling the Process Group with the given ID",
+            response = VersionControlInformationEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+                @Authorization(value = "Read - /process-groups/{uuid}")
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -545,10 +557,14 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("process-groups/{id}")
-    @ApiOperation(value = "Stops version controlling the Process Group with the given ID", response = VersionControlInformationEntity.class, authorizations = {
-        @Authorization(value = "Read - /process-groups/{uuid}"),
-        @Authorization(value = "Write - /process-groups/{uuid}"),
-    })
+    @ApiOperation(
+            value = "Stops version controlling the Process Group with the given ID",
+            response = VersionControlInformationEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+                @Authorization(value = "Read - /process-groups/{uuid}"),
+                @Authorization(value = "Write - /process-groups/{uuid}"),
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -557,10 +573,14 @@ public class VersionsResource extends ApplicationResource {
         @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
     })
     public Response stopVersionControl(
-        @ApiParam(value = "The version is used to verify the client is working with the latest version of the flow.", required = false) @QueryParam(VERSION) final LongParameter version,
-
-        @ApiParam(value = "If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.", required = false) @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
-
+        @ApiParam(
+                value = "The version is used to verify the client is working with the latest version of the flow.",
+                required = false)
+        @QueryParam(VERSION) final LongParameter version,
+        @ApiParam(
+                value = "If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.",
+                required = false)
+        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
         @ApiParam("The process group id.") @PathParam("id") final String groupId) throws IOException {
 
         if (isReplicateRequest()) {
@@ -584,8 +604,8 @@ public class VersionsResource extends ApplicationResource {
                 }
             },
             (revision, groupEntity) -> {
-                // set the version control info to null
-                final VersionControlInformationEntity entity = serviceFacade.setVersionControlInformation(requestRevision, groupId, null, null);
+                // disconnect from version control
+                final VersionControlInformationEntity entity = serviceFacade.deleteVersionControl(requestRevision, groupId);
 
                 // generate the response
                 return generateOkResponse(entity).build();
@@ -597,12 +617,14 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("process-groups/{id}")
-    @ApiOperation(value = "For a Process Group that is already under Version Control, this will update the version of the flow to a different version",
-        response = VersionControlInformationEntity.class,
-        authorizations = {
-            @Authorization(value = "Read - /process-groups/{uuid}"),
-            @Authorization(value = "Write - /process-groups/{uuid}")
-        })
+    @ApiOperation(
+            value = "For a Process Group that is already under Version Control, this will update the version of the flow to a different version",
+            response = VersionControlInformationEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+                @Authorization(value = "Read - /process-groups/{uuid}"),
+                @Authorization(value = "Write - /process-groups/{uuid}")
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -686,10 +708,12 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("update-requests/{id}")
-    @ApiOperation(value = "Returns the Update Request with the given ID",
-        response = VersionedFlowUpdateRequestEntity.class,
-        authorizations = {
-        })
+    @ApiOperation(
+            value = "Returns the Update Request with the given ID",
+            response = VersionedFlowUpdateRequestEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -705,10 +729,12 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("revert-requests/{id}")
-    @ApiOperation(value = "Returns the Revert Request with the given ID",
-        response = VersionedFlowUpdateRequestEntity.class,
-        authorizations = {
-        })
+    @ApiOperation(
+            value = "Returns the Revert Request with the given ID",
+            response = VersionedFlowUpdateRequestEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -750,8 +776,12 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("update-requests/{id}")
-    @ApiOperation(value = "Deletes the Update Request with the given ID", response = VersionedFlowUpdateRequestEntity.class, authorizations = {
-    })
+    @ApiOperation(
+            value = "Deletes the Update Request with the given ID",
+            response = VersionedFlowUpdateRequestEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -767,8 +797,12 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("revert-requests/{id}")
-    @ApiOperation(value = "Deletes the Revert Request with the given ID", response = VersionedFlowUpdateRequestEntity.class, authorizations = {
-    })
+    @ApiOperation(
+            value = "Deletes the Revert Request with the given ID",
+            response = VersionedFlowUpdateRequestEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -813,11 +847,15 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("update-requests/process-groups/{id}")
-    @ApiOperation(value = "For a Process Group that is already under Version Control, this will initiate the action of changing "
-        + "from a specific version of the flow in the Flow Registry to a different version of the flow.", response = VersionedFlowUpdateRequestEntity.class, authorizations = {
-            @Authorization(value = "Read - /process-groups/{uuid}"),
-            @Authorization(value = "Write - /process-groups/{uuid}")
-        })
+    @ApiOperation(
+            value = "For a Process Group that is already under Version Control, this will initiate the action of changing "
+                    + "from a specific version of the flow in the Flow Registry to a different version of the flow.",
+            response = VersionedFlowUpdateRequestEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+                @Authorization(value = "Read - /process-groups/{uuid}"),
+                @Authorization(value = "Write - /process-groups/{uuid}")
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -970,12 +1008,16 @@ public class VersionsResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @Path("revert-requests/process-groups/{id}")
-    @ApiOperation(value = "For a Process Group that is already under Version Control, this will initiate the action of reverting "
-        + "any changes that have been made to the Process Group since it was last synchronized with the Flow Registry. This will result in the "
-        + "flow matching the Versioned Flow that exists in the Flow Registry.", response = VersionedFlowUpdateRequestEntity.class, authorizations = {
-            @Authorization(value = "Read - /process-groups/{uuid}"),
-            @Authorization(value = "Write - /process-groups/{uuid}")
-        })
+    @ApiOperation(
+            value = "For a Process Group that is already under Version Control, this will initiate the action of reverting "
+                + "any changes that have been made to the Process Group since it was last synchronized with the Flow Registry. This will result in the "
+                + "flow matching the Versioned Flow that exists in the Flow Registry.",
+            response = VersionedFlowUpdateRequestEntity.class,
+            notes = NON_GUARANTEED_ENDPOINT,
+            authorizations = {
+                @Authorization(value = "Read - /process-groups/{uuid}"),
+                @Authorization(value = "Write - /process-groups/{uuid}")
+            })
     @ApiResponses(value = {
         @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
         @ApiResponse(code = 401, message = "Client could not be authenticated."),
@@ -1250,7 +1292,7 @@ public class VersionsResource extends ApplicationResource {
     private <T> T getResponseEntity(final NodeResponse nodeResponse, final Class<T> clazz) {
         T entity = (T) nodeResponse.getUpdatedEntity();
         if (entity == null) {
-            entity = nodeResponse.getClientResponse().getEntity(clazz);
+            entity = nodeResponse.getClientResponse().readEntity(clazz);
         }
         return entity;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsynchronousWebRequest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsynchronousWebRequest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsynchronousWebRequest.java
index d09f895..2c14008 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsynchronousWebRequest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsynchronousWebRequest.java
@@ -17,10 +17,10 @@
 
 package org.apache.nifi.web.api.concurrent;
 
-import java.util.Date;
-
 import org.apache.nifi.authorization.user.NiFiUser;
 
+import java.util.Date;
+
 public interface AsynchronousWebRequest<T> {
 
     /**
@@ -67,7 +67,7 @@ public interface AsynchronousWebRequest<T> {
     /**
      * Indicates the reason that the request failed, or <code>null</code> if the request has not failed
      *
-     * @param explanation the reason that the request failed, or <code>null</code> if the request has not failed
+     * @return the reason that the request failed, or <code>null</code> if the request has not failed
      */
     String getFailureReason();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 489e590..ae3fc56 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -16,33 +16,7 @@
  */
 package org.apache.nifi.web.api.dto;
 
-import java.text.Collator;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import javax.ws.rs.WebApplicationException;
-
+import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
@@ -139,6 +113,7 @@ import org.apache.nifi.provenance.lineage.LineageEdge;
 import org.apache.nifi.provenance.lineage.LineageNode;
 import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
 import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.flow.VersionControlInformation;
 import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedConnection;
 import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedControllerService;
@@ -209,6 +184,32 @@ import org.apache.nifi.web.api.entity.VariableEntity;
 import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.revision.RevisionManager;
 
+import javax.ws.rs.WebApplicationException;
+import java.text.Collator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
 public final class DtoFactory {
 
     @SuppressWarnings("rawtypes")
@@ -225,6 +226,7 @@ public final class DtoFactory {
     private EntityFactory entityFactory;
     private Authorizer authorizer;
     private NiFiProperties properties;
+    private FlowRegistryClient flowRegistryClient;
 
     public ControllerConfigurationDTO createControllerConfigurationDto(final ControllerFacade controllerFacade) {
         final ControllerConfigurationDTO dto = new ControllerConfigurationDTO();
@@ -242,6 +244,7 @@ public final class DtoFactory {
         dto.setSupportsManagedAuthorizer(AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer));
         dto.setSupportsConfigurableUsersAndGroups(AuthorizerCapabilityDetection.isConfigurableUserGroupProvider(authorizer));
         dto.setSupportsConfigurableAuthorizer(AuthorizerCapabilityDetection.isConfigurableAccessPolicyProvider(authorizer));
+        dto.setSupportsFlowVersioning(CollectionUtils.isNotEmpty(flowRegistryClient.getRegistryIdentifiers()));
 
         final Date now = new Date();
         dto.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime()));
@@ -1687,6 +1690,9 @@ public final class DtoFactory {
         dto.setId(group.getIdentifier());
         dto.setName(group.getName());
 
+        final VersionControlInformationDTO versionControlInformation = createVersionControlInformationDto(group);
+        dto.setVersionControlInformation(versionControlInformation);
+
         return dto;
     }
 
@@ -2145,7 +2151,7 @@ public final class DtoFactory {
         dto.setComments(group.getComments());
         dto.setName(group.getName());
         dto.setVersionedComponentId(group.getVersionedComponentId().orElse(null));
-        dto.setVersionControlInformation(createVersionControlInformationDto(group.getVersionControlInformation()));
+        dto.setVersionControlInformation(createVersionControlInformationDto(group));
 
         final Map<String, String> variables = group.getVariableRegistry().getVariableMap().entrySet().stream()
             .collect(Collectors.toMap(entry -> entry.getKey().getName(), entry -> entry.getValue()));
@@ -2169,12 +2175,18 @@ public final class DtoFactory {
         return dto;
     }
 
-    public VersionControlInformationDTO createVersionControlInformationDto(final VersionControlInformation versionControlInfo) {
+    public VersionControlInformationDTO createVersionControlInformationDto(final ProcessGroup group) {
+        if (group == null) {
+            return null;
+        }
+
+        final VersionControlInformation versionControlInfo = group.getVersionControlInformation();
         if (versionControlInfo == null) {
             return null;
         }
 
         final VersionControlInformationDTO dto = new VersionControlInformationDTO();
+        dto.setGroupId(group.getIdentifier());
         dto.setRegistryId(versionControlInfo.getRegistryIdentifier());
         dto.setBucketId(versionControlInfo.getBucketIdentifier());
         dto.setFlowId(versionControlInfo.getFlowIdentifier());
@@ -3722,4 +3734,8 @@ public final class DtoFactory {
     public void setProperties(final NiFiProperties properties) {
         this.properties = properties;
     }
+
+    public void setFlowRegistryClient(FlowRegistryClient flowRegistryClient) {
+        this.flowRegistryClient = flowRegistryClient;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
index dd8d67f..1e6167c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
@@ -213,6 +213,7 @@ public final class EntityFactory {
 
     public ProcessGroupEntity createProcessGroupEntity(final ProcessGroupDTO dto, final RevisionDTO revision, final PermissionsDTO permissions,
                                                        final ProcessGroupStatusDTO status, final List<BulletinEntity> bulletins) {
+
         final ProcessGroupEntity entity = new ProcessGroupEntity();
         entity.setRevision(revision);
         if (dto != null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
index 5f4dba5..806979f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
@@ -16,10 +16,6 @@
  */
 package org.apache.nifi.web.dao;
 
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Future;
-
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
@@ -28,6 +24,10 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO;
 import org.apache.nifi.web.api.dto.VariableRegistryDTO;
 import org.apache.nifi.web.api.dto.VersionControlInformationDTO;
 
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Future;
+
 public interface ProcessGroupDAO {
 
     /**
@@ -112,7 +112,7 @@ public interface ProcessGroupDAO {
      * @param groupId the ID of the process group
      * @param proposedSnapshot Flow the new version of the flow
      * @param versionControlInformation the new Version Control Information
-     * @param the seed value to use for generating ID's for new components
+     * @param componentIdSeed the seed value to use for generating ID's for new components
      * @return the process group
      */
     ProcessGroup updateProcessGroupFlow(String groupId, VersionedFlowSnapshot proposedSnapshot, VersionControlInformationDTO versionControlInformation, String componentIdSeed,
@@ -128,6 +128,14 @@ public interface ProcessGroupDAO {
     ProcessGroup updateVersionControlInformation(VersionControlInformationDTO versionControlInformation, Map<String, String> versionedComponentMapping);
 
     /**
+     * Disconnects the specified group from version control.
+     *
+     * @param groupId group id
+     * @return the corresponding Process Group
+     */
+    ProcessGroup disconnectVersionControl(String groupId);
+
+    /**
      * Updates the specified variable registry
      *
      * @param variableRegistry the Variable Registry

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
index 6fa316d..f842a8c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
@@ -16,13 +16,6 @@
  */
 package org.apache.nifi.web.dao.impl;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Future;
-
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Position;
@@ -43,6 +36,13 @@ import org.apache.nifi.web.api.dto.VersionControlInformationDTO;
 import org.apache.nifi.web.api.entity.VariableEntity;
 import org.apache.nifi.web.dao.ProcessGroupDAO;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+
 public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGroupDAO {
 
     private FlowController flowController;
@@ -246,6 +246,12 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
         return group;
     }
 
+    public ProcessGroup disconnectVersionControl(final String groupId) {
+        final ProcessGroup group = locateProcessGroup(flowController, groupId);
+        group.disconnectVersionControl();
+        return group;
+    }
+
     @Override
     public ProcessGroup updateProcessGroupFlow(final String groupId, final VersionedFlowSnapshot proposedSnapshot, final VersionControlInformationDTO versionControlInformation,
         final String componentIdSeed, final boolean verifyNotModified) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
index c41d13b..3961be7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/ClusterReplicationComponentLifecycle.java
@@ -17,19 +17,6 @@
 
 package org.apache.nifi.web.util;
 
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.Response.Status;
-
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
 import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
@@ -55,7 +42,18 @@ import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.sun.jersey.core.util.MultivaluedMapImpl;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response.Status;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 public class ClusterReplicationComponentLifecycle implements ComponentLifecycle {
     private static final Logger logger = LoggerFactory.getLogger(ClusterReplicationComponentLifecycle.class);
@@ -155,8 +153,9 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
      * Periodically polls the process group with the given ID, waiting for all processors whose ID's are given to have the given Scheduled State.
      *
      * @param user the user making the request
+     * @param originalUri the original uri
      * @param groupId the ID of the Process Group to poll
-     * @param processorIds the ID of all Processors whose state should be equal to the given desired state
+     * @param processors the Processors whose state should be equal to the given desired state
      * @param desiredState the desired state for all processors with the ID's given
      * @param pause the Pause that can be used to wait between polling
      * @return <code>true</code> if successful, <code>false</code> if unable to wait for processors to reach the desired state
@@ -172,7 +171,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
         }
 
         final Map<String, String> headers = new HashMap<>();
-        final MultivaluedMap<String, String> requestEntity = new MultivaluedMapImpl();
+        final MultivaluedMap<String, String> requestEntity = new MultivaluedHashMap<>();
 
         boolean continuePolling = true;
         while (continuePolling) {
@@ -217,7 +216,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
     private <T> T getResponseEntity(final NodeResponse nodeResponse, final Class<T> clazz) {
         T entity = (T) nodeResponse.getUpdatedEntity();
         if (entity == null) {
-            entity = nodeResponse.getClientResponse().getEntity(clazz);
+            entity = nodeResponse.getClientResponse().readEntity(clazz);
         }
         return entity;
     }
@@ -354,7 +353,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
         }
 
         final Map<String, String> headers = new HashMap<>();
-        final MultivaluedMap<String, String> requestEntity = new MultivaluedMapImpl();
+        final MultivaluedMap<String, String> requestEntity = new MultivaluedHashMap<>();
 
         boolean continuePolling = true;
         while (continuePolling) {
@@ -405,7 +404,7 @@ public class ClusterReplicationComponentLifecycle implements ComponentLifecycle
      * Updates the affected controller services in the specified updateRequest with the serviceEntities.
      *
      * @param serviceEntities service entities
-     * @param updateRequest update request
+     * @param affectedServices affected services
      */
     private void updateAffectedControllerServices(final Set<ControllerServiceEntity> serviceEntities, final Map<String, AffectedComponentEntity> affectedServices) {
         // update the affected components

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
index 22ffec7..e005d28 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/LocalComponentLifecycle.java
@@ -17,12 +17,6 @@
 
 package org.apache.nifi.web.util;
 
-import java.net.URI;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceState;
@@ -38,6 +32,12 @@ import org.apache.nifi.web.revision.RevisionManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.net.URI;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
 public class LocalComponentLifecycle implements ComponentLifecycle {
     private static final Logger logger = LoggerFactory.getLogger(LocalComponentLifecycle.class);
 
@@ -232,7 +232,7 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
      * Periodically polls the process group with the given ID, waiting for all controller services whose ID's are given to have the given Controller Service State.
      *
      * @param groupId the ID of the Process Group to poll
-     * @param serviceIds the ID of all Controller Services whose state should be equal to the given desired state
+     * @param affectedServices all Controller Services whose state should be equal to the given desired state
      * @param desiredState the desired state for all services with the ID's given
      * @param pause the Pause that can be used to wait between polling
      * @param user the user that is retrieving the controller services
@@ -275,7 +275,7 @@ public class LocalComponentLifecycle implements ComponentLifecycle {
      * Updates the affected controller services in the specified updateRequest with the serviceEntities.
      *
      * @param serviceEntities service entities
-     * @param updateRequest update request
+     * @param affectedServices all Controller Services whose state should be equal to the given desired state
      */
     private void updateAffectedControllerServices(final Set<ControllerServiceEntity> serviceEntities, final Map<String, AffectedComponentEntity> affectedServices) {
         // update the affected components

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 48db565..99a4f1c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -53,6 +53,7 @@
         <property name="authorizer" ref="authorizer"/>
         <property name="bulletinRepository" ref="bulletinRepository"/>
         <property name="properties" ref="nifiProperties"/>
+        <property name="flowRegistryClient" ref="flowRegistryClient" />
     </bean>
 
     <!-- snippet utils -->
@@ -208,6 +209,7 @@
         <property name="remoteProcessGroupResource" ref="remoteProcessGroupResource"/>
         <property name="connectionResource" ref="connectionResource"/>
         <property name="templateResource" ref="templateResource"/>
+        <property name="controllerResource" ref="controllerResource"/>
         <property name="controllerServiceResource" ref="controllerServiceResource"/>
         <property name="reportingTaskResource" ref="reportingTaskResource"/>
         <property name="processGroupResource" ref="processGroupResource"/>
@@ -215,6 +217,7 @@
         <property name="clusterCoordinator" ref="clusterCoordinator"/>
         <property name="requestReplicator" ref="requestReplicator" />
         <property name="flowController" ref="flowController" />
+        <property name="flowRegistryClient" ref="flowRegistryClient" />
     </bean>
     <bean id="resourceResource" class="org.apache.nifi.web.api.ResourceResource" scope="singleton">
         <property name="serviceFacade" ref="serviceFacade"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 4f2cc8e..a6f0385 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -475,6 +475,7 @@
                                                 <include>${staging.dir}/js/nf/canvas/nf-draggable.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-connectable.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-graph.js</include>
+                                                <include>${staging.dir}/js/nf/canvas/nf-flow-version.js</include>
                                                 <include>${staging.dir}/js/nf/nf-filtered-dialog-common.js</include>
                                                 <include>${staging.dir}/js/nf/nf-status-history.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-queue-listing.js</include>

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
index 413c6c2..40a3b1f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
@@ -57,6 +57,7 @@ nf.canvas.script.tags=<script type="text/javascript" src="js/nf/nf-ng-bridge.js?
 <script type="text/javascript" src="js/nf/canvas/nf-draggable.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-connectable.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-graph.js?${project.version}"></script>\n\
+<script type="text/javascript" src="js/nf/canvas/nf-flow-version.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/nf-filtered-dialog-common.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/nf-status-history.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-queue-listing.js?${project.version}"></script>\n\

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
index 3c7d407..c57b76f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
@@ -115,6 +115,8 @@
         <jsp:include page="/WEB-INF/partials/canvas/instantiate-template-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/fill-color-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/connections-dialog.jsp"/>
+        <jsp:include page="/WEB-INF/partials/canvas/save-flow-version-dialog.jsp"/>
+        <jsp:include page="/WEB-INF/partials/canvas/registry-configuration-dialog.jsp"/>
         <div id="canvas-container" class="unselectable"></div>
         <div id="canvas-tooltips">
             <div id="processor-tooltips"></div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
index 13e2146..7dd481f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp
@@ -22,56 +22,56 @@
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.processor}}"
                     id="processor-component"
                     class="component-button icon icon-processor"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.processorComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.inputPort}}"
                     id="port-in-component"
                     class="component-button icon icon-port-in"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.inputPortComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.outputPort}}"
                     id="port-out-component"
                     class="component-button icon icon-port-out"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.outputPortComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.processGroup}}"
                     id="group-component"
                     class="component-button icon icon-group"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.groupComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.remoteProcessGroup}}"
                     id="group-remote-component"
                     class="component-button icon icon-group-remote"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.remoteGroupComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.funnel}}"
                     id="funnel-component"
                     class="component-button icon icon-funnel"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.funnelComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.template}}"
                     id="template-component"
                     class="component-button icon icon-template"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.templateComponent);">
                 <span class="component-button-grip"></span>
             </button>
             <button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.label}}"
                     id="label-component"
                     class="component-button icon icon-label"
-                    ng-disabled="!appCtrl.nf.CanvasUtils.canWrite();"
+                    ng-disabled="!appCtrl.nf.CanvasUtils.canWriteCurrentGroup();"
                     nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.labelComponent);">
                 <span class="component-button-grip"></span>
             </button>

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registry-configuration-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registry-configuration-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registry-configuration-dialog.jsp
new file mode 100644
index 0000000..7fa90f7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registry-configuration-dialog.jsp
@@ -0,0 +1,40 @@
+<%--
+ 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.
+--%>
+<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
+<div id="registry-configuration-dialog" layout="column" class="hidden medium-dialog">
+    <div class="dialog-content">
+        <div class="setting">
+            <div class="setting-name">Name</div>
+            <div class="setting-field">
+                <span id="registry-id" class="hidden"></span>
+                <input type="text" id="registry-name" class="setting-input"/>
+            </div>
+        </div>
+        <div class="setting">
+            <div class="setting-name">Location</div>
+            <div class="setting-field">
+                <input type="text" id="registry-location" class="setting-input"/>
+            </div>
+        </div>
+        <div class="setting">
+            <div class="setting-name">Description</div>
+            <div class="setting-field">
+                <textarea id="registry-description" class="setting-input"></textarea>
+            </div>
+        </div>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/save-flow-version-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/save-flow-version-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/save-flow-version-dialog.jsp
new file mode 100644
index 0000000..dfed409
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/save-flow-version-dialog.jsp
@@ -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.
+--%>
+<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
+<div id="save-flow-version-dialog" layout="column" class="hidden large-dialog">
+    <div class="dialog-content">
+        <div class="setting">
+            <div class="setting-name">Registry</div>
+            <div class="setting-field">
+                <div id="flow-version-registry-combo"></div>
+                <div id="flow-version-registry" class="hidden"></div>
+            </div>
+        </div>
+        <div class="setting">
+            <div class="setting-name">Location</div>
+            <div class="setting-field">
+                <div id="flow-version-bucket-combo"></div>
+                <div id="flow-version-bucket" class="hidden"></div>
+            </div>
+        </div>
+        <div class="setting">
+            <div class="setting-name">Name</div>
+            <div class="setting-field">
+                <span id="flow-version-process-group-id" class="hidden"></span>
+                <input type="text" id="flow-version-name" class="setting-input"/>
+            </div>
+        </div>
+        <div class="setting">
+            <div class="setting-name">Description</div>
+            <div class="setting-field">
+                <textarea id="flow-version-description" class="setting-input"></textarea>
+            </div>
+        </div>
+        <div class="setting">
+            <div class="setting-name">Change Comments</div>
+            <div class="setting-field">
+                <textarea id="flow-version-change-comments" class="setting-input"></textarea>
+            </div>
+        </div>
+    </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
index ca22c04..57f43c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
@@ -62,6 +62,9 @@
             <div id="reporting-tasks-tab-content" class="configuration-tab controller-settings-table">
                 <div id="reporting-tasks-table" class="settings-table"></div>
             </div>
+            <div id="registries-tab-content" class="configuration-tab controller-settings-table">
+                <div id="registries-table" class="settings-table"></div>
+            </div>
         </div>
     </div>
     <div id="settings-refresh-container">

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
index cb5282b..1c44e71 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
@@ -213,6 +213,14 @@ div.progress-label {
 }
 
 /*
+    Flow Version
+ */
+
+#flow-version-description, #flow-version-change-comments {
+    height: 85px;
+}
+
+/*
     Variable Registry
  */
 
@@ -252,6 +260,14 @@ div.slick-cell div.overridden {
 }
 
 /*
+    Registry configuration dialog
+ */
+
+#registry-description {
+    height: 85px;
+}
+
+/*
     General dialog styles.
 */
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/graph.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/graph.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/graph.css
index f5c1a6f..caad5bd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/graph.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/graph.css
@@ -422,6 +422,14 @@ text.process-group-name {
     font-size: 14px;
 }
 
+text.version-control {
+    font-family: FontAwesome;
+    font-size: 18px;
+    fill: rgba(0, 255, 0, 0.65);
+    stroke: rgba(0, 0, 0, 0.65);
+    visibility: hidden;
+}
+
 text.process-group-contents-count {
     fill: #775351;
     font-size: 15px;

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/navigation.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/navigation.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/navigation.css
index 2439d22..fc930eb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/navigation.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/navigation.css
@@ -278,6 +278,11 @@ rect.birdseye-brush {
     top: 8px;
 }
 
+span.breadcrumb-version-control {
+    color: #0f0;
+    text-shadow: 0px 0px 1px #000;
+}
+
 #breadcrumbs-left-border {
     position: absolute;
     left: 0;

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-breadcrumbs-controller.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-breadcrumbs-controller.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-breadcrumbs-controller.js
index 72cd57b..5bf70b1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-breadcrumbs-controller.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-breadcrumbs-controller.js
@@ -75,6 +75,21 @@
             },
 
             /**
+             * Updates the version control information for the specified process group.
+             *
+             * @param processGroupId
+             * @param versionControlInformation
+             */
+            updateVersionControlInformation: function (processGroupId, versionControlInformation) {
+                $.each(this.breadcrumbs, function (_, breadcrumbEntity) {
+                    if (breadcrumbEntity.id === processGroupId) {
+                        breadcrumbEntity.breadcrumb.versionControlInformation = versionControlInformation;
+                        return false;
+                    }
+                });
+            },
+
+            /**
              * Reset the breadcrumbs.
              */
             resetBreadcrumbs: function () {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-canvas-graph-controls-controller.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-canvas-graph-controls-controller.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-canvas-graph-controls-controller.js
index e661338..3c9df0b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-canvas-graph-controls-controller.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/controllers/nf-ng-canvas-graph-controls-controller.js
@@ -234,7 +234,7 @@
              */
             getContextName: function () {
                 var selection = nfCanvasUtils.getSelection();
-                var canRead = nfCanvasUtils.canReadFromGroup();
+                var canRead = nfCanvasUtils.canReadCurrentGroup();
 
                 if (selection.empty()) {
                     if (canRead) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
index 90a1140..ff09330 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
@@ -33,6 +33,7 @@
                 'nf.Shell',
                 'nf.VariableRegistry',
                 'nf.ComponentState',
+                'nf.FlowVersion',
                 'nf.Draggable',
                 'nf.Birdseye',
                 'nf.Connection',
@@ -55,8 +56,8 @@
                 'nf.ComponentVersion',
                 'nf.QueueListing',
                 'nf.StatusHistory'],
-            function ($, d3, nfCanvasUtils, nfCommon, nfDialog, nfClient, nfErrorHandler, nfClipboard, nfSnippet, nfGoto, nfNgBridge, nfShell, nfVariableRegistry, nfComponentState, nfDraggable, nfBirdseye, nfConnection, nfGraph, nfProcessGroupConfiguration, nfProcessorConfiguration, nfProcessorDetails, nfLabelConfiguration, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupDetails, nfPortConfiguration, nfPortDetails, nfConnectionConfiguration, nfConnectionDetails, nfPolicyManagement, nfRemoteProcessGroup, nfLabel, nfProcessor, nfRemoteProcessGroupPorts, nfComponentVersion, nfQueueListing, nfStatusHistory) {
-                return (nf.Actions = factory($, d3, nfCanvasUtils, nfCommon, nfDialog, nfClient, nfErrorHandler, nfClipboard, nfSnippet, nfGoto, nfNgBridge, nfShell, nfVariableRegistry, nfComponentState, nfDraggable, nfBirdseye, nfConnection, nfGraph, nfProcessGroupConfiguration, nfProcessorConfiguration, nfProcessorDetails, nfLabelConfiguration, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupDetails, nfPortConfiguration, nfPortDetails, nfConnectionConfiguration, nfConnectionDetails, nfPolicyManagement, nfRemoteProcessGroup, nfLabel, nfProcessor, nfRemoteProcessGroupPorts, nfComponentVersion, nfQueueListing, nfStatusHistory));
+            function ($, d3, nfCanvasUtils, nfCommon, nfDialog, nfClient, nfErrorHandler, nfClipboard, nfSnippet, nfGoto, nfNgBridge, nfShell, nfVariableRegistry, nfComponentState, nfFlowVersion, nfDraggable, nfBirdseye, nfConnection, nfGraph, nfProcessGroupConfiguration, nfProcessorConfiguration, nfProcessorDetails, nfLabelConfiguration, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupDetails, nfPortConfiguration, nfPortDetails, nfConnectionConfiguration, nfConnectionDetails, nfPolicyManagement, nfRemoteProcessGroup, nfLabel, nfProcessor, nfRemoteProcessGroupPorts, nfComponentVersion, nfQueueListing, nfStatusHistory) {
+                return (nf.Actions = factory($, d3, nfCanvasUtils, nfCommon, nfDialog, nfClient, nfErrorHandler, nfClipboard, nfSnippet, nfGoto, nfNgBridge, nfShell, nfVariableRegistry, nfComponentState, nfFlowVersion, nfDraggable, nfBirdseye, nfConnection, nfGraph, nfProcessGroupConfiguration, nfProcessorConfiguration, nfProcessorDetails, nfLabelConfiguration, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupDetails, nfPortConfiguration, nfPortDetails, nfConnectionConfiguration, nfConnectionDetails, nfPolicyManagement, nfRemoteProcessGroup, nfLabel, nfProcessor, nfRemoteProcessGroupPorts, nfComponentVersion, nfQueueListing, nfStatusHistory));
             });
     } else if (typeof exports === 'object' && typeof module === 'object') {
         module.exports = (nf.Actions =
@@ -74,6 +75,7 @@
                 require('nf.Shell'),
                 require('nf.VariableRegistry'),
                 require('nf.ComponentState'),
+                require('nf.FlowVersion'),
                 require('nf.Draggable'),
                 require('nf.Birdseye'),
                 require('nf.Connection'),
@@ -111,6 +113,7 @@
             root.nf.Shell,
             root.nf.VariableRegistry,
             root.nf.ComponentState,
+            root.nf.FlowVersion,
             root.nf.Draggable,
             root.nf.Birdseye,
             root.nf.Connection,
@@ -134,7 +137,7 @@
             root.nf.QueueListing,
             root.nf.StatusHistory);
     }
-}(this, function ($, d3, nfCanvasUtils, nfCommon, nfDialog, nfClient, nfErrorHandler, nfClipboard, nfSnippet, nfGoto, nfNgBridge, nfShell, nfVariableRegistry, nfComponentState, nfDraggable, nfBirdseye, nfConnection, nfGraph, nfProcessGroupConfiguration, nfProcessorConfiguration, nfProcessorDetails, nfLabelConfiguration, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupDetails, nfPortConfiguration, nfPortDetails, nfConnectionConfiguration, nfConnectionDetails, nfPolicyManagement, nfRemoteProcessGroup, nfLabel, nfProcessor, nfRemoteProcessGroupPorts, nfComponentVersion, nfQueueListing, nfStatusHistory) {
+}(this, function ($, d3, nfCanvasUtils, nfCommon, nfDialog, nfClient, nfErrorHandler, nfClipboard, nfSnippet, nfGoto, nfNgBridge, nfShell, nfVariableRegistry, nfComponentState, nfFlowVersion, nfDraggable, nfBirdseye, nfConnection, nfGraph, nfProcessGroupConfiguration, nfProcessorConfiguration, nfProcessorDetails, nfLabelConfiguration, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupDetails, nfPortConfiguration, nfPortDetails, nfConnectionConfiguration, nfConnectionDetails, nfPolicyManagement, nfRemoteProcessGroup, nfLabel, nfProcessor, nfRemoteProcessGroupPorts, nfComponentVersion, nfQueueListing, nfStatusHistory) {
     'use strict';
 
     var config = {
@@ -1237,6 +1240,51 @@
         },
 
         /**
+         * Shows the flow version dialog.
+         */
+        saveFlowVersion: function (selection) {
+            if (selection.empty()) {
+                nfFlowVersion.showFlowVersionDialog(nfCanvasUtils.getGroupId());
+            } else if (selection.size() === 1) {
+                var selectionData = selection.datum();
+                if (nfCanvasUtils.isProcessGroup(selection)) {
+                    nfFlowVersion.showFlowVersionDialog(selectionData.id);
+                }
+            }
+        },
+
+        /**
+         * Reverts outstanding changes.
+         */
+        revertFlowChanges: function (selection) {
+            if (selection.empty()) {
+                nfFlowVersion.revertFlowChanges(nfCanvasUtils.getGroupId());
+            } else if (selection.size() === 1) {
+                var selectionData = selection.datum();
+                nfFlowVersion.revertFlowChanges(selectionData.id);
+            }
+        },
+
+        /**
+         * Changes the flow version.
+         */
+        changeFlowVersion: function (selection) {
+
+        },
+
+        /**
+         * Disconnects a Process Group from flow versioning.
+         */
+        disconnectFlowVersioning: function (selection) {
+            if (selection.empty()) {
+                nfFlowVersion.disconnectFlowVersioning(nfCanvasUtils.getGroupId());
+            } else if (selection.size() === 1) {
+                var selectionData = selection.datum();
+                nfFlowVersion.disconnectFlowVersioning(selectionData.id);
+            }
+        },
+
+        /**
          * Opens the variable registry for the specified selection of the current group if the selection is emtpy.
          *
          * @param {selection} selection

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-bootstrap.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-bootstrap.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-bootstrap.js
index e3ab5c9..6e17b27 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-bootstrap.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-bootstrap.js
@@ -39,6 +39,7 @@
                 'nf.QueueListing',
                 'nf.VariableRegistry',
                 'nf.ComponentState',
+                'nf.FlowVersion',
                 'nf.ComponentVersion',
                 'nf.Draggable',
                 'nf.Connectable',
@@ -82,8 +83,8 @@
                 'nf.ng.Canvas.OperateCtrl',
                 'nf.ng.BreadcrumbsDirective',
                 'nf.ng.DraggableDirective'],
-            function ($, angular, nfCommon, nfCanvasUtils, nfErrorHandler, nfClient, nfClusterSummary, nfDialog, nfStorage, nfCanvas, nfGraph, nfContextMenu, nfQuickSelect, nfShell, nfSettings, nfActions, nfSnippet, nfQueueListing, nfVariableRegistry, nfComponentState, nfComponentVersion, nfDraggable, nfConnectable, nfStatusHistory, nfBirdseye, nfConnectionConfiguration, nfControllerService, nfReportingTask, nfPolicyManagement, nfProcessorConfiguration, nfProcessGroupConfiguration, nfControllerServices, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupPorts, nfPortConfiguration, nfLabelConfiguration, nfProcessorDetails, nfPortDetails, nfConnectionDetails, nfRemoteProcessGroupDetails, nfGoto, nfNgBridge, appCtrl, appConfig, serviceProvider, breadcrumbsCtrl, headerCtrl, flowStatusCtrl, globalMenuCtrl, toolboxCtrl, processorComponent, inputPortComponent, outputPortComponent, processGroupComponent, remoteProcessGroupComponent, funnelComponent, templateComponent, labelComponent, gr
 aphControlsCtrl, navigateCtrl, operateCtrl, breadcrumbsDirective, draggableDirective) {
-                return factory($, angular, nfCommon, nfCanvasUtils, nfErrorHandler, nfClient, nfClusterSummary, nfDialog, nfStorage, nfCanvas, nfGraph, nfContextMenu, nfQuickSelect, nfShell, nfSettings, nfActions, nfSnippet, nfQueueListing, nfVariableRegistry, nfComponentState, nfComponentVersion, nfDraggable, nfConnectable, nfStatusHistory, nfBirdseye, nfConnectionConfiguration, nfControllerService, nfReportingTask, nfPolicyManagement, nfProcessorConfiguration, nfProcessGroupConfiguration, nfControllerServices, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupPorts, nfPortConfiguration, nfLabelConfiguration, nfProcessorDetails, nfPortDetails, nfConnectionDetails, nfRemoteProcessGroupDetails, nfGoto, nfNgBridge, appCtrl, appConfig, serviceProvider, breadcrumbsCtrl, headerCtrl, flowStatusCtrl, globalMenuCtrl, toolboxCtrl, processorComponent, inputPortComponent, outputPortComponent, processGroupComponent, remoteProcessGroupComponent, funnelComponent, templateComponent, labelComp
 onent, graphControlsCtrl, navigateCtrl, operateCtrl, breadcrumbsDirective, draggableDirective);
+            function ($, angular, nfCommon, nfCanvasUtils, nfErrorHandler, nfClient, nfClusterSummary, nfDialog, nfStorage, nfCanvas, nfGraph, nfContextMenu, nfQuickSelect, nfShell, nfSettings, nfActions, nfSnippet, nfQueueListing, nfVariableRegistry, nfComponentState, nfFlowVersion, nfComponentVersion, nfDraggable, nfConnectable, nfStatusHistory, nfBirdseye, nfConnectionConfiguration, nfControllerService, nfReportingTask, nfPolicyManagement, nfProcessorConfiguration, nfProcessGroupConfiguration, nfControllerServices, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupPorts, nfPortConfiguration, nfLabelConfiguration, nfProcessorDetails, nfPortDetails, nfConnectionDetails, nfRemoteProcessGroupDetails, nfGoto, nfNgBridge, appCtrl, appConfig, serviceProvider, breadcrumbsCtrl, headerCtrl, flowStatusCtrl, globalMenuCtrl, toolboxCtrl, processorComponent, inputPortComponent, outputPortComponent, processGroupComponent, remoteProcessGroupComponent, funnelComponent, templateComponent, lab
 elComponent, graphControlsCtrl, navigateCtrl, operateCtrl, breadcrumbsDirective, draggableDirective) {
+                return factory($, angular, nfCommon, nfCanvasUtils, nfErrorHandler, nfClient, nfClusterSummary, nfDialog, nfStorage, nfCanvas, nfGraph, nfContextMenu, nfQuickSelect, nfShell, nfSettings, nfActions, nfSnippet, nfQueueListing, nfVariableRegistry, nfComponentState, nfFlowVersion, nfComponentVersion, nfDraggable, nfConnectable, nfStatusHistory, nfBirdseye, nfConnectionConfiguration, nfControllerService, nfReportingTask, nfPolicyManagement, nfProcessorConfiguration, nfProcessGroupConfiguration, nfControllerServices, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupPorts, nfPortConfiguration, nfLabelConfiguration, nfProcessorDetails, nfPortDetails, nfConnectionDetails, nfRemoteProcessGroupDetails, nfGoto, nfNgBridge, appCtrl, appConfig, serviceProvider, breadcrumbsCtrl, headerCtrl, flowStatusCtrl, globalMenuCtrl, toolboxCtrl, processorComponent, inputPortComponent, outputPortComponent, processGroupComponent, remoteProcessGroupComponent, funnelComponent, templateCompo
 nent, labelComponent, graphControlsCtrl, navigateCtrl, operateCtrl, breadcrumbsDirective, draggableDirective);
             });
     } else if (typeof exports === 'object' && typeof module === 'object') {
         module.exports = factory(require('jquery'),
@@ -106,6 +107,7 @@
             require('nf.QueueListing'),
             require('nf.VariableRegistry'),
             require('nf.ComponentState'),
+            require('nf.FlowVersion'),
             require('nf.ComponentVersion'),
             require('nf.Draggable'),
             require('nf.Connectable'),
@@ -170,6 +172,7 @@
             root.nf.QueueListing,
             root.nf.VariableRegistry,
             root.nf.ComponentState,
+            root.nf.FlowVersion,
             root.nf.ComponentVersion,
             root.nf.Draggable,
             root.nf.Connectable,
@@ -214,7 +217,7 @@
             root.nf.ng.BreadcrumbsDirective,
             root.nf.ng.DraggableDirective);
     }
-}(this, function ($, angular, nfCommon, nfCanvasUtils, nfErrorHandler, nfClient, nfClusterSummary, nfDialog, nfStorage, nfCanvas, nfGraph, nfContextMenu, nfQuickSelect, nfShell, nfSettings, nfActions, nfSnippet, nfQueueListing, nfVariableRegistry, nfComponentState, nfComponentVersion, nfDraggable, nfConnectable, nfStatusHistory, nfBirdseye, nfConnectionConfiguration, nfControllerService, nfReportingTask, nfPolicyManagement, nfProcessorConfiguration, nfProcessGroupConfiguration, nfControllerServices, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupPorts, nfPortConfiguration, nfLabelConfiguration, nfProcessorDetails, nfPortDetails, nfConnectionDetails, nfRemoteProcessGroupDetails, nfGoto, nfNgBridge, appCtrl, appConfig, serviceProvider, breadcrumbsCtrl, headerCtrl, flowStatusCtrl, globalMenuCtrl, toolboxCtrl, processorComponent, inputPortComponent, outputPortComponent, processGroupComponent, remoteProcessGroupComponent, funnelComponent, templateComponent, labelComponent, graphC
 ontrolsCtrl, navigateCtrl, operateCtrl, breadcrumbsDirective, draggableDirective) {
+}(this, function ($, angular, nfCommon, nfCanvasUtils, nfErrorHandler, nfClient, nfClusterSummary, nfDialog, nfStorage, nfCanvas, nfGraph, nfContextMenu, nfQuickSelect, nfShell, nfSettings, nfActions, nfSnippet, nfQueueListing, nfVariableRegistry, nfComponentState, nfFlowVersion, nfComponentVersion, nfDraggable, nfConnectable, nfStatusHistory, nfBirdseye, nfConnectionConfiguration, nfControllerService, nfReportingTask, nfPolicyManagement, nfProcessorConfiguration, nfProcessGroupConfiguration, nfControllerServices, nfRemoteProcessGroupConfiguration, nfRemoteProcessGroupPorts, nfPortConfiguration, nfLabelConfiguration, nfProcessorDetails, nfPortDetails, nfConnectionDetails, nfRemoteProcessGroupDetails, nfGoto, nfNgBridge, appCtrl, appConfig, serviceProvider, breadcrumbsCtrl, headerCtrl, flowStatusCtrl, globalMenuCtrl, toolboxCtrl, processorComponent, inputPortComponent, outputPortComponent, processGroupComponent, remoteProcessGroupComponent, funnelComponent, templateComponent, labelCo
 mponent, graphControlsCtrl, navigateCtrl, operateCtrl, breadcrumbsDirective, draggableDirective) {
 
     var config = {
         urls: {
@@ -334,6 +337,7 @@
                     nfCanvas.setManagedAuthorizer(configDetails.supportsManagedAuthorizer);
                     nfCanvas.setConfigurableAuthorizer(configDetails.supportsConfigurableAuthorizer);
                     nfCanvas.setConfigurableUsersAndGroups(configDetails.supportsConfigurableUsersAndGroups);
+                    nfCanvas.setSupportsFlowVersioning(configDetails.supportsFlowVersioning);
 
                     // init nfStorage
                     nfStorage.init();
@@ -352,6 +356,7 @@
                     nfQueueListing.init();
                     nfVariableRegistry.init();
                     nfComponentState.init();
+                    nfFlowVersion.init();
                     nfComponentVersion.init(nfSettings);
 
                     // initialize the component behaviors

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
index 68a918a..9506fef 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
@@ -1820,6 +1820,13 @@
         },
 
         /**
+         * Returns whether this NiFi supports flow versioning.
+         */
+        supportsFlowVersioning: function () {
+            return nfCanvas.supportsFlowVersioning();
+        },
+
+        /**
          * Returns whether the authorizer is managed.
          */
         isManagedAuthorizer: function () {
@@ -1884,7 +1891,7 @@
          *
          * @returns {boolean}   can write
          */
-        canReadFromGroup: function () {
+        canReadCurrentGroup: function () {
             return nfCanvas.canRead();
         },
 
@@ -1893,7 +1900,7 @@
          *
          * @returns {boolean}   can write
          */
-        canWrite: function () {
+        canWriteCurrentGroup: function () {
             return nfCanvas.canWrite();
         },
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/7a0a900a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
index 49ded8c..d0ad4ee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
@@ -85,6 +85,7 @@
     var permissions = null;
     var parentGroupId = null;
     var managedAuthorizer = false;
+    var supportsFlowVersioning = false;
     var configurableAuthorizer = false;
     var configurableUsersAndGroups = false;
     var svg = null;
@@ -909,6 +910,23 @@
         },
 
         /**
+         * Set whether this NiFi supports flow versioning.
+         *
+         * @param bool Whether this NiFi supports flow versioning
+         */
+        setSupportsFlowVersioning: function (bool) {
+            supportsFlowVersioning = bool;
+        },
+
+        /**
+         *
+         * @returns {boolean}
+         */
+        supportsFlowVersioning: function () {
+            return supportsFlowVersioning;
+        },
+
+        /**
          * Set whether the authorizer is configurable.
          *
          * @param bool The boolean value representing whether the authorizer is configurable.