You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2016/04/29 22:32:22 UTC

[09/13] nifi git commit: NIFI-1554: - Populating component entities in the REST API to decouple key fields from the configuration DTOs. - Added initial support for components in UI when access isn't allowed. Formal styling to come later.

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
index b8f1350..2eb7bd6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
@@ -25,12 +25,14 @@ import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
 import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.authorization.user.NiFiUserDetails;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.reporting.ReportingTaskProvider;
-import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
@@ -40,8 +42,6 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
 import org.apache.nifi.web.api.entity.ProcessorEntity;
 import org.apache.nifi.web.api.entity.ReportingTaskEntity;
-import org.apache.nifi.web.security.user.NiFiUserDetails;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.web.util.ClientResponseUtils;
 import org.apache.nifi.web.util.WebUtils;
 import org.slf4j.Logger;
@@ -288,7 +288,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
                 // create the request URL
                 URI requestUrl;
                 try {
-                    String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(id, "UTF-8");
+                    String path = "/nifi-api/processors/" + URLEncoder.encode(id, "UTF-8");
                     requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null);
                 } catch (final URISyntaxException | UnsupportedEncodingException use) {
                     throw new ClusterRequestException(use);
@@ -309,9 +309,9 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
                 if (entity == null) {
                     entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class);
                 }
-                processor = entity.getProcessor();
+                processor = entity.getComponent();
             } else {
-                processor = serviceFacade.getProcessor(id);
+                processor = serviceFacade.getProcessor(id).getComponent();
             }
 
             // return the processor info
@@ -328,7 +328,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
                 // create the request URL
                 URI requestUrl;
                 try {
-                    String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(id, "UTF-8");
+                    String path = "/nifi-api/processors/" + URLEncoder.encode(id, "UTF-8");
                     requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null);
                 } catch (final URISyntaxException | UnsupportedEncodingException use) {
                     throw new ClusterRequestException(use);
@@ -345,7 +345,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
 
                 // create the processor dto
                 ProcessorDTO processorDto = new ProcessorDTO();
-                processorEntity.setProcessor(processorDto);
+                processorEntity.setComponent(processorDto);
                 processorDto.setId(id);
 
                 // create the processor configuration with the given annotation data
@@ -368,7 +368,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
                 if (entity == null) {
                     entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class);
                 }
-                processor = entity.getProcessor();
+                processor = entity.getComponent();
             } else {
                 final ConfigurationSnapshot<ProcessorDTO> response = serviceFacade.setProcessorAnnotationData(revision, id, annotationData);
                 processor = response.getConfiguration();

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
index 158dbfa..6180cee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
@@ -25,18 +25,18 @@ import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
 import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.authorization.user.NiFiUserDetails;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
-import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.entity.ProcessorEntity;
-import org.apache.nifi.web.security.user.NiFiUserDetails;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.web.util.ClientResponseUtils;
 import org.apache.nifi.web.util.WebUtils;
 import org.slf4j.Logger;
@@ -166,7 +166,7 @@ public class StandardNiFiWebContext implements NiFiWebContext {
             // create the request URL
             URI requestUrl;
             try {
-                String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(processorId, "UTF-8");
+                String path = "/nifi-api/processors/" + URLEncoder.encode(processorId, "UTF-8");
                 requestUrl = new URI(config.getScheme(), null, "localhost", 0, path, null, null);
             } catch (final URISyntaxException | UnsupportedEncodingException use) {
                 throw new ClusterRequestException(use);
@@ -200,9 +200,9 @@ public class StandardNiFiWebContext implements NiFiWebContext {
             if (entity == null) {
                 entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class);
             }
-            processor = entity.getProcessor();
+            processor = entity.getComponent();
         } else {
-            processor = serviceFacade.getProcessor(processorId);
+            processor = serviceFacade.getProcessor(processorId).getComponent();
         }
 
         // return the processor info
@@ -233,7 +233,7 @@ public class StandardNiFiWebContext implements NiFiWebContext {
             // create the request URL
             URI requestUrl;
             try {
-                String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(processorId, "UTF-8");
+                String path = "/nifi-api/processors/" + URLEncoder.encode(processorId, "UTF-8");
                 requestUrl = new URI(config.getScheme(), null, "localhost", 0, path, null, null);
             } catch (final URISyntaxException | UnsupportedEncodingException use) {
                 throw new ClusterRequestException(use);
@@ -250,7 +250,7 @@ public class StandardNiFiWebContext implements NiFiWebContext {
 
             // create the processor dto
             ProcessorDTO processorDto = new ProcessorDTO();
-            processorEntity.setProcessor(processorDto);
+            processorEntity.setComponent(processorDto);
             processorDto.setId(processorId);
 
             // create the processor configuration with the given annotation data

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/UpdateResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/UpdateResult.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/UpdateResult.java
new file mode 100644
index 0000000..66247e3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/UpdateResult.java
@@ -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.
+ */
+package org.apache.nifi.web;
+
+/**
+ * Result from an update operation.
+ */
+@SuppressWarnings("serial")
+public class UpdateResult<T>  {
+
+    private final T result;
+    private final boolean isNew;
+
+    public UpdateResult(T result, boolean isNew) {
+        this.result = result;
+        this.isNew = isNew;
+    }
+
+    public T getResult() {
+        return result;
+    }
+
+    public boolean isNew() {
+        return isNew;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.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/AccessResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
index bb8e701..22ebf29 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/AccessResource.java
@@ -29,8 +29,10 @@ import org.apache.nifi.authentication.LoginCredentials;
 import org.apache.nifi.authentication.LoginIdentityProvider;
 import org.apache.nifi.authentication.exception.IdentityAccessException;
 import org.apache.nifi.authentication.exception.InvalidLoginCredentialsException;
+import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.security.util.CertificateUtils;
-import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.AccessConfigurationDTO;
@@ -48,12 +50,10 @@ import org.apache.nifi.web.security.kerberos.KerberosService;
 import org.apache.nifi.web.security.otp.OtpService;
 import org.apache.nifi.web.security.token.LoginAuthenticationToken;
 import org.apache.nifi.web.security.token.OtpAuthenticationToken;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.web.security.x509.X509CertificateExtractor;
 import org.apache.nifi.web.security.x509.X509IdentityProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.springframework.security.access.AccessDeniedException;
 import org.springframework.security.authentication.AccountStatusException;
 import org.springframework.security.authentication.AuthenticationServiceException;
 import org.springframework.security.core.Authentication;

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.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/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index 966aa29..8026400 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -26,6 +26,7 @@ import org.apache.commons.lang3.builder.ToStringStyle;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
+import org.apache.nifi.authorization.user.NiFiUserDetails;
 import org.apache.nifi.cluster.context.ClusterContext;
 import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@@ -33,7 +34,6 @@ import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
-import org.apache.nifi.web.security.user.NiFiUserDetails;
 import org.apache.nifi.web.util.WebUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.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/ClusterResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java
index 5266632..1dce576 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java
@@ -24,34 +24,23 @@ import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.cluster.node.Node;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.IllegalClusterResourceRequestException;
 import org.apache.nifi.web.NiFiServiceFacade;
-import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.ClusterDTO;
 import org.apache.nifi.web.api.dto.NodeDTO;
-import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.search.NodeSearchResultDTO;
 import org.apache.nifi.web.api.entity.ClusterEntity;
 import org.apache.nifi.web.api.entity.ClusterSearchResultsEntity;
-import org.apache.nifi.web.api.entity.ProcessorEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.nifi.web.api.request.LongParameter;
 
-import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DefaultValue;
-import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
 import javax.ws.rs.HEAD;
-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;
@@ -233,218 +222,6 @@ public class ClusterResource extends ApplicationResource {
         throw new IllegalClusterResourceRequestException("Only a cluster manager can process the request.");
     }
 
-    /**
-     * Gets the processor.
-     *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
-     * @param id The id of the processor
-     * @return A processorEntity
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_XML, MediaType.APPLICATION_JSON})
-    @Path("/processors/{id}")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Gets the specified processor",
-            response = ProcessorEntity.class,
-            authorizations = {
-                @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
-                @Authorization(value = "DFM", type = "ROLE_DFM"),
-                @Authorization(value = "Admin", type = "ROLE_ADMIN")
-            }
-    )
-    @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."),
-                @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
-                @ApiResponse(code = 404, message = "The specified resource could not be found."),
-                @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 getProcessor(
-            @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
-                    value = "The processor id.",
-                    required = true
-            )
-            @PathParam("id") String id) {
-
-        if (!properties.isClusterManager()) {
-
-            final ProcessorDTO dto = serviceFacade.getProcessor(id);
-
-            // create the revision
-            RevisionDTO revision = new RevisionDTO();
-            revision.setClientId(clientId.getClientId());
-
-            // create entity
-            final ProcessorEntity entity = new ProcessorEntity();
-            entity.setProcessor(dto);
-            entity.setRevision(revision);
-
-            // generate the response
-            return generateOkResponse(entity).build();
-        }
-
-        throw new IllegalClusterResourceRequestException("Only a node can process the request.");
-    }
-
-    /**
-     * Updates the processors annotation data.
-     *
-     * @param httpServletRequest request
-     * @param version The revision is used to verify the client is working with the latest version of the flow.
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
-     * @param processorId The id of the processor.
-     * @param annotationData The annotation data to set.
-     * @return A processorEntity.
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_XML, MediaType.APPLICATION_JSON})
-    @Path("/processors/{id}")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')")
-    public Response updateProcessor(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @PathParam("id") String processorId,
-            @FormParam("annotationData") String annotationData) {
-
-        if (!properties.isClusterManager()) {
-
-            // create the processor configuration with the given annotation data
-            ProcessorConfigDTO configDto = new ProcessorConfigDTO();
-            configDto.setAnnotationData(annotationData);
-
-            // create the processor dto
-            ProcessorDTO processorDto = new ProcessorDTO();
-            processorDto.setId(processorId);
-            processorDto.setConfig(configDto);
-
-            // create the revision
-            RevisionDTO revision = new RevisionDTO();
-            revision.setClientId(clientId.getClientId());
-
-            if (version != null) {
-                revision.setVersion(version.getLong());
-            }
-
-            // create the processor entity
-            ProcessorEntity processorEntity = new ProcessorEntity();
-            processorEntity.setRevision(revision);
-            processorEntity.setProcessor(processorDto);
-
-            return updateProcessor(httpServletRequest, processorId, processorEntity);
-        }
-
-        throw new IllegalClusterResourceRequestException("Only a node can process the request.");
-    }
-
-    /**
-     * Updates the processors annotation data.
-     *
-     * @param httpServletRequest request
-     * @param processorId The id of the processor.
-     * @param processorEntity A processorEntity.
-     * @return A processorEntity.
-     */
-    @PUT
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_XML, MediaType.APPLICATION_JSON})
-    @Path("/processors/{id}")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')")
-    @ApiOperation(
-            value = "Updates processor annotation data",
-            response = ProcessorEntity.class,
-            authorizations = {
-                @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
-                @Authorization(value = "DFM", type = "ROLE_DFM"),
-                @Authorization(value = "Admin", type = "ROLE_ADMIN")
-            }
-    )
-    @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."),
-                @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
-                @ApiResponse(code = 404, message = "The specified resource could not be found."),
-                @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 updateProcessor(
-            @Context HttpServletRequest httpServletRequest,
-            @ApiParam(
-                    value = "The processor id.",
-                    required = true
-            )
-            @PathParam("id") final String processorId,
-            @ApiParam(
-                    value = "The processor configuration details. The only configuration that will be honored at this endpoint is the processor annontation data.",
-                    required = true
-            )
-            final ProcessorEntity processorEntity) {
-
-        if (!properties.isClusterManager()) {
-
-            if (processorEntity == null || processorEntity.getProcessor() == null) {
-                throw new IllegalArgumentException("Processor details must be specified.");
-            }
-
-            if (processorEntity.getRevision() == null) {
-                throw new IllegalArgumentException("Revision must be specified.");
-            }
-
-            // ensure the same id is being used
-            ProcessorDTO requestProcessorDTO = processorEntity.getProcessor();
-            if (!processorId.equals(requestProcessorDTO.getId())) {
-                throw new IllegalArgumentException(String.format("The processor id (%s) in the request body does "
-                        + "not equal the processor id of the requested resource (%s).", requestProcessorDTO.getId(), processorId));
-            }
-
-            // get the processor configuration
-            ProcessorConfigDTO config = requestProcessorDTO.getConfig();
-            if (config == null) {
-                throw new IllegalArgumentException("Processor configuration must be specified.");
-            }
-
-            // handle expects request (usually from the cluster manager)
-            final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
-            if (expects != null) {
-                serviceFacade.verifyUpdateProcessor(requestProcessorDTO);
-                return generateContinueResponse().build();
-            }
-
-            // update the processor
-            final RevisionDTO revision = processorEntity.getRevision();
-            final ConfigurationSnapshot<ProcessorDTO> controllerResponse = serviceFacade.setProcessorAnnotationData(
-                    new Revision(revision.getVersion(), revision.getClientId()), processorId, config.getAnnotationData());
-
-            // get the processor dto
-            final ProcessorDTO responseProcessorDTO = controllerResponse.getConfiguration();
-
-            // update the revision
-            RevisionDTO updatedRevision = new RevisionDTO();
-            updatedRevision.setClientId(revision.getClientId());
-            updatedRevision.setVersion(controllerResponse.getVersion());
-
-            // generate the response entity
-            final ProcessorEntity entity = new ProcessorEntity();
-            entity.setRevision(updatedRevision);
-            entity.setProcessor(responseProcessorDTO);
-
-            return generateOkResponse(entity).build();
-        }
-
-        throw new IllegalClusterResourceRequestException("Only a node can process the request.");
-    }
-
     // setters
     public void setServiceFacade(NiFiServiceFacade serviceFacade) {
         this.serviceFacade = serviceFacade;

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.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/ConnectionResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
index 7d5df50..d02c1ef 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
@@ -25,9 +25,9 @@ import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.UpdateResult;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
 import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
 import org.apache.nifi.web.api.dto.ListingRequestDTO;
@@ -72,6 +72,32 @@ public class ConnectionResource extends ApplicationResource {
     /**
      * Populate the URIs for the specified connections.
      *
+     * @param connectionEntities connections
+     * @return dtos
+     */
+    public Set<ConnectionEntity> populateRemainingConnectionEntitiesContent(Set<ConnectionEntity> connectionEntities) {
+        for (ConnectionEntity connectionEntity : connectionEntities) {
+            populateRemainingConnectionEntityContent(connectionEntity);
+        }
+        return connectionEntities;
+    }
+
+    /**
+     * Populate the URIs for the specified connection.
+     *
+     * @param connectionEntity connection
+     * @return dto
+     */
+    public ConnectionEntity populateRemainingConnectionEntityContent(ConnectionEntity connectionEntity) {
+        if (connectionEntity.getComponent() != null) {
+            populateRemainingConnectionContent(connectionEntity.getComponent());
+        }
+        return connectionEntity;
+    }
+
+    /**
+     * Populate the URIs for the specified connections.
+     *
      * @param connections connections
      * @return dtos
      */
@@ -129,7 +155,6 @@ public class ConnectionResource extends ApplicationResource {
     /**
      * Retrieves the specified connection.
      *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the connection.
      * @return A connectionEntity.
      */
@@ -158,11 +183,6 @@ public class ConnectionResource extends ApplicationResource {
     )
     public Response getConnection(
             @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
                     value = "The connection id.",
                     required = true
             )
@@ -174,16 +194,8 @@ public class ConnectionResource extends ApplicationResource {
         }
 
         // get the specified relationship
-        ConnectionDTO connection = serviceFacade.getConnection(id);
-
-        // create the revision
-        RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        ConnectionEntity entity = new ConnectionEntity();
-        entity.setRevision(revision);
-        entity.setConnection(populateRemainingConnectionContent(connection));
+        ConnectionEntity entity = serviceFacade.getConnection(id);
+        populateRemainingConnectionEntityContent(entity);
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -230,7 +242,7 @@ public class ConnectionResource extends ApplicationResource {
                     required = true
             ) ConnectionEntity connectionEntity) {
 
-        if (connectionEntity == null || connectionEntity.getConnection() == null) {
+        if (connectionEntity == null || connectionEntity.getComponent() == null) {
             throw new IllegalArgumentException("Connection details must be specified.");
         }
 
@@ -239,7 +251,7 @@ public class ConnectionResource extends ApplicationResource {
         }
 
         // ensure the ids are the same
-        final ConnectionDTO connection = connectionEntity.getConnection();
+        final ConnectionDTO connection = connectionEntity.getComponent();
         if (!id.equals(connection.getId())) {
             throw new IllegalArgumentException(String.format("The connection id "
                     + "(%s) in the request body does not equal the connection id of the "
@@ -265,26 +277,14 @@ public class ConnectionResource extends ApplicationResource {
 
         // update the relationship target
         final RevisionDTO revision = connectionEntity.getRevision();
-        final ConfigurationSnapshot<ConnectionDTO> controllerResponse = serviceFacade.updateConnection(
-                new Revision(revision.getVersion(), revision.getClientId()), connection);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
+        final UpdateResult<ConnectionEntity> updateResult = serviceFacade.updateConnection(new Revision(revision.getVersion(), revision.getClientId()), connection);
 
-        // marshall the target and add the source processor
-        final ConnectionDTO connectionDTO = controllerResponse.getConfiguration();
-        populateRemainingConnectionContent(connectionDTO);
-
-        // create the response entity
-        ConnectionEntity entity = new ConnectionEntity();
-        entity.setRevision(updatedRevision);
-        entity.setConnection(connectionDTO);
+        final ConnectionEntity entity = updateResult.getResult();
+        populateRemainingConnectionEntityContent(entity);
 
         // generate the response
-        if (controllerResponse.isNew()) {
-            return clusterContext(generateCreatedResponse(URI.create(connectionDTO.getUri()), entity)).build();
+        if (updateResult.isNew()) {
+            return clusterContext(generateCreatedResponse(URI.create(entity.getUri()), entity)).build();
         } else {
             return clusterContext(generateOkResponse(entity)).build();
         }
@@ -357,16 +357,7 @@ public class ConnectionResource extends ApplicationResource {
         }
 
         // delete the connection
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteConnection(new Revision(clientVersion, clientId.getClientId()), id);
-
-        // create the revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(clientId.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // create the response entity
-        final ConnectionEntity entity = new ConnectionEntity();
-        entity.setRevision(updatedRevision);
+        final ConnectionEntity entity = serviceFacade.deleteConnection(new Revision(clientVersion, clientId.getClientId()), id);
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.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/ControllerResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 10f4456..8add2bb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -24,6 +24,7 @@ import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.context.ClusterContext;
 import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
 import org.apache.nifi.cluster.manager.NodeResponse;
@@ -31,7 +32,7 @@ import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.cluster.node.Node;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
@@ -50,7 +51,6 @@ import org.apache.nifi.web.api.entity.ProcessGroupEntity;
 import org.apache.nifi.web.api.entity.ReportingTaskEntity;
 import org.apache.nifi.web.api.entity.ReportingTasksEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.web.util.Availability;
 
 import javax.servlet.http.HttpServletRequest;

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.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/ControllerServiceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
index 849d9f2..3cfae3a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
@@ -152,9 +152,6 @@ public class ControllerServiceResource extends ApplicationResource {
     /**
      * Retrieves the specified controller service.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
      * @param availability Whether the controller service is available on the
      * NCM only (ncm) or on the nodes only (node). If this instance is not
      * clustered all services should use the node availability.
@@ -186,11 +183,6 @@ public class ControllerServiceResource extends ApplicationResource {
     )
     public Response getControllerService(
             @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
                     value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
                     allowableValues = "NCM, NODE",
                     required = true
@@ -212,13 +204,8 @@ public class ControllerServiceResource extends ApplicationResource {
         // get the controller service
         final ControllerServiceDTO controllerService = serviceFacade.getControllerService(id);
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
         // create the response entity
         final ControllerServiceEntity entity = new ControllerServiceEntity();
-        entity.setRevision(revision);
         entity.setControllerService(populateRemainingControllerServiceContent(availability, controllerService));
 
         return clusterContext(generateOkResponse(entity)).build();
@@ -227,9 +214,6 @@ public class ControllerServiceResource extends ApplicationResource {
     /**
      * Returns the descriptor for the specified property.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
      * @param availability avail
      * @param id The id of the controller service.
      * @param propertyName The property
@@ -260,11 +244,6 @@ public class ControllerServiceResource extends ApplicationResource {
     )
     public Response getPropertyDescriptor(
             @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
                     value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
                     allowableValues = "NCM, NODE",
                     required = true
@@ -296,13 +275,8 @@ public class ControllerServiceResource extends ApplicationResource {
         // get the property descriptor
         final PropertyDescriptorDTO descriptor = serviceFacade.getControllerServicePropertyDescriptor(id, propertyName);
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
         // generate the response entity
         final PropertyDescriptorEntity entity = new PropertyDescriptorEntity();
-        entity.setRevision(revision);
         entity.setPropertyDescriptor(descriptor);
 
         // generate the response
@@ -312,7 +286,6 @@ public class ControllerServiceResource extends ApplicationResource {
     /**
      * Gets the state for a controller service.
      *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param availability Whether the controller service is available on the
      * NCM only (ncm) or on the nodes only (node). If this instance is not
      * clustered all services should use the node availability.
@@ -342,11 +315,6 @@ public class ControllerServiceResource extends ApplicationResource {
     )
     public Response getState(
         @ApiParam(
-            value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-            required = false
-        )
-        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-        @ApiParam(
             value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
             allowableValues = "NCM, NODE",
             required = true
@@ -368,13 +336,8 @@ public class ControllerServiceResource extends ApplicationResource {
         // get the component state
         final ComponentStateDTO state = serviceFacade.getControllerServiceState(id);
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
         // generate the response entity
         final ComponentStateEntity entity = new ComponentStateEntity();
-        entity.setRevision(revision);
         entity.setComponentState(state);
 
         // generate the response
@@ -465,9 +428,6 @@ public class ControllerServiceResource extends ApplicationResource {
     /**
      * Retrieves the references of the specified controller service.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
      * @param availability Whether the controller service is available on the
      * NCM only (ncm) or on the nodes only (node). If this instance is not
      * clustered all services should use the node availability.
@@ -499,11 +459,6 @@ public class ControllerServiceResource extends ApplicationResource {
     )
     public Response getControllerServiceReferences(
             @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
                     value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
                     allowableValues = "NCM, NODE",
                     required = true
@@ -525,13 +480,8 @@ public class ControllerServiceResource extends ApplicationResource {
         // get the controller service
         final Set<ControllerServiceReferencingComponentDTO> controllerServiceReferences = serviceFacade.getControllerServiceReferencingComponents(id);
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
         // create the response entity
         final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity();
-        entity.setRevision(revision);
         entity.setControllerServiceReferencingComponents(controllerServiceReferences);
 
         return clusterContext(generateOkResponse(entity)).build();

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.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/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index e1beecc..337cb67 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -24,20 +24,32 @@ import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.AuthorizationRequest;
+import org.apache.nifi.authorization.AuthorizationResult;
+import org.apache.nifi.authorization.AuthorizationResult.Result;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.ResourceFactory;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.cluster.node.Node;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.dto.AboutDTO;
 import org.apache.nifi.web.api.dto.BannerDTO;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
 import org.apache.nifi.web.api.dto.BulletinQueryDTO;
 import org.apache.nifi.web.api.dto.ControllerConfigurationDTO;
+import org.apache.nifi.web.api.dto.ProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
 import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
@@ -58,6 +70,8 @@ import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.IdentityEntity;
 import org.apache.nifi.web.api.entity.PortStatusEntity;
 import org.apache.nifi.web.api.entity.PrioritizerTypesEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity;
 import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
 import org.apache.nifi.web.api.entity.ProcessorTypesEntity;
@@ -69,7 +83,6 @@ import org.apache.nifi.web.api.request.BulletinBoardPatternParameter;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.IntegerParameter;
 import org.apache.nifi.web.api.request.LongParameter;
-import org.apache.nifi.web.security.user.NiFiUserUtils;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DefaultValue;
@@ -92,7 +105,7 @@ import java.util.Set;
 @Path("/flow")
 @Api(
     value = "/flow",
-    description = "Endpoint for accessing the flow structure and component statuses."
+    description = "Endpoint for accessing the flow structure and component status."
 )
 public class FlowResource extends ApplicationResource {
 
@@ -101,10 +114,164 @@ public class FlowResource extends ApplicationResource {
     private NiFiServiceFacade serviceFacade;
     private WebClusterManager clusterManager;
     private NiFiProperties properties;
+    private Authorizer authorizer;
 
     @Context
     private ResourceContext resourceContext;
 
+    private ProcessorResource processorResource;
+    private InputPortResource inputPortResource;
+    private OutputPortResource outputPortResource;
+    private FunnelResource funnelResource;
+    private LabelResource labelResource;
+    private RemoteProcessGroupResource remoteProcessGroupResource;
+    private ConnectionResource connectionResource;
+    private TemplateResource templateResource;
+    private ProcessGroupResource processGroupResource;
+    private ControllerServiceResource controllerServiceResource;
+
+    /**
+     * Populates the remaining fields in the specified process group.
+     *
+     * @param flow group
+     * @return group dto
+     */
+    private ProcessGroupFlowDTO populateRemainingFlowContent(ProcessGroupFlowDTO flow) {
+        FlowDTO flowStructure = flow.getFlow();
+
+        // populate the remaining fields for the processors, connections, process group refs, remote process groups, and labels if appropriate
+        if (flowStructure != null) {
+            populateRemainingFlowStructure(flowStructure);
+        }
+
+        // set the process group uri
+        flow.setUri(generateResourceUri("flow", "process-groups",  flow.getId()));
+
+        return flow;
+    }
+
+    /**
+     * Populates the remaining content of the specified snippet.
+     */
+    private FlowDTO populateRemainingFlowStructure(FlowDTO flowStructure) {
+        processorResource.populateRemainingProcessorEntitiesContent(flowStructure.getProcessors());
+        connectionResource.populateRemainingConnectionEntitiesContent(flowStructure.getConnections());
+        inputPortResource.populateRemainingInputPortEntitiesContent(flowStructure.getInputPorts());
+        outputPortResource.populateRemainingOutputPortEntitiesContent(flowStructure.getOutputPorts());
+        remoteProcessGroupResource.populateRemainingRemoteProcessGroupEntitiesContent(flowStructure.getRemoteProcessGroups());
+        funnelResource.populateRemainingFunnelEntitiesContent(flowStructure.getFunnels());
+        labelResource.populateRemainingLabelEntitiesContent(flowStructure.getLabels());
+        processGroupResource.populateRemainingProcessGroupEntitiesContent(flowStructure.getProcessGroups());
+
+        // go through each process group child and populate its uri
+        for (final ProcessGroupEntity processGroupEntity : flowStructure.getProcessGroups()) {
+            final ProcessGroupDTO processGroup = processGroupEntity.getComponent();
+            if (processGroup != null) {
+                processGroup.setContents(null);
+            }
+        }
+
+        return flowStructure;
+    }
+
+    private void authorizeFlow() {
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        final AuthorizationRequest request = new AuthorizationRequest.Builder()
+            .resource(ResourceFactory.getFlowResource())
+            .identity(user.getIdentity())
+            .anonymous(user.isAnonymous())
+            .accessAttempt(true)
+            .action(RequestAction.READ)
+            .build();
+
+        final AuthorizationResult result = authorizer.authorize(request);
+        if (!Result.Approved.equals(result.getResult())) {
+            final String message = StringUtils.isNotBlank(result.getExplanation()) ? result.getExplanation() : "Access is denied";
+            throw new AccessDeniedException(message);
+        }
+    }
+
+    // ----
+    // flow
+    // ----
+
+    /**
+     * Retrieves the contents of the specified group.
+     *
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
+     * @param recursive Optional recursive flag that defaults to false. If set to true, all descendent groups and their content will be included if the verbose flag is also set to true.
+     * @param groupId The id of the process group.
+     * @return A processGroupEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("process-groups/{id}")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @ApiOperation(
+        value = "Gets a process group",
+        response = ProcessGroupEntity.class,
+        authorizations = {
+            @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
+            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
+            @Authorization(value = "Administrator", type = "ROLE_ADMIN")
+        }
+    )
+    @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."),
+            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+            @ApiResponse(code = 404, message = "The specified resource could not be found."),
+            @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 getFlow(
+        @ApiParam(
+            value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
+            required = false
+        )
+        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
+        @ApiParam(
+            value = "The process group id.",
+            required = false
+        )
+        @PathParam("id") String groupId,
+        @ApiParam(
+            value = "Whether the response should contain all encapsulated components or just the immediate children.",
+            required = false
+        )
+        @QueryParam("recursive") @DefaultValue(RECURSIVE) Boolean recursive) {
+
+        authorizeFlow();
+
+        // replicate if cluster manager
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
+        }
+
+        // get this process group contents
+        final ConfigurationSnapshot<ProcessGroupFlowDTO> controllerResponse = serviceFacade.getProcessGroupFlow(groupId, recursive);
+        final ProcessGroupFlowDTO flow = controllerResponse.getConfiguration();
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+        revision.setVersion(controllerResponse.getVersion());
+
+        // create the response entity
+        final ProcessGroupFlowEntity processGroupEntity = new ProcessGroupFlowEntity();
+        processGroupEntity.setRevision(revision);
+        processGroupEntity.setProcessGroupFlow(populateRemainingFlowContent(flow));
+
+        return clusterContext(generateOkResponse(processGroupEntity)).build();
+    }
+
+    // ------
+    // search
+    // ------
+
     /**
      * Performs a search request in this flow.
      *
@@ -134,6 +301,8 @@ public class FlowResource extends ApplicationResource {
             }
     )
     public Response searchFlow(@QueryParam("q") @DefaultValue(StringUtils.EMPTY) String value) {
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -186,6 +355,8 @@ public class FlowResource extends ApplicationResource {
             }
     )
     public Response getRevision() {
+        authorizeFlow();
+
         // create the current revision
         final RevisionDTO revision = serviceFacade.getRevision();
 
@@ -232,6 +403,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
 
+        authorizeFlow();
+
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
         }
@@ -327,6 +500,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
 
+        authorizeFlow();
+
         // get the banner from the properties - will come from the NCM when clustered
         final String bannerText = properties.getBannerText();
 
@@ -383,6 +558,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -442,6 +619,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam("serviceType") String serviceType) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -495,6 +674,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -548,6 +729,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -601,6 +784,8 @@ public class FlowResource extends ApplicationResource {
             )
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -708,6 +893,8 @@ public class FlowResource extends ApplicationResource {
         )
         @QueryParam("limit") IntegerParameter limit) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -807,6 +994,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // ensure a valid request
         if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
             throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
@@ -907,6 +1096,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // ensure a valid request
         if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
             throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
@@ -1007,6 +1198,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // ensure a valid request
         if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
             throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
@@ -1107,6 +1300,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // ensure a valid request
         if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
             throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
@@ -1216,6 +1411,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String groupId) {
 
+        authorizeFlow();
+
         // ensure a valid request
         if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
             throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
@@ -1337,6 +1534,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // ensure a valid request
         if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
             throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
@@ -1431,6 +1630,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -1490,6 +1691,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String groupId) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -1553,6 +1756,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -1616,6 +1821,8 @@ public class FlowResource extends ApplicationResource {
         )
         @PathParam("id") String id) {
 
+        authorizeFlow();
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@@ -1646,6 +1853,50 @@ public class FlowResource extends ApplicationResource {
         this.clusterManager = clusterManager;
     }
 
+    public void setProcessorResource(ProcessorResource processorResource) {
+        this.processorResource = processorResource;
+    }
+
+    public void setInputPortResource(InputPortResource inputPortResource) {
+        this.inputPortResource = inputPortResource;
+    }
+
+    public void setOutputPortResource(OutputPortResource outputPortResource) {
+        this.outputPortResource = outputPortResource;
+    }
+
+    public void setFunnelResource(FunnelResource funnelResource) {
+        this.funnelResource = funnelResource;
+    }
+
+    public void setLabelResource(LabelResource labelResource) {
+        this.labelResource = labelResource;
+    }
+
+    public void setRemoteProcessGroupResource(RemoteProcessGroupResource remoteProcessGroupResource) {
+        this.remoteProcessGroupResource = remoteProcessGroupResource;
+    }
+
+    public void setConnectionResource(ConnectionResource connectionResource) {
+        this.connectionResource = connectionResource;
+    }
+
+    public void setTemplateResource(TemplateResource templateResource) {
+        this.templateResource = templateResource;
+    }
+
+    public void setProcessGroupResource(ProcessGroupResource processGroupResource) {
+        this.processGroupResource = processGroupResource;
+    }
+
+    public void setControllerServiceResource(ControllerServiceResource controllerServiceResource) {
+        this.controllerServiceResource = controllerServiceResource;
+    }
+
+    public void setAuthorizer(Authorizer authorizer) {
+        this.authorizer = authorizer;
+    }
+
     public void setProperties(NiFiProperties properties) {
         this.properties = properties;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.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/FunnelResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java
index 1c4a0ea..b3a050f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java
@@ -25,9 +25,9 @@ import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.UpdateResult;
 import org.apache.nifi.web.api.dto.FunnelDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.entity.FunnelEntity;
@@ -74,6 +74,32 @@ public class FunnelResource extends ApplicationResource {
     /**
      * Populates the uri for the specified funnels.
      *
+     * @param funnelEntities funnels
+     * @return funnels
+     */
+    public Set<FunnelEntity> populateRemainingFunnelEntitiesContent(Set<FunnelEntity> funnelEntities) {
+        for (FunnelEntity funnelEntity : funnelEntities) {
+            populateRemainingFunnelEntityContent(funnelEntity);
+        }
+        return funnelEntities;
+    }
+
+    /**
+     * Populates the uri for the specified funnel.
+     *
+     * @param funnelEntity funnel
+     * @return funnel
+     */
+    public FunnelEntity populateRemainingFunnelEntityContent(FunnelEntity funnelEntity) {
+        if (funnelEntity.getComponent() != null) {
+            populateRemainingFunnelContent(funnelEntity.getComponent());
+        }
+        return funnelEntity;
+    }
+
+    /**
+     * Populates the uri for the specified funnels.
+     *
      * @param funnels funnels
      * @return funnels
      */
@@ -96,9 +122,6 @@ public class FunnelResource extends ApplicationResource {
     /**
      * Retrieves the specified funnel.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
      * @param id The id of the funnel to retrieve
      * @return A funnelEntity.
      */
@@ -127,11 +150,6 @@ public class FunnelResource extends ApplicationResource {
     )
     public Response getFunnel(
             @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
                     value = "The funnel id.",
                     required = true
             )
@@ -143,16 +161,8 @@ public class FunnelResource extends ApplicationResource {
         }
 
         // get the funnel
-        final FunnelDTO funnel = serviceFacade.getFunnel(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        final FunnelEntity entity = new FunnelEntity();
-        entity.setRevision(revision);
-        entity.setFunnel(populateRemainingFunnelContent(funnel));
+        final FunnelEntity entity = serviceFacade.getFunnel(id);
+        populateRemainingFunnelEntityContent(entity);
 
         return clusterContext(generateOkResponse(entity)).build();
     }
@@ -198,7 +208,7 @@ public class FunnelResource extends ApplicationResource {
                     required = true
             ) FunnelEntity funnelEntity) {
 
-        if (funnelEntity == null || funnelEntity.getFunnel() == null) {
+        if (funnelEntity == null || funnelEntity.getComponent() == null) {
             throw new IllegalArgumentException("Funnel details must be specified.");
         }
 
@@ -207,7 +217,7 @@ public class FunnelResource extends ApplicationResource {
         }
 
         // ensure the ids are the same
-        final FunnelDTO requestFunnelDTO = funnelEntity.getFunnel();
+        final FunnelDTO requestFunnelDTO = funnelEntity.getComponent();
         if (!id.equals(requestFunnelDTO.getId())) {
             throw new IllegalArgumentException(String.format("The funnel id (%s) in the request body does not equal the "
                     + "funnel id of the requested resource (%s).", requestFunnelDTO.getId(), id));
@@ -231,25 +241,15 @@ public class FunnelResource extends ApplicationResource {
 
         // update the funnel
         final RevisionDTO revision = funnelEntity.getRevision();
-        final ConfigurationSnapshot<FunnelDTO> controllerResponse = serviceFacade.updateFunnel(
+        final UpdateResult<FunnelEntity> updateResult = serviceFacade.updateFunnel(
                 new Revision(revision.getVersion(), revision.getClientId()), requestFunnelDTO);
 
         // get the results
-        final FunnelDTO responseFunnelDTO = controllerResponse.getConfiguration();
-        populateRemainingFunnelContent(responseFunnelDTO);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // build the response entity
-        final FunnelEntity entity = new FunnelEntity();
-        entity.setRevision(updatedRevision);
-        entity.setFunnel(responseFunnelDTO);
+        final FunnelEntity entity = updateResult.getResult();
+        populateRemainingFunnelEntityContent(entity);
 
-        if (controllerResponse.isNew()) {
-            return clusterContext(generateCreatedResponse(URI.create(responseFunnelDTO.getUri()), entity)).build();
+        if (updateResult.isNew()) {
+            return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
         } else {
             return clusterContext(generateOkResponse(entity)).build();
         }
@@ -325,17 +325,7 @@ public class FunnelResource extends ApplicationResource {
         }
 
         // delete the specified funnel
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteFunnel(new Revision(clientVersion, clientId.getClientId()), id);
-
-        // get the updated revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        revision.setVersion(controllerResponse.getVersion());
-
-        // build the response entity
-        final FunnelEntity entity = new FunnelEntity();
-        entity.setRevision(revision);
-
+        final FunnelEntity entity = serviceFacade.deleteFunnel(new Revision(clientVersion, clientId.getClientId()), id);
         return clusterContext(generateOkResponse(entity)).build();
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/ff98d823/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.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/InputPortResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java
index 1d0629f..a378ec3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java
@@ -25,12 +25,12 @@ import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.UpdateResult;
 import org.apache.nifi.web.api.dto.PortDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.entity.InputPortEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 
@@ -70,6 +70,32 @@ public class InputPortResource extends ApplicationResource {
     /**
      * Populates the uri for the specified input ports.
      *
+     * @param inputPortEntites ports
+     * @return ports
+     */
+    public Set<PortEntity> populateRemainingInputPortEntitiesContent(Set<PortEntity> inputPortEntites) {
+        for (PortEntity inputPortEntity : inputPortEntites) {
+            populateRemainingInputPortEntityContent(inputPortEntity);
+        }
+        return inputPortEntites;
+    }
+
+        /**
+         * Populates the uri for the specified input port.
+         *
+         * @param inputPortEntity port
+         * @return ports
+         */
+    public PortEntity populateRemainingInputPortEntityContent(PortEntity inputPortEntity) {
+        if (inputPortEntity.getComponent() != null) {
+            populateRemainingInputPortContent(inputPortEntity.getComponent());
+        }
+        return inputPortEntity;
+    }
+
+    /**
+     * Populates the uri for the specified input ports.
+     *
      * @param inputPorts ports
      * @return ports
      */
@@ -92,7 +118,6 @@ public class InputPortResource extends ApplicationResource {
     /**
      * Retrieves the specified input port.
      *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the input port to retrieve
      * @return A inputPortEntity.
      */
@@ -103,7 +128,7 @@ public class InputPortResource extends ApplicationResource {
     // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets an input port",
-            response = InputPortEntity.class,
+            response = PortEntity.class,
             authorizations = {
                 @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
                 @Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
@@ -121,11 +146,6 @@ public class InputPortResource extends ApplicationResource {
     )
     public Response getInputPort(
             @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
                     value = "The input port id.",
                     required = true
             )
@@ -137,16 +157,8 @@ public class InputPortResource extends ApplicationResource {
         }
 
         // get the port
-        final PortDTO port = serviceFacade.getInputPort(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        final InputPortEntity entity = new InputPortEntity();
-        entity.setRevision(revision);
-        entity.setInputPort(populateRemainingInputPortContent(port));
+        final PortEntity entity = serviceFacade.getInputPort(id);
+        populateRemainingInputPortEntityContent(entity);
 
         return clusterContext(generateOkResponse(entity)).build();
     }
@@ -166,7 +178,7 @@ public class InputPortResource extends ApplicationResource {
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates an input port",
-            response = InputPortEntity.class,
+            response = PortEntity.class,
             authorizations = {
                 @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
             }
@@ -190,9 +202,9 @@ public class InputPortResource extends ApplicationResource {
             @ApiParam(
                     value = "The input port configuration details.",
                     required = true
-            ) InputPortEntity portEntity) {
+            ) PortEntity portEntity) {
 
-        if (portEntity == null || portEntity.getInputPort() == null) {
+        if (portEntity == null || portEntity.getComponent() == null) {
             throw new IllegalArgumentException("Input port details must be specified.");
         }
 
@@ -201,7 +213,7 @@ public class InputPortResource extends ApplicationResource {
         }
 
         // ensure the ids are the same
-        final PortDTO requestPortDTO = portEntity.getInputPort();
+        final PortDTO requestPortDTO = portEntity.getComponent();
         if (!id.equals(requestPortDTO.getId())) {
             throw new IllegalArgumentException(String.format("The input port id (%s) in the request body does not equal the "
                     + "input port id of the requested resource (%s).", requestPortDTO.getId(), id));
@@ -226,25 +238,15 @@ public class InputPortResource extends ApplicationResource {
 
         // update the input port
         final RevisionDTO revision = portEntity.getRevision();
-        final ConfigurationSnapshot<PortDTO> controllerResponse = serviceFacade.updateInputPort(
+        final UpdateResult<PortEntity> updateResult = serviceFacade.updateInputPort(
                 new Revision(revision.getVersion(), revision.getClientId()), requestPortDTO);
 
-        // get the results
-        final PortDTO responsePortDTO = controllerResponse.getConfiguration();
-        populateRemainingInputPortContent(responsePortDTO);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
         // build the response entity
-        final InputPortEntity entity = new InputPortEntity();
-        entity.setRevision(updatedRevision);
-        entity.setInputPort(responsePortDTO);
+        final PortEntity entity = updateResult.getResult();
+        populateRemainingInputPortEntityContent(entity);
 
-        if (controllerResponse.isNew()) {
-            return clusterContext(generateCreatedResponse(URI.create(responsePortDTO.getUri()), entity)).build();
+        if (updateResult.isNew()) {
+            return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
         } else {
             return clusterContext(generateOkResponse(entity)).build();
         }
@@ -266,7 +268,7 @@ public class InputPortResource extends ApplicationResource {
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes an input port",
-            response = InputPortEntity.class,
+            response = PortEntity.class,
             authorizations = {
                 @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
             }
@@ -317,17 +319,7 @@ public class InputPortResource extends ApplicationResource {
         }
 
         // delete the specified input port
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteInputPort(new Revision(clientVersion, clientId.getClientId()), id);
-
-        // get the updated revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        revision.setVersion(controllerResponse.getVersion());
-
-        // build the response entity
-        final InputPortEntity entity = new InputPortEntity();
-        entity.setRevision(revision);
-
+        final PortEntity entity = serviceFacade.deleteInputPort(new Revision(clientVersion, clientId.getClientId()), id);
         return clusterContext(generateOkResponse(entity)).build();
     }