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

[07/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/api/ProcessGroupResource.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/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index 208d30d..49a3264 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -32,34 +32,31 @@ 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.ControllerServiceDTO;
-import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.FunnelDTO;
-import org.apache.nifi.web.api.dto.LabelDTO;
-import org.apache.nifi.web.api.dto.PortDTO;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.SnippetDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.dto.flow.FlowDTO;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
 import org.apache.nifi.web.api.entity.ConnectionsEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
 import org.apache.nifi.web.api.entity.ControllerServicesEntity;
 import org.apache.nifi.web.api.entity.CopySnippetRequestEntity;
 import org.apache.nifi.web.api.entity.CreateTemplateRequestEntity;
+import org.apache.nifi.web.api.entity.FlowEntity;
 import org.apache.nifi.web.api.entity.FlowSnippetEntity;
 import org.apache.nifi.web.api.entity.FunnelEntity;
 import org.apache.nifi.web.api.entity.FunnelsEntity;
-import org.apache.nifi.web.api.entity.InputPortEntity;
 import org.apache.nifi.web.api.entity.InputPortsEntity;
 import org.apache.nifi.web.api.entity.InstantiateTemplateRequestEntity;
 import org.apache.nifi.web.api.entity.LabelEntity;
 import org.apache.nifi.web.api.entity.LabelsEntity;
-import org.apache.nifi.web.api.entity.OutputPortEntity;
 import org.apache.nifi.web.api.entity.OutputPortsEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupsEntity;
 import org.apache.nifi.web.api.entity.ProcessorEntity;
@@ -141,6 +138,32 @@ public class ProcessGroupResource extends ApplicationResource {
     /**
      * Populates the remaining fields in the specified process groups.
      *
+     * @param processGroupEntities groups
+     * @return group dto
+     */
+    public Set<ProcessGroupEntity> populateRemainingProcessGroupEntitiesContent(Set<ProcessGroupEntity> processGroupEntities) {
+        for (ProcessGroupEntity processGroupEntity : processGroupEntities) {
+            populateRemainingProcessGroupEntityContent(processGroupEntity);
+        }
+        return processGroupEntities;
+    }
+
+    /**
+     * Populates the remaining fields in the specified process group.
+     *
+     * @param processGroupEntity group
+     * @return group dto
+     */
+    public ProcessGroupEntity populateRemainingProcessGroupEntityContent(ProcessGroupEntity processGroupEntity) {
+        if (processGroupEntity.getComponent() != null) {
+            populateRemainingProcessGroupContent(processGroupEntity.getComponent());
+        }
+        return processGroupEntity;
+    }
+
+    /**
+     * Populates the remaining fields in the specified process groups.
+     *
      * @param processGroups groups
      * @return group dto
      */
@@ -158,37 +181,28 @@ public class ProcessGroupResource extends ApplicationResource {
      * @return group dto
      */
     private ProcessGroupDTO populateRemainingProcessGroupContent(ProcessGroupDTO processGroup) {
-        FlowSnippetDTO flowSnippet = processGroup.getContents();
-
-        // populate the remaining fields for the processors, connections, process group refs, remote process groups, and labels if appropriate
-        if (flowSnippet != null) {
-            populateRemainingSnippetContent(flowSnippet);
-        }
-
-        // set the process group uri
         processGroup.setUri(generateResourceUri("process-groups",  processGroup.getId()));
-
         return processGroup;
     }
 
     /**
      * Populates the remaining content of the specified snippet.
      */
-    private FlowSnippetDTO populateRemainingSnippetContent(FlowSnippetDTO snippet) {
-        processorResource.populateRemainingProcessorsContent(snippet.getProcessors());
-        connectionResource.populateRemainingConnectionsContent(snippet.getConnections());
-        inputPortResource.populateRemainingInputPortsContent(snippet.getInputPorts());
-        outputPortResource.populateRemainingOutputPortsContent(snippet.getOutputPorts());
-        remoteProcessGroupResource.populateRemainingRemoteProcessGroupsContent(snippet.getRemoteProcessGroups());
-        funnelResource.populateRemainingFunnelsContent(snippet.getFunnels());
-        labelResource.populateRemainingLabelsContent(snippet.getLabels());
+    private FlowDTO populateRemainingSnippetContent(FlowDTO flow) {
+        processorResource.populateRemainingProcessorEntitiesContent(flow.getProcessors());
+        connectionResource.populateRemainingConnectionEntitiesContent(flow.getConnections());
+        inputPortResource.populateRemainingInputPortEntitiesContent(flow.getInputPorts());
+        outputPortResource.populateRemainingOutputPortEntitiesContent(flow.getOutputPorts());
+        remoteProcessGroupResource.populateRemainingRemoteProcessGroupEntitiesContent(flow.getRemoteProcessGroups());
+        funnelResource.populateRemainingFunnelEntitiesContent(flow.getFunnels());
+        labelResource.populateRemainingLabelEntitiesContent(flow.getLabels());
 
         // go through each process group child and populate its uri
-        if (snippet.getProcessGroups() != null) {
-            populateRemainingProcessGroupsContent(snippet.getProcessGroups());
+        if (flow.getProcessGroups() != null) {
+            populateRemainingProcessGroupEntitiesContent(flow.getProcessGroups());
         }
 
-        return snippet;
+        return flow;
     }
 
     /**
@@ -196,26 +210,17 @@ public class ProcessGroupResource extends ApplicationResource {
      */
     private SnippetDTO populateRemainingSnippetContent(SnippetDTO snippet) {
         String snippetGroupId = snippet.getParentGroupId();
-        FlowSnippetDTO snippetContents = snippet.getContents();
 
         // populate the snippet href
         snippet.setUri(generateResourceUri("process-groups", snippetGroupId, "snippets", snippet.getId()));
 
-        // populate the snippet content uris
-        if (snippet.getContents() != null) {
-            populateRemainingSnippetContent(snippetContents);
-        }
-
         return snippet;
     }
 
     /**
      * 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.
-     * @param verbose Optional verbose flag that defaults to false. If the verbose flag is set to true processor configuration and property details will be included in the response.
      * @return A processGroupEntity.
      */
     @GET
@@ -243,54 +248,26 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getProcessGroup(
             @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,
-            @ApiParam(
-                    value = "Whether to include any encapulated components or just details about the process group.",
-                    required = false
-            )
-            @QueryParam("verbose") @DefaultValue(VERBOSE) Boolean verbose) {
+            @PathParam("id") String groupId) {
 
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
         }
 
-        // only recurse if the request is verbose and recursive
-        final boolean recurse = verbose && recursive;
-
         // get this process group contents
-        final ConfigurationSnapshot<ProcessGroupDTO> controllerResponse = serviceFacade.getProcessGroup(groupId, recurse);
-        final ProcessGroupDTO processGroup = controllerResponse.getConfiguration();
+        final ProcessGroupEntity entity = serviceFacade.getProcessGroup(groupId);
+        populateRemainingProcessGroupEntityContent(entity);
 
-        // prune the response if necessary
-        if (!verbose) {
-            processGroup.setContents(null);
+        if (entity.getComponent() != null) {
+            entity.getComponent().setContents(null);
         }
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        revision.setVersion(controllerResponse.getVersion());
-
-        // create the response entity
-        final ProcessGroupEntity processGroupEntity = new ProcessGroupEntity();
-        processGroupEntity.setRevision(revision);
-        processGroupEntity.setProcessGroup(populateRemainingProcessGroupContent(processGroup));
 
-        return clusterContext(generateOkResponse(processGroupEntity)).build();
+        return clusterContext(generateOkResponse(entity)).build();
     }
 
     /**
@@ -335,7 +312,7 @@ public class ProcessGroupResource extends ApplicationResource {
             )
             ProcessGroupEntity processGroupEntity) {
 
-        if (processGroupEntity == null || processGroupEntity.getProcessGroup() == null) {
+        if (processGroupEntity == null || processGroupEntity.getComponent() == null) {
             throw new IllegalArgumentException("Process group details must be specified.");
         }
 
@@ -344,7 +321,7 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // ensure the same id is being used
-        final ProcessGroupDTO requestProcessGroupDTO = processGroupEntity.getProcessGroup();
+        final ProcessGroupDTO requestProcessGroupDTO = processGroupEntity.getComponent();
         if (!id.equals(requestProcessGroupDTO.getId())) {
             throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
                     + "not equal the process group id of the requested resource (%s).", requestProcessGroupDTO.getId(), id));
@@ -363,22 +340,12 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // update the process group
         final RevisionDTO revision = processGroupEntity.getRevision();
-        final ConfigurationSnapshot<ProcessGroupDTO> response = serviceFacade.updateProcessGroup(
-                new Revision(revision.getVersion(), revision.getClientId()), requestProcessGroupDTO);
-        final ProcessGroupDTO processGroup = response.getConfiguration();
+        final UpdateResult<ProcessGroupEntity> updateResult = serviceFacade.updateProcessGroup(new Revision(revision.getVersion(), revision.getClientId()), requestProcessGroupDTO);
+        final ProcessGroupEntity entity = updateResult.getResult();
+        populateRemainingProcessGroupEntityContent(entity);
 
-        // create the revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(response.getVersion());
-
-        // create the response entity
-        final ProcessGroupEntity entity = new ProcessGroupEntity();
-        entity.setRevision(updatedRevision);
-        entity.setProcessGroup(populateRemainingProcessGroupContent(processGroup));
-
-        if (response.isNew()) {
-            return clusterContext(generateCreatedResponse(URI.create(processGroup.getUri()), entity)).build();
+        if (updateResult.isNew()) {
+            return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
         } else {
             return clusterContext(generateOkResponse(entity)).build();
         }
@@ -451,16 +418,7 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // delete the process group
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteProcessGroup(new Revision(clientVersion, clientId.getClientId()), id);
-
-        // get the updated revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        revision.setVersion(controllerResponse.getVersion());
-
-        // create the response entity
-        final ProcessGroupEntity entity = new ProcessGroupEntity();
-        entity.setRevision(revision);
+        final ProcessGroupEntity entity = serviceFacade.deleteProcessGroup(new Revision(clientVersion, clientId.getClientId()), id);
 
         // create the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -508,7 +466,7 @@ public class ProcessGroupResource extends ApplicationResource {
         )
             ProcessGroupEntity processGroupEntity) {
 
-        if (processGroupEntity == null || processGroupEntity.getProcessGroup() == null) {
+        if (processGroupEntity == null || processGroupEntity.getComponent() == null) {
             throw new IllegalArgumentException("Process group details must be specified.");
         }
 
@@ -516,10 +474,16 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (processGroupEntity.getProcessGroup().getId() != null) {
+        if (processGroupEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Process group ID cannot be specified.");
         }
 
+        if (processGroupEntity.getComponent().getParentGroupId() != null && !groupId.equals(processGroupEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                processGroupEntity.getComponent().getParentGroupId(), groupId));
+        }
+        processGroupEntity.getComponent().setParentGroupId(groupId);
+
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(processGroupEntity), getHeaders()).getResponse();
         }
@@ -533,36 +497,24 @@ public class ProcessGroupResource extends ApplicationResource {
         // set the processor id as appropriate
         final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
         if (clusterContext != null) {
-            processGroupEntity.getProcessGroup().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+            processGroupEntity.getComponent().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
         } else {
-            processGroupEntity.getProcessGroup().setId(UUID.randomUUID().toString());
+            processGroupEntity.getComponent().setId(UUID.randomUUID().toString());
         }
 
         // create the process group contents
         final RevisionDTO revision = processGroupEntity.getRevision();
-        final ConfigurationSnapshot<ProcessGroupDTO> controllerResponse = serviceFacade.createProcessGroup(groupId,
-            new Revision(revision.getVersion(), revision.getClientId()), processGroupEntity.getProcessGroup());
-        final ProcessGroupDTO processGroup = controllerResponse.getConfiguration();
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // create the response entity
-        final ProcessGroupEntity entity = new ProcessGroupEntity();
-        entity.setRevision(updatedRevision);
-        entity.setProcessGroup(populateRemainingProcessGroupContent(processGroup));
+        final ProcessGroupEntity entity = serviceFacade.createProcessGroup(groupId, new Revision(revision.getVersion(), revision.getClientId()), processGroupEntity.getComponent());
+        populateRemainingProcessGroupEntityContent(entity);
 
         // generate a 201 created response
-        String uri = processGroup.getUri();
+        String uri = entity.getComponent().getUri();
         return clusterContext(generateCreatedResponse(URI.create(uri), entity)).build();
     }
 
     /**
      * Retrieves all the processors in this NiFi.
      *
-     * @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.
      * @return A processorsEntity.
      */
     @GET
@@ -589,7 +541,6 @@ public class ProcessGroupResource extends ApplicationResource {
         }
     )
     public Response getProcessGroups(
-        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
         @ApiParam(
             value = "The process group id.",
             required = true
@@ -602,16 +553,18 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // get the process groups
-        final Set<ProcessGroupDTO> processGroupDTOs = serviceFacade.getProcessGroups(groupId);
+        final Set<ProcessGroupEntity> entities = serviceFacade.getProcessGroups(groupId);
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        // always prune the contents
+        for (final ProcessGroupEntity entity : entities) {
+            if (entity.getComponent() != null) {
+                entity.getComponent().setContents(null);
+            }
+        }
 
         // create the response entity
         final ProcessGroupsEntity entity = new ProcessGroupsEntity();
-        entity.setRevision(revision);
-        entity.setProcessGroups(populateRemainingProcessGroupsContent(processGroupDTOs));
+        entity.setProcessGroups(populateRemainingProcessGroupEntitiesContent(entities));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -663,7 +616,7 @@ public class ProcessGroupResource extends ApplicationResource {
             )
             ProcessorEntity processorEntity) {
 
-        if (processorEntity == null || processorEntity.getProcessor() == null) {
+        if (processorEntity == null || processorEntity.getComponent() == null) {
             throw new IllegalArgumentException("Processor details must be specified.");
         }
 
@@ -671,14 +624,20 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (processorEntity.getProcessor().getId() != null) {
+        if (processorEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Processor ID cannot be specified.");
         }
 
-        if (StringUtils.isBlank(processorEntity.getProcessor().getType())) {
+        if (StringUtils.isBlank(processorEntity.getComponent().getType())) {
             throw new IllegalArgumentException("The type of processor to create must be specified.");
         }
 
+        if (processorEntity.getComponent().getParentGroupId() != null && !groupId.equals(processorEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                processorEntity.getComponent().getParentGroupId(), groupId));
+        }
+        processorEntity.getComponent().setParentGroupId(groupId);
+
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(processorEntity), getHeaders()).getResponse();
         }
@@ -692,37 +651,25 @@ public class ProcessGroupResource extends ApplicationResource {
         // set the processor id as appropriate
         final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
         if (clusterContext != null) {
-            processorEntity.getProcessor().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+            processorEntity.getComponent().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
         } else {
-            processorEntity.getProcessor().setId(UUID.randomUUID().toString());
+            processorEntity.getComponent().setId(UUID.randomUUID().toString());
         }
 
         // create the new processor
         final RevisionDTO revision = processorEntity.getRevision();
-        final ConfigurationSnapshot<ProcessorDTO> controllerResponse = serviceFacade.createProcessor(
-            new Revision(revision.getVersion(), revision.getClientId()), groupId, processorEntity.getProcessor());
-        final ProcessorDTO processor = controllerResponse.getConfiguration();
-        processorResource.populateRemainingProcessorContent(processor);
-
-        // get the updated revision
-        final 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(processor);
+        final ProcessorEntity entity = serviceFacade.createProcessor(new Revision(revision.getVersion(), revision.getClientId()), groupId, processorEntity.getComponent());
+        processorResource.populateRemainingProcessorEntityContent(entity);
 
         // generate a 201 created response
-        String uri = processor.getUri();
+        String uri = entity.getComponent().getUri();
         return clusterContext(generateCreatedResponse(URI.create(uri), entity)).build();
     }
 
     /**
      * Retrieves all the processors in this NiFi.
      *
-     * @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 groupId group id
      * @return A processorsEntity.
      */
     @GET
@@ -749,7 +696,6 @@ public class ProcessGroupResource extends ApplicationResource {
         }
     )
     public Response getProcessors(
-        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
         @ApiParam(
             value = "The process group id.",
             required = true
@@ -762,16 +708,11 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // get the processors
-        final Set<ProcessorDTO> processorDTOs = serviceFacade.getProcessors(groupId);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        final Set<ProcessorEntity> processors = serviceFacade.getProcessors(groupId);
 
         // create the response entity
         final ProcessorsEntity entity = new ProcessorsEntity();
-        entity.setRevision(revision);
-        entity.setProcessors(processorResource.populateRemainingProcessorsContent(processorDTOs));
+        entity.setProcessors(processorResource.populateRemainingProcessorEntitiesContent(processors));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -796,7 +737,7 @@ public class ProcessGroupResource extends ApplicationResource {
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
         value = "Creates an input port",
-        response = InputPortEntity.class,
+        response = PortEntity.class,
         authorizations = {
             @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
         }
@@ -820,9 +761,9 @@ public class ProcessGroupResource 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("Port details must be specified.");
         }
 
@@ -830,10 +771,16 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (portEntity.getInputPort().getId() != null) {
+        if (portEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Input port ID cannot be specified.");
         }
 
+        if (portEntity.getComponent().getParentGroupId() != null && !groupId.equals(portEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                portEntity.getComponent().getParentGroupId(), groupId));
+        }
+        portEntity.getComponent().setParentGroupId(groupId);
+
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(portEntity), getHeaders()).getResponse();
         }
@@ -847,36 +794,23 @@ public class ProcessGroupResource extends ApplicationResource {
         // set the processor id as appropriate
         final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
         if (clusterContext != null) {
-            portEntity.getInputPort().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+            portEntity.getComponent().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
         } else {
-            portEntity.getInputPort().setId(UUID.randomUUID().toString());
+            portEntity.getComponent().setId(UUID.randomUUID().toString());
         }
 
         // create the input port and generate the json
         final RevisionDTO revision = portEntity.getRevision();
-        final ConfigurationSnapshot<PortDTO> controllerResponse = serviceFacade.createInputPort(
-            new Revision(revision.getVersion(), revision.getClientId()), groupId, portEntity.getInputPort());
-        final PortDTO port = controllerResponse.getConfiguration();
-        inputPortResource.populateRemainingInputPortContent(port);
-
-        // 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(port);
+        final PortEntity entity = serviceFacade.createInputPort(new Revision(revision.getVersion(), revision.getClientId()), groupId, portEntity.getComponent());
+        inputPortResource.populateRemainingInputPortEntityContent(entity);
 
         // build the response
-        return clusterContext(generateCreatedResponse(URI.create(port.getUri()), entity)).build();
+        return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
     }
 
     /**
      * Retrieves all the of input ports in this NiFi.
      *
-     * @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.
      * @return A inputPortsEntity.
      */
     @GET
@@ -904,11 +838,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getInputPorts(
         @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 = true
         )
@@ -920,16 +849,10 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // get all the input ports
-        final Set<PortDTO> inputPorts = inputPortResource.populateRemainingInputPortsContent(serviceFacade.getInputPorts(groupId));
+        final Set<PortEntity> inputPorts = serviceFacade.getInputPorts(groupId);
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
         final InputPortsEntity entity = new InputPortsEntity();
-        entity.setRevision(revision);
-        entity.setInputPorts(inputPorts);
+        entity.setInputPorts(inputPortResource.populateRemainingInputPortEntitiesContent(inputPorts));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -954,7 +877,7 @@ public class ProcessGroupResource extends ApplicationResource {
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
         value = "Creates an output port",
-        response = OutputPortEntity.class,
+        response = PortEntity.class,
         authorizations = {
             @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
         }
@@ -978,9 +901,9 @@ public class ProcessGroupResource extends ApplicationResource {
         @ApiParam(
             value = "The output port configuration.",
             required = true
-        ) OutputPortEntity portEntity) {
+        ) PortEntity portEntity) {
 
-        if (portEntity == null || portEntity.getOutputPort() == null) {
+        if (portEntity == null || portEntity.getComponent() == null) {
             throw new IllegalArgumentException("Port details must be specified.");
         }
 
@@ -988,10 +911,16 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (portEntity.getOutputPort().getId() != null) {
+        if (portEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Output port ID cannot be specified.");
         }
 
+        if (portEntity.getComponent().getParentGroupId() != null && !groupId.equals(portEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                portEntity.getComponent().getParentGroupId(), groupId));
+        }
+        portEntity.getComponent().setParentGroupId(groupId);
+
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(portEntity), getHeaders()).getResponse();
         }
@@ -1005,36 +934,24 @@ public class ProcessGroupResource extends ApplicationResource {
         // set the processor id as appropriate
         final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
         if (clusterContext != null) {
-            portEntity.getOutputPort().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+            portEntity.getComponent().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
         } else {
-            portEntity.getOutputPort().setId(UUID.randomUUID().toString());
+            portEntity.getComponent().setId(UUID.randomUUID().toString());
         }
 
         // create the output port and generate the json
         final RevisionDTO revision = portEntity.getRevision();
-        final ConfigurationSnapshot<PortDTO> controllerResponse = serviceFacade.createOutputPort(
-            new Revision(revision.getVersion(), revision.getClientId()), groupId, portEntity.getOutputPort());
-        final PortDTO port = controllerResponse.getConfiguration();
-        outputPortResource.populateRemainingOutputPortContent(port);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // build the response entity
-        final OutputPortEntity entity = new OutputPortEntity();
-        entity.setRevision(updatedRevision);
-        entity.setOutputPort(port);
+        final PortEntity entity = serviceFacade.createOutputPort(
+            new Revision(revision.getVersion(), revision.getClientId()), groupId, portEntity.getComponent());
+        outputPortResource.populateRemainingOutputPortEntityContent(entity);
 
         // build the response
-        return clusterContext(generateCreatedResponse(URI.create(port.getUri()), entity)).build();
+        return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
     }
 
     /**
      * Retrieves all the of output ports in this NiFi.
      *
-     * @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.
      * @return A outputPortsEntity.
      */
     @GET
@@ -1062,11 +979,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getOutputPorts(
         @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 = true
         )
@@ -1078,16 +990,11 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // get all the output ports
-        final Set<PortDTO> outputPorts = outputPortResource.populateRemainingOutputPortsContent(serviceFacade.getOutputPorts(groupId));
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        final Set<PortEntity> outputPorts = serviceFacade.getOutputPorts(groupId);
 
         // create the response entity
         final OutputPortsEntity entity = new OutputPortsEntity();
-        entity.setRevision(revision);
-        entity.setOutputPorts(outputPorts);
+        entity.setOutputPorts(outputPortResource.populateRemainingOutputPortEntitiesContent(outputPorts));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -1138,7 +1045,7 @@ public class ProcessGroupResource 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.");
         }
 
@@ -1146,17 +1053,18 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (funnelEntity.getFunnel().getId() != null) {
+        if (funnelEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Funnel ID cannot be specified.");
         }
 
-        if (properties.isClusterManager()) {
-            // change content type to JSON for serializing entity if request came through overloaded endpoint
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
+        if (funnelEntity.getComponent().getParentGroupId() != null && !groupId.equals(funnelEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                funnelEntity.getComponent().getParentGroupId(), groupId));
+        }
+        funnelEntity.getComponent().setParentGroupId(groupId);
 
-            // replicate the request
-            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(funnelEntity), getHeaders(headersToOverride)).getResponse();
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(funnelEntity), getHeaders()).getResponse();
         }
 
         // handle expects request (usually from the cluster manager)
@@ -1168,38 +1076,23 @@ public class ProcessGroupResource extends ApplicationResource {
         // set the processor id as appropriate
         final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
         if (clusterContext != null) {
-            funnelEntity.getFunnel().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+            funnelEntity.getComponent().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
         } else {
-            funnelEntity.getFunnel().setId(UUID.randomUUID().toString());
+            funnelEntity.getComponent().setId(UUID.randomUUID().toString());
         }
 
         // create the funnel and generate the json
         final RevisionDTO revision = funnelEntity.getRevision();
-        final ConfigurationSnapshot<FunnelDTO> controllerResponse = serviceFacade.createFunnel(
-            new Revision(revision.getVersion(), revision.getClientId()), groupId, funnelEntity.getFunnel());
-        final FunnelDTO funnel = controllerResponse.getConfiguration();
-        funnelResource.populateRemainingFunnelContent(funnel);
-
-        // 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(funnel);
+        final FunnelEntity entity = serviceFacade.createFunnel(new Revision(revision.getVersion(), revision.getClientId()), groupId, funnelEntity.getComponent());
+        funnelResource.populateRemainingFunnelEntityContent(entity);
 
         // build the response
-        return clusterContext(generateCreatedResponse(URI.create(funnel.getUri()), entity)).build();
+        return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
     }
 
     /**
      * Retrieves all the of funnels in this NiFi.
      *
-     * @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.
      * @return A funnelsEntity.
      */
     @GET
@@ -1227,11 +1120,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getFunnels(
         @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 = true
         )
@@ -1243,16 +1131,11 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // get all the funnels
-        final Set<FunnelDTO> funnels = funnelResource.populateRemainingFunnelsContent(serviceFacade.getFunnels(groupId));
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        final Set<FunnelEntity> funnels = serviceFacade.getFunnels(groupId);
 
         // create the response entity
         final FunnelsEntity entity = new FunnelsEntity();
-        entity.setRevision(revision);
-        entity.setFunnels(funnels);
+        entity.setFunnels(funnelResource.populateRemainingFunnelEntitiesContent(funnels));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -1303,7 +1186,7 @@ public class ProcessGroupResource extends ApplicationResource {
             required = true
         ) LabelEntity labelEntity) {
 
-        if (labelEntity == null || labelEntity.getLabel() == null) {
+        if (labelEntity == null || labelEntity.getComponent() == null) {
             throw new IllegalArgumentException("Label details must be specified.");
         }
 
@@ -1311,10 +1194,16 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (labelEntity.getLabel().getId() != null) {
+        if (labelEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Label ID cannot be specified.");
         }
 
+        if (labelEntity.getComponent().getParentGroupId() != null && !groupId.equals(labelEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                labelEntity.getComponent().getParentGroupId(), groupId));
+        }
+        labelEntity.getComponent().setParentGroupId(groupId);
+
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(labelEntity), getHeaders()).getResponse();
         }
@@ -1328,41 +1217,29 @@ public class ProcessGroupResource extends ApplicationResource {
         // set the processor id as appropriate
         final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
         if (clusterContext != null) {
-            labelEntity.getLabel().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+            labelEntity.getComponent().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
         } else {
-            labelEntity.getLabel().setId(UUID.randomUUID().toString());
+            labelEntity.getComponent().setId(UUID.randomUUID().toString());
         }
 
         // create the label and generate the json
         final RevisionDTO revision = labelEntity.getRevision();
-        final ConfigurationSnapshot<LabelDTO> controllerResponse = serviceFacade.createLabel(
-            new Revision(revision.getVersion(), revision.getClientId()), groupId, labelEntity.getLabel());
-        final LabelDTO label = controllerResponse.getConfiguration();
-        labelResource.populateRemainingLabelContent(label);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // build the response entity
-        final LabelEntity entity = new LabelEntity();
-        entity.setRevision(updatedRevision);
-        entity.setLabel(label);
+        final LabelEntity entity = serviceFacade.createLabel(
+            new Revision(revision.getVersion(), revision.getClientId()), groupId, labelEntity.getComponent());
+        labelResource.populateRemainingLabelEntityContent(entity);
 
         // build the response
-        return clusterContext(generateCreatedResponse(URI.create(label.getUri()), entity)).build();
+        return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
     }
 
     /**
      * Retrieves all the of labels in this NiFi.
      *
-     * @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.
      * @return A labelsEntity.
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}/labels")
     // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
@@ -1385,11 +1262,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getLabels(
         @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 = true
         )
@@ -1401,16 +1273,11 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // get all the labels
-        final Set<LabelDTO> labels = labelResource.populateRemainingLabelsContent(serviceFacade.getLabels(groupId));
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        final Set<LabelEntity> labels = serviceFacade.getLabels(groupId);
 
         // create the response entity
         final LabelsEntity entity = new LabelsEntity();
-        entity.setRevision(revision);
-        entity.setLabels(labels);
+        entity.setLabels(labelResource.populateRemainingLabelEntitiesContent(labels));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -1461,7 +1328,7 @@ public class ProcessGroupResource extends ApplicationResource {
             required = true
         ) RemoteProcessGroupEntity remoteProcessGroupEntity) {
 
-        if (remoteProcessGroupEntity == null || remoteProcessGroupEntity.getRemoteProcessGroup() == null) {
+        if (remoteProcessGroupEntity == null || remoteProcessGroupEntity.getComponent() == null) {
             throw new IllegalArgumentException("Remote process group details must be specified.");
         }
 
@@ -1469,7 +1336,7 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        final RemoteProcessGroupDTO requestProcessGroupDTO = remoteProcessGroupEntity.getRemoteProcessGroup();
+        final RemoteProcessGroupDTO requestProcessGroupDTO = remoteProcessGroupEntity.getComponent();
 
         if (requestProcessGroupDTO.getId() != null) {
             throw new IllegalArgumentException("Remote process group ID cannot be specified.");
@@ -1479,14 +1346,14 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("The URI of the process group must be specified.");
         }
 
-        // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate
-        if (properties.isClusterManager()) {
-            // change content type to JSON for serializing entity if request came through overloaded endpoint
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
+        if (requestProcessGroupDTO.getParentGroupId() != null && !groupId.equals(requestProcessGroupDTO.getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                requestProcessGroupDTO.getParentGroupId(), groupId));
+        }
+        requestProcessGroupDTO.setParentGroupId(groupId);
 
-            // replicate the request
-            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(remoteProcessGroupEntity), getHeaders(headersToOverride)).getResponse();
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(remoteProcessGroupEntity), getHeaders()).getResponse();
         }
 
         // handle expects request (usually from the cluster manager)
@@ -1531,30 +1398,15 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // create the remote process group
         final RevisionDTO revision = remoteProcessGroupEntity.getRevision();
-        final ConfigurationSnapshot<RemoteProcessGroupDTO> controllerResponse
-            = serviceFacade.createRemoteProcessGroup(new Revision(revision.getVersion(), revision.getClientId()), groupId, requestProcessGroupDTO);
-
-        // prepare the response
-        final RemoteProcessGroupDTO remoteProcessGroup = controllerResponse.getConfiguration();
-        remoteProcessGroupResource.populateRemainingRemoteProcessGroupContent(remoteProcessGroup);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
+        final RemoteProcessGroupEntity entity = serviceFacade.createRemoteProcessGroup(new Revision(revision.getVersion(), revision.getClientId()), groupId, requestProcessGroupDTO);
+        remoteProcessGroupResource.populateRemainingRemoteProcessGroupEntityContent(entity);
 
-        // build the response entity
-        final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity();
-        entity.setRevision(updatedRevision);
-        entity.setRemoteProcessGroup(remoteProcessGroup);
-
-        return clusterContext(generateCreatedResponse(URI.create(remoteProcessGroup.getUri()), entity)).build();
+        return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
     }
 
     /**
      * Retrieves all the of remote process groups in this NiFi.
      *
-     * @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 verbose Optional verbose flag that defaults to false. If the verbose flag is set to true remote group contents (ports) will be included.
      * @return A remoteProcessGroupEntity.
      */
@@ -1583,11 +1435,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getRemoteProcessGroups(
         @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 to include any encapulated ports or just details about the remote process group.",
             required = false
         )
@@ -1603,24 +1450,21 @@ public class ProcessGroupResource extends ApplicationResource {
             return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
         }
 
-        // get all the labels
-        final Set<RemoteProcessGroupDTO> remoteProcessGroups = serviceFacade.getRemoteProcessGroups(groupId);
+        // get all the remote process groups
+        final Set<RemoteProcessGroupEntity> remoteProcessGroups = serviceFacade.getRemoteProcessGroups(groupId);
 
         // prune response as necessary
         if (!verbose) {
-            for (RemoteProcessGroupDTO remoteProcessGroup : remoteProcessGroups) {
-                remoteProcessGroup.setContents(null);
+            for (RemoteProcessGroupEntity remoteProcessGroupEntity : remoteProcessGroups) {
+                if (remoteProcessGroupEntity.getComponent() != null) {
+                    remoteProcessGroupEntity.getComponent().setContents(null);
+                }
             }
         }
 
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
         // create the response entity
         final RemoteProcessGroupsEntity entity = new RemoteProcessGroupsEntity();
-        entity.setRevision(revision);
-        entity.setRemoteProcessGroups(remoteProcessGroupResource.populateRemainingRemoteProcessGroupsContent(remoteProcessGroups));
+        entity.setRemoteProcessGroups(remoteProcessGroupResource.populateRemainingRemoteProcessGroupEntitiesContent(remoteProcessGroups));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -1671,11 +1515,11 @@ public class ProcessGroupResource 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.");
         }
 
-        if (connectionEntity.getConnection().getId() != null) {
+        if (connectionEntity.getComponent().getId() != null) {
             throw new IllegalArgumentException("Connection ID cannot be specified.");
         }
 
@@ -1683,17 +1527,18 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("Revision must be specified.");
         }
 
-        if (properties.isClusterManager()) {
-            // change content type to JSON for serializing entity if request came through overloaded endpoint
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
+        if (connectionEntity.getComponent().getParentGroupId() != null && !groupId.equals(connectionEntity.getComponent().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                connectionEntity.getComponent().getParentGroupId(), groupId));
+        }
+        connectionEntity.getComponent().setParentGroupId(groupId);
 
-            // replicate the request
-            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(connectionEntity), getHeaders(headersToOverride)).getResponse();
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(connectionEntity), getHeaders()).getResponse();
         }
 
         // get the connection
-        final ConnectionDTO connection = connectionEntity.getConnection();
+        final ConnectionDTO connection = connectionEntity.getComponent();
 
         // handle expects request (usually from the cluster manager)
         final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
@@ -1712,33 +1557,17 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // create the new relationship target
         final RevisionDTO revision = connectionEntity.getRevision();
-        final ConfigurationSnapshot<ConnectionDTO> controllerResponse = serviceFacade.createConnection(
-            new Revision(revision.getVersion(), revision.getClientId()), groupId, connection);
-        ConnectionDTO connectionDTO = controllerResponse.getConfiguration();
-
-        // marshall the target and add the source processor
-        connectionResource.populateRemainingConnectionContent(connectionDTO);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // create the response entity
-        ConnectionEntity entity = new ConnectionEntity();
-        entity.setRevision(updatedRevision);
-        entity.setConnection(connectionDTO);
+        final ConnectionEntity entity = serviceFacade.createConnection(new Revision(revision.getVersion(), revision.getClientId()), groupId, connection);
+        connectionResource.populateRemainingConnectionEntityContent(entity);
 
         // extract the href and build the response
-        String href = connectionDTO.getUri();
-
-        return clusterContext(generateCreatedResponse(URI.create(href), entity)).build();
+        String uri = entity.getComponent().getUri();
+        return clusterContext(generateCreatedResponse(URI.create(uri), entity)).build();
     }
 
     /**
      * Gets all the connections.
      *
-     * @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.
      * @return A connectionsEntity.
      */
     @GET
@@ -1766,11 +1595,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getConnections(
         @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 = true
         )
@@ -1782,16 +1606,11 @@ public class ProcessGroupResource extends ApplicationResource {
         }
 
         // all of the relationships for the specified source processor
-        Set<ConnectionDTO> connections = serviceFacade.getConnections(groupId);
-
-        // create the revision
-        RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        Set<ConnectionEntity> connections = serviceFacade.getConnections(groupId);
 
         // create the client response entity
         ConnectionsEntity entity = new ConnectionsEntity();
-        entity.setRevision(revision);
-        entity.setConnections(connectionResource.populateRemainingConnectionsContent(connections));
+        entity.setConnections(connectionResource.populateRemainingConnectionEntitiesContent(connections));
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -1861,6 +1680,12 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("The group id must be specified when creating a snippet.");
         }
 
+        if (snippetEntity.getSnippet().getParentGroupId() != null && !groupId.equals(snippetEntity.getSnippet().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                snippetEntity.getSnippet().getParentGroupId(), groupId));
+        }
+        snippetEntity.getSnippet().setParentGroupId(groupId);
+
         if (properties.isClusterManager()) {
             return (Response) clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(snippetEntity), getHeaders()).getResponse();
         }
@@ -1886,9 +1711,6 @@ public class ProcessGroupResource extends ApplicationResource {
         // get the snippet
         final SnippetDTO snippet = response.getConfiguration();
 
-        // always prune the response when creating
-        snippet.setContents(null);
-
         // get the updated revision
         final RevisionDTO updatedRevision = new RevisionDTO();
         updatedRevision.setClientId(revision.getClientId());
@@ -1906,11 +1728,6 @@ public class ProcessGroupResource extends ApplicationResource {
     /**
      * Retrieves the specified snippet.
      *
-     * @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 verbose Whether or not to include the contents of the snippet in
-     * the response.
      * @param id The id of the snippet to retrieve.
      * @return A snippetEntity.
      */
@@ -1939,16 +1756,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getSnippet(
         @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 to include configuration details for the components specified in the snippet.",
-            required = false
-        )
-        @QueryParam("verbose") @DefaultValue(VERBOSE) Boolean verbose,
-        @ApiParam(
             value = "The process group id.",
             required = true
         )
@@ -1967,14 +1774,8 @@ public class ProcessGroupResource extends ApplicationResource {
         // get the snippet
         final SnippetDTO snippet = serviceFacade.getSnippet(id);
 
-        // prune the response if necessary
-        if (!verbose) {
-            snippet.setContents(null);
-        }
-
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
 
         // create the response entity
         final SnippetEntity entity = new SnippetEntity();
@@ -2066,9 +1867,6 @@ public class ProcessGroupResource extends ApplicationResource {
         // get the results
         final SnippetDTO snippet = controllerResponse.getConfiguration();
 
-        // always prune update responses
-        snippet.setContents(null);
-
         // get the updated revision
         final RevisionDTO updatedRevision = new RevisionDTO();
         updatedRevision.setClientId(revision.getClientId());
@@ -2241,18 +2039,16 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // copy the specified snippet
         final RevisionDTO requestRevision = copySnippetEntity.getRevision();
-        final ConfigurationSnapshot<FlowSnippetDTO> controllerResponse = serviceFacade.copySnippet(
+        final ConfigurationSnapshot<FlowDTO> controllerResponse = serviceFacade.copySnippet(
             new Revision(requestRevision.getVersion(), requestRevision.getClientId()),
             groupId, copySnippetEntity.getSnippetId(), copySnippetEntity.getOriginX(), copySnippetEntity.getOriginY());
 
         // get the snippet
-        final FlowSnippetDTO flowSnippet = controllerResponse.getConfiguration();
+        final FlowDTO flow = controllerResponse.getConfiguration();
 
         // prune response as necessary
-        for (ProcessGroupDTO group : flowSnippet.getProcessGroups()) {
-            if (group.getContents() != null) {
-                group.setContents(null);
-            }
+        for (ProcessGroupEntity childGroupEntity : flow.getProcessGroups()) {
+            childGroupEntity.getComponent().setContents(null);
         }
 
         // get the updated revision
@@ -2261,9 +2057,9 @@ public class ProcessGroupResource extends ApplicationResource {
         revision.setVersion(controllerResponse.getVersion());
 
         // create the response entity
-        final FlowSnippetEntity entity = new FlowSnippetEntity();
+        final FlowEntity entity = new FlowEntity();
         entity.setRevision(revision);
-        entity.setContents(populateRemainingSnippetContent(flowSnippet));
+        entity.setFlow(populateRemainingSnippetContent(flow));
 
         // generate the response
         return clusterContext(generateCreatedResponse(getAbsolutePath(), entity)).build();
@@ -2282,7 +2078,7 @@ public class ProcessGroupResource extends ApplicationResource {
      * @param httpServletRequest request
      * @param groupId The group id
      * @param instantiateTemplateRequestEntity The instantiate template request
-     * @return A flowSnippetEntity.
+     * @return A flowEntity.
      */
     @POST
     @Consumes(MediaType.APPLICATION_JSON)
@@ -2291,7 +2087,7 @@ public class ProcessGroupResource extends ApplicationResource {
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
         value = "Instantiates a template",
-        response = FlowSnippetEntity.class,
+        response = FlowEntity.class,
         authorizations = {
             @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
         }
@@ -2335,17 +2131,15 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // create the template and generate the json
         final RevisionDTO requestRevision = instantiateTemplateRequestEntity.getRevision();
-        final ConfigurationSnapshot<FlowSnippetDTO> response = serviceFacade.createTemplateInstance(
+        final ConfigurationSnapshot<FlowDTO> response = serviceFacade.createTemplateInstance(
             new Revision(requestRevision.getVersion(), requestRevision.getClientId()), groupId, instantiateTemplateRequestEntity.getOriginX(),
             instantiateTemplateRequestEntity.getOriginY(), instantiateTemplateRequestEntity.getTemplateId());
 
-        final FlowSnippetDTO flowSnippet = response.getConfiguration();
+        final FlowDTO flowSnippet = response.getConfiguration();
 
         // prune response as necessary
-        for (ProcessGroupDTO group : flowSnippet.getProcessGroups()) {
-            if (group.getContents() != null) {
-                group.setContents(null);
-            }
+        for (ProcessGroupEntity childGroupEntity : flowSnippet.getProcessGroups()) {
+            childGroupEntity.getComponent().setContents(null);
         }
 
         // get the updated revision
@@ -2354,9 +2148,9 @@ public class ProcessGroupResource extends ApplicationResource {
         revision.setVersion(response.getVersion());
 
         // create the response entity
-        final FlowSnippetEntity entity = new FlowSnippetEntity();
+        final FlowEntity entity = new FlowEntity();
         entity.setRevision(revision);
-        entity.setContents(populateRemainingSnippetContent(flowSnippet));
+        entity.setFlow(populateRemainingSnippetContent(flowSnippet));
 
         // generate the response
         return clusterContext(generateCreatedResponse(getAbsolutePath(), entity)).build();
@@ -2369,9 +2163,6 @@ public class ProcessGroupResource extends ApplicationResource {
     /**
      * Retrieves all the of templates in this NiFi.
      *
-     * @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.
      * @return A templatesEntity.
      */
     @GET
@@ -2401,12 +2192,7 @@ public class ProcessGroupResource extends ApplicationResource {
             value = "The process group id.",
             required = true
         )
-        @PathParam("id") String groupId,
-        @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) {
+        @PathParam("id") String groupId) {
 
         // replicate if cluster manager
         if (properties.isClusterManager()) {
@@ -2418,7 +2204,6 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
 
         // create the response entity
         final TemplatesEntity entity = new TemplatesEntity();
@@ -2470,6 +2255,8 @@ public class ProcessGroupResource extends ApplicationResource {
             required = true
         ) CreateTemplateRequestEntity createTemplateRequestEntity) {
 
+        // TODO - verify parent group id
+
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(createTemplateRequestEntity), getHeaders()).getResponse();
@@ -2716,6 +2503,12 @@ public class ProcessGroupResource extends ApplicationResource {
             throw new IllegalArgumentException("The type of controller service to create must be specified.");
         }
 
+        if (controllerServiceEntity.getControllerService().getParentGroupId() != null && !groupId.equals(controllerServiceEntity.getControllerService().getParentGroupId())) {
+            throw new IllegalArgumentException(String.format("If specified, the parent process group id %s must be the same as specified in the URI %s",
+                controllerServiceEntity.getControllerService().getParentGroupId(), groupId));
+        }
+        controllerServiceEntity.getControllerService().setParentGroupId(groupId);
+
         // get the revision
         final RevisionDTO revision = controllerServiceEntity.getRevision();
 
@@ -2749,8 +2542,8 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // build the response entity
         final ControllerServiceEntity entity = new ControllerServiceEntity();
-        entity.setRevision(updatedRevision);
         entity.setControllerService(controllerServiceResource.populateRemainingControllerServiceContent(availability, controllerService));
+        entity.setRevision(updatedRevision);
 
         // build the response
         return clusterContext(generateCreatedResponse(URI.create(controllerService.getUri()), entity)).build();
@@ -2759,9 +2552,6 @@ public class ProcessGroupResource extends ApplicationResource {
     /**
      * Retrieves all the of controller services in this NiFi.
      *
-     * @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.
@@ -2791,11 +2581,6 @@ public class ProcessGroupResource extends ApplicationResource {
     )
     public Response getControllerServices(
         @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 = true
         )
@@ -2819,7 +2604,6 @@ public class ProcessGroupResource extends ApplicationResource {
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
 
         // create the response entity
         final ControllerServicesEntity entity = new ControllerServicesEntity();

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/ProcessorResource.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/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
index 6202561..9dddd70 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
@@ -34,6 +34,7 @@ import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.UiExtensionType;
+import org.apache.nifi.web.UpdateResult;
 import org.apache.nifi.web.api.dto.ComponentStateDTO;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
@@ -89,6 +90,34 @@ public class ProcessorResource extends ApplicationResource {
     /**
      * Populate the uri's for the specified processors and their relationships.
      *
+     * @param processorEntities processors
+     * @return dtos
+     */
+    public Set<ProcessorEntity> populateRemainingProcessorEntitiesContent(Set<ProcessorEntity> processorEntities) {
+        for (ProcessorEntity processorEntity : processorEntities) {
+            if (processorEntity.getComponent() != null) {
+                populateRemainingProcessorContent(processorEntity.getComponent());
+            }
+        }
+        return processorEntities;
+    }
+
+    /**
+     * Populate the uri's for the specified processors and their relationships.
+     *
+     * @param processorEntity processors
+     * @return dtos
+     */
+    public ProcessorEntity populateRemainingProcessorEntityContent(ProcessorEntity processorEntity) {
+        if (processorEntity.getComponent() != null) {
+            populateRemainingProcessorContent(processorEntity.getComponent());
+        }
+        return processorEntity;
+    }
+
+    /**
+     * Populate the uri's for the specified processors and their relationships.
+     *
      * @param processors processors
      * @return dtos
      */
@@ -133,7 +162,6 @@ public class ProcessorResource extends ApplicationResource {
     /**
      * Retrieves the specified 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 to retrieve.
      * @return A processorEntity.
      */
@@ -162,11 +190,6 @@ public class ProcessorResource extends ApplicationResource {
     )
     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
             )
@@ -178,16 +201,8 @@ public class ProcessorResource extends ApplicationResource {
         }
 
         // get the specified processor
-        final ProcessorDTO processor = serviceFacade.getProcessor(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // generate the response entity
-        final ProcessorEntity entity = new ProcessorEntity();
-        entity.setRevision(revision);
-        entity.setProcessor(populateRemainingProcessorContent(processor));
+        final ProcessorEntity entity = serviceFacade.getProcessor(id);
+        populateRemainingProcessorEntityContent(entity);
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -196,7 +211,6 @@ public class ProcessorResource 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 id The id of the processor
      * @param propertyName The property
      * @return a propertyDescriptorEntity
@@ -254,13 +268,8 @@ public class ProcessorResource extends ApplicationResource {
         // get the property descriptor
         final PropertyDescriptorDTO descriptor = serviceFacade.getProcessorPropertyDescriptor(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
@@ -270,7 +279,6 @@ public class ProcessorResource extends ApplicationResource {
     /**
      * Gets the state for a 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 componentStateEntity
      */
@@ -297,11 +305,6 @@ public class ProcessorResource 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 = "The processor id.",
             required = true
         )
@@ -315,13 +318,8 @@ public class ProcessorResource extends ApplicationResource {
         // get the component state
         final ComponentStateDTO state = serviceFacade.getProcessorState(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
@@ -446,7 +444,7 @@ public class ProcessorResource extends ApplicationResource {
             )
             ProcessorEntity processorEntity) {
 
-        if (processorEntity == null || processorEntity.getProcessor() == null) {
+        if (processorEntity == null || processorEntity.getComponent() == null) {
             throw new IllegalArgumentException("Processor details must be specified.");
         }
 
@@ -455,7 +453,7 @@ public class ProcessorResource extends ApplicationResource {
         }
 
         // ensure the same id is being used
-        final ProcessorDTO requestProcessorDTO = processorEntity.getProcessor();
+        final ProcessorDTO requestProcessorDTO = processorEntity.getComponent();
         if (!id.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(), id));
@@ -485,25 +483,12 @@ public class ProcessorResource extends ApplicationResource {
 
         // update the processor
         final RevisionDTO revision = processorEntity.getRevision();
-        final ConfigurationSnapshot<ProcessorDTO> controllerResponse = serviceFacade.updateProcessor(
-                new Revision(revision.getVersion(), revision.getClientId()), requestProcessorDTO);
-
-        // get the processor dto
-        final ProcessorDTO responseProcessorDTO = controllerResponse.getConfiguration();
-        populateRemainingProcessorContent(responseProcessorDTO);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
+        final UpdateResult<ProcessorEntity> result = serviceFacade.updateProcessor(new Revision(revision.getVersion(), revision.getClientId()), requestProcessorDTO);
+        final ProcessorEntity entity = result.getResult();
+        populateRemainingProcessorEntityContent(entity);
 
-        // generate the response entity
-        final ProcessorEntity entity = new ProcessorEntity();
-        entity.setRevision(updatedRevision);
-        entity.setProcessor(responseProcessorDTO);
-
-        if (controllerResponse.isNew()) {
-            return clusterContext(generateCreatedResponse(URI.create(responseProcessorDTO.getUri()), entity)).build();
+        if (result.isNew()) {
+            return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
         } else {
             return clusterContext(generateOkResponse(entity)).build();
         }
@@ -576,16 +561,7 @@ public class ProcessorResource extends ApplicationResource {
         }
 
         // delete the processor
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteProcessor(new Revision(clientVersion, clientId.getClientId()), id);
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(clientId.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // generate the response entity
-        final ProcessorEntity entity = new ProcessorEntity();
-        entity.setRevision(updatedRevision);
+        final ProcessorEntity entity = serviceFacade.deleteProcessor(new Revision(clientVersion, clientId.getClientId()), id);
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();