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

[12/22] nifi git commit: NIFI-1551: - Removing the AuthorityProvider. - Refactoring REST API in preparation for introduction of the Authorizer. - Updating UI accordingly. - Removing unneeded properties from nifi.properties. - Addressing comments from PR.

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/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 685fac8..2cff337 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
@@ -16,13 +16,14 @@
  */
 package org.apache.nifi.web.api;
 
-import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiParam;
 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.context.ClusterContext;
+import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceState;
@@ -42,20 +43,20 @@ import org.apache.nifi.web.api.entity.ComponentStateEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
 import org.apache.nifi.web.api.entity.ControllerServicesEntity;
+import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
+import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 import org.apache.nifi.web.util.Availability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.springframework.security.access.prepost.PreAuthorize;
 
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
-import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.POST;
@@ -64,15 +65,12 @@ import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.core.Response;
 import java.net.URI;
-import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -81,7 +79,7 @@ import java.util.UUID;
 /**
  * RESTful endpoint for managing a Controller Service.
  */
-@Api(hidden = true)
+@Path("controller-services")
 public class ControllerServiceResource extends ApplicationResource {
 
     private static final Logger logger = LoggerFactory.getLogger(ControllerServiceResource.class);
@@ -99,7 +97,7 @@ public class ControllerServiceResource extends ApplicationResource {
      * @param controllerServices services
      * @return dtos
      */
-    private Set<ControllerServiceDTO> populateRemainingControllerServicesContent(final String availability, final Set<ControllerServiceDTO> controllerServices) {
+    public Set<ControllerServiceDTO> populateRemainingControllerServicesContent(final String availability, final Set<ControllerServiceDTO> controllerServices) {
         for (ControllerServiceDTO controllerService : controllerServices) {
             populateRemainingControllerServiceContent(availability, controllerService);
         }
@@ -109,9 +107,9 @@ public class ControllerServiceResource extends ApplicationResource {
     /**
      * Populates the uri for the specified controller service.
      */
-    private ControllerServiceDTO populateRemainingControllerServiceContent(final String availability, final ControllerServiceDTO controllerService) {
+    public ControllerServiceDTO populateRemainingControllerServiceContent(final String availability, final ControllerServiceDTO controllerService) {
         // populate the controller service href
-        controllerService.setUri(generateResourceUri("controller", "controller-services", availability, controllerService.getId()));
+        controllerService.setUri(generateResourceUri("controller-services", availability, controllerService.getId()));
         controllerService.setAvailability(availability);
 
         // see if this processor has any ui extensions
@@ -164,9 +162,9 @@ public class ControllerServiceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets all controller services",
             response = ControllerServicesEntity.class,
@@ -221,52 +219,6 @@ public class ControllerServiceResource extends ApplicationResource {
     }
 
     /**
-     * Creates a new controller service.
-     *
-     * @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 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.
-     * @param type The type of controller service to create.
-     * @return A controllerServiceEntity.
-     */
-    @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response createControllerService(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @PathParam("availability") String availability,
-            @FormParam("type") String type) {
-
-        // create the controller service DTO
-        final ControllerServiceDTO controllerServiceDTO = new ControllerServiceDTO();
-        controllerServiceDTO.setType(type);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the controller service entity
-        final ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity();
-        controllerServiceEntity.setRevision(revision);
-        controllerServiceEntity.setControllerService(controllerServiceDTO);
-
-        return createControllerService(httpServletRequest, availability, controllerServiceEntity);
-    }
-
-    /**
      * Creates a new Controller Service.
      *
      * @param httpServletRequest request
@@ -277,10 +229,10 @@ public class ControllerServiceResource extends ApplicationResource {
      * @return A controllerServiceEntity.
      */
     @POST
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Creates a new controller service",
             response = ControllerServiceEntity.class,
@@ -330,28 +282,8 @@ public class ControllerServiceResource extends ApplicationResource {
         // get the revision
         final RevisionDTO revision = controllerServiceEntity.getRevision();
 
-        // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate
-        if (properties.isClusterManager() && Availability.NODE.equals(avail)) {
-            // create ID for resource
-            final String id = UUID.randomUUID().toString();
-
-            // set ID for resource
-            controllerServiceEntity.getControllerService().setId(id);
-
-            // convert POST request to PUT request to force entity ID to be the same across nodes
-            URI putUri = null;
-            try {
-                putUri = new URI(getAbsolutePath().toString() + "/" + id);
-            } catch (final URISyntaxException e) {
-                throw new WebApplicationException(e);
-            }
-
-            // change content type to JSON for serializing entity
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
-
-            // replicate put request
-            return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(controllerServiceEntity), getHeaders(headersToOverride)).getResponse();
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(controllerServiceEntity), getHeaders()).getResponse();
         }
 
         // handle expects request (usually from the cluster manager)
@@ -360,6 +292,14 @@ public class ControllerServiceResource extends ApplicationResource {
             return generateContinueResponse().build();
         }
 
+        // set the processor id as appropriate
+        final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
+        if (clusterContext != null) {
+            controllerServiceEntity.getControllerService().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+        } else {
+            controllerServiceEntity.getControllerService().setId(UUID.randomUUID().toString());
+        }
+
         // create the controller service and generate the json
         final ConfigurationSnapshot<ControllerServiceDTO> controllerResponse = serviceFacade.createControllerService(
                 new Revision(revision.getVersion(), revision.getClientId()), controllerServiceEntity.getControllerService());
@@ -393,9 +333,9 @@ public class ControllerServiceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets a controller service",
             response = ControllerServiceEntity.class,
@@ -467,9 +407,9 @@ public class ControllerServiceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}/descriptors")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}/descriptors")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets a controller service property descriptor",
             response = PropertyDescriptorEntity.class,
@@ -551,9 +491,9 @@ public class ControllerServiceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}/state")
-    @PreAuthorize("hasAnyRole('ROLE_DFM')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}/state")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')")
     @ApiOperation(
         value = "Gets the state for a controller service",
         response = ComponentStateDTO.class,
@@ -614,8 +554,7 @@ public class ControllerServiceResource extends ApplicationResource {
     /**
      * Clears 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 version The revision is used to verify the client is working with the latest version of the flow.
+     * @param revisionEntity The revision is used to verify the client is working with the latest version of the flow.
      * @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.
@@ -623,10 +562,10 @@ public class ControllerServiceResource extends ApplicationResource {
      * @return a componentStateEntity
      */
     @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}/state/clear-requests")
-    @PreAuthorize("hasAnyRole('ROLE_DFM')")
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}/state/clear-requests")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')")
     @ApiOperation(
         value = "Clears the state for a controller service",
         response = ComponentStateDTO.class,
@@ -646,15 +585,10 @@ public class ControllerServiceResource extends ApplicationResource {
     public Response clearState(
         @Context HttpServletRequest httpServletRequest,
         @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
-        )
-        @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-        @ApiParam(
-            value = "The revision is used to verify the client is working with the latest version of the flow.",
+            value = "The revision used to verify the client is working with the latest version of the flow.",
             required = true
         )
-        @FormParam(VERSION) LongParameter version,
+        Entity revisionEntity,
         @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",
@@ -681,23 +615,18 @@ public class ControllerServiceResource extends ApplicationResource {
             return generateContinueResponse().build();
         }
 
-        // get the revision specified by the user
-        Long revision = null;
-        if (version != null) {
-            revision = version.getLong();
-        }
-
         // get the component state
-        final ConfigurationSnapshot<Void> snapshot = serviceFacade.clearControllerServiceState(new Revision(revision, clientId.getClientId()), id);
+        final RevisionDTO requestRevision = revisionEntity.getRevision();
+        final ConfigurationSnapshot<Void> snapshot = serviceFacade.clearControllerServiceState(new Revision(requestRevision.getVersion(), requestRevision.getClientId()), id);
 
         // create the revision
-        final RevisionDTO revisionDTO = new RevisionDTO();
-        revisionDTO.setClientId(clientId.getClientId());
-        revisionDTO.setVersion(snapshot.getVersion());
+        final RevisionDTO responseRevision = new RevisionDTO();
+        responseRevision.setClientId(requestRevision.getClientId());
+        responseRevision.setVersion(snapshot.getVersion());
 
         // generate the response entity
         final ComponentStateEntity entity = new ComponentStateEntity();
-        entity.setRevision(revisionDTO);
+        entity.setRevision(responseRevision);
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -717,9 +646,9 @@ public class ControllerServiceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}/references")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}/references")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets a controller service",
             response = ControllerServiceEntity.class,
@@ -782,29 +711,20 @@ public class ControllerServiceResource extends ApplicationResource {
      * Updates the references of the specified controller service.
      *
      * @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 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.
-     * @param id The id of the controller service to retrieve
-     * @param state Sets the state of referencing components. A value of RUNNING
-     * or STOPPED will update referencing schedulable components (Processors and
-     * Reporting Tasks). A value of ENABLED or DISABLED will update referencing
-     * controller services.
-     * @return A controllerServiceEntity.
+     * @param updateReferenceRequest The update request
+     * @return A controllerServiceReferencingComponentsEntity.
      */
     @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}/references")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}/references")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a controller services references",
-            response = ControllerServiceEntity.class,
+            response = ControllerServiceReferencingComponentsEntity.class,
             authorizations = {
                 @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
             }
@@ -821,32 +741,19 @@ public class ControllerServiceResource extends ApplicationResource {
     public Response updateControllerServiceReferences(
             @Context HttpServletRequest httpServletRequest,
             @ApiParam(
-                    value = "The revision is used to verify the client is working with the latest version of the flow.",
-                    required = false
-            )
-            @FormParam(VERSION) LongParameter version,
-            @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
-            )
-            @FormParam(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
+                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
             )
             @PathParam("availability") String availability,
             @ApiParam(
-                    value = "The controller service id.",
-                    required = true
-            )
-            @PathParam("id") String id,
-            @ApiParam(
-                    value = "The new state of the references for the controller service.",
-                    allowableValues = "ENABLED, DISABLED, RUNNING, STOPPED",
-                    required = true
-            )
-            @FormParam("state") @DefaultValue(StringUtils.EMPTY) String state) {
+                value = "The controller service request update request.",
+                required = true
+            ) UpdateControllerServiceReferenceRequestEntity updateReferenceRequest) {
+
+        if (updateReferenceRequest.getId() == null) {
+            throw new IllegalArgumentException("The controller service identifier must be specified.");
+        }
 
         // parse the state to determine the desired action
         // need to consider controller service state first as it shares a state with
@@ -854,14 +761,14 @@ public class ControllerServiceResource extends ApplicationResource {
         // but not referencing schedulable components
         ControllerServiceState controllerServiceState = null;
         try {
-            controllerServiceState = ControllerServiceState.valueOf(state);
+            controllerServiceState = ControllerServiceState.valueOf(updateReferenceRequest.getState());
         } catch (final IllegalArgumentException iae) {
             // ignore
         }
 
         ScheduledState scheduledState = null;
         try {
-            scheduledState = ScheduledState.valueOf(state);
+            scheduledState = ScheduledState.valueOf(updateReferenceRequest.getState());
         } catch (final IllegalArgumentException iae) {
             // ignore
         }
@@ -889,23 +796,18 @@ public class ControllerServiceResource extends ApplicationResource {
         // handle expects request (usually from the cluster manager)
         final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
         if (expects != null) {
-            serviceFacade.verifyUpdateControllerServiceReferencingComponents(id, scheduledState, controllerServiceState);
+            serviceFacade.verifyUpdateControllerServiceReferencingComponents(updateReferenceRequest.getId(), scheduledState, controllerServiceState);
             return generateContinueResponse().build();
         }
 
-        // determine the specified version
-        Long clientVersion = null;
-        if (version != null) {
-            clientVersion = version.getLong();
-        }
-
         // get the controller service
-        final ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> response
-                = serviceFacade.updateControllerServiceReferencingComponents(new Revision(clientVersion, clientId.getClientId()), id, scheduledState, controllerServiceState);
+        final RevisionDTO requestRevision = updateReferenceRequest.getRevision();
+        final ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> response = serviceFacade.updateControllerServiceReferencingComponents(
+            new Revision(requestRevision.getVersion(), requestRevision.getClientId()), updateReferenceRequest.getId(), scheduledState, controllerServiceState);
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        revision.setClientId(requestRevision.getClientId());
         revision.setVersion(response.getVersion());
 
         // create the response entity
@@ -917,109 +819,6 @@ public class ControllerServiceResource extends ApplicationResource {
     }
 
     /**
-     * Updates the specified controller service.
-     *
-     * @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 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.
-     * @param id The id of the controller service to update.
-     * @param name The name of the controller service
-     * @param annotationData The annotation data for the controller service
-     * @param comments The comments for the controller service
-     * @param state The state of this controller service. Should be ENABLED or
-     * DISABLED.
-     * @param markedForDeletion Array of property names whose value should be
-     * removed.
-     * @param formParams Additionally, the processor properties and styles are
-     * specified in the form parameters. Because the property names and styles
-     * differ from processor to processor they are specified in a map-like
-     * fashion:
-     * <br>
-     * <ul>
-     * <li>properties[required.file.path]=/path/to/file</li>
-     * <li>properties[required.hostname]=localhost</li>
-     * <li>properties[required.port]=80</li>
-     * <li>properties[optional.file.path]=/path/to/file</li>
-     * <li>properties[optional.hostname]=localhost</li>
-     * <li>properties[optional.port]=80</li>
-     * <li>properties[user.defined.pattern]=^.*?s.*$</li>
-     * </ul>
-     * @return A controllerServiceEntity.
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response updateControllerService(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @PathParam("availability") String availability, @PathParam("id") String id, @FormParam("name") String name,
-            @FormParam("annotationData") String annotationData, @FormParam("comments") String comments,
-            @FormParam("state") String state, @FormParam("markedForDeletion[]") List<String> markedForDeletion,
-            MultivaluedMap<String, String> formParams) {
-
-        // create collections for holding the controller service properties
-        final Map<String, String> updatedProperties = new LinkedHashMap<>();
-
-        // go through each parameter and look for processor properties
-        for (String parameterName : formParams.keySet()) {
-            if (StringUtils.isNotBlank(parameterName)) {
-                // see if the parameter name starts with an expected parameter type...
-                // if so, store the parameter name and value in the corresponding collection
-                if (parameterName.startsWith("properties")) {
-                    final int startIndex = StringUtils.indexOf(parameterName, "[");
-                    final int endIndex = StringUtils.lastIndexOf(parameterName, "]");
-                    if (startIndex != -1 && endIndex != -1) {
-                        final String propertyName = StringUtils.substring(parameterName, startIndex + 1, endIndex);
-                        updatedProperties.put(propertyName, formParams.getFirst(parameterName));
-                    }
-                }
-            }
-        }
-
-        // set the properties to remove
-        for (String propertyToDelete : markedForDeletion) {
-            updatedProperties.put(propertyToDelete, null);
-        }
-
-        // create the controller service DTO
-        final ControllerServiceDTO controllerServiceDTO = new ControllerServiceDTO();
-        controllerServiceDTO.setId(id);
-        controllerServiceDTO.setName(name);
-        controllerServiceDTO.setAnnotationData(annotationData);
-        controllerServiceDTO.setComments(comments);
-        controllerServiceDTO.setState(state);
-
-        // only set the properties when appropriate
-        if (!updatedProperties.isEmpty()) {
-            controllerServiceDTO.setProperties(updatedProperties);
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the controller service entity
-        final ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity();
-        controllerServiceEntity.setRevision(revision);
-        controllerServiceEntity.setControllerService(controllerServiceDTO);
-
-        // update the controller service
-        return updateControllerService(httpServletRequest, availability, id, controllerServiceEntity);
-    }
-
-    /**
      * Updates the specified a new Controller Service.
      *
      * @param httpServletRequest request
@@ -1031,10 +830,10 @@ public class ControllerServiceResource extends ApplicationResource {
      * @return A controllerServiceEntity.
      */
     @PUT
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a controller service",
             response = ControllerServiceEntity.class,
@@ -1145,9 +944,9 @@ public class ControllerServiceResource extends ApplicationResource {
      */
     @DELETE
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{availability}/{id}")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes a controller service",
             response = ControllerServiceEntity.class,

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/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 0ab6c32..541241c 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
@@ -16,57 +16,48 @@
  */
 package org.apache.nifi.web.api;
 
-import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiParam;
 import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+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.api.dto.FunnelDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.FunnelEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+import org.apache.nifi.web.api.request.LongParameter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
-import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
 import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-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.api.dto.FunnelDTO;
-import org.apache.nifi.web.api.dto.PositionDTO;
-import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.entity.FunnelEntity;
-import org.apache.nifi.web.api.entity.FunnelsEntity;
-import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.nifi.web.api.request.DoubleParameter;
-import org.apache.nifi.web.api.request.LongParameter;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.access.prepost.PreAuthorize;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * RESTful endpoint for managing a Funnel.
  */
-@Api(hidden = true)
+@Path("funnels")
 public class FunnelResource extends ApplicationResource {
 
     private static final Logger logger = LoggerFactory.getLogger(FunnelResource.class);
@@ -74,7 +65,6 @@ public class FunnelResource extends ApplicationResource {
     private NiFiServiceFacade serviceFacade;
     private WebClusterManager clusterManager;
     private NiFiProperties properties;
-    private String groupId;
 
     /**
      * Populates the uri for the specified funnels.
@@ -92,220 +82,13 @@ public class FunnelResource extends ApplicationResource {
     /**
      * Populates the uri for the specified funnel.
      */
-    private FunnelDTO populateRemainingFunnelContent(FunnelDTO funnel) {
+    public FunnelDTO populateRemainingFunnelContent(FunnelDTO funnel) {
         // populate the funnel href
-        funnel.setUri(generateResourceUri("controller", "process-groups", groupId, "funnels", funnel.getId()));
+        funnel.setUri(generateResourceUri("funnels", funnel.getId()));
         return funnel;
     }
 
     /**
-     * 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
-    @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Gets all funnels",
-            response = FunnelsEntity.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 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) {
-
-        // replicate if cluster manager
-        if (properties.isClusterManager()) {
-            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
-        }
-
-        // get all the funnels
-        final Set<FunnelDTO> funnels = populateRemainingFunnelsContent(serviceFacade.getFunnels(groupId));
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        final FunnelsEntity entity = new FunnelsEntity();
-        entity.setRevision(revision);
-        entity.setFunnels(funnels);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
-     * Creates a new funnel.
-     *
-     * @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 x The x coordinate for this funnels position.
-     * @param y The y coordinate for this funnels position.
-     * @return A funnelEntity.
-     */
-    @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response createFunnel(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @FormParam("x") DoubleParameter x, @FormParam("y") DoubleParameter y) {
-
-        // ensure the position has been specified
-        if (x == null || y == null) {
-            throw new IllegalArgumentException("The position (x, y) must be specified");
-        }
-
-        // create the funnel DTO
-        final FunnelDTO funnelDTO = new FunnelDTO();
-        funnelDTO.setPosition(new PositionDTO(x.getDouble(), y.getDouble()));
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the funnel entity
-        final FunnelEntity funnelEntity = new FunnelEntity();
-        funnelEntity.setRevision(revision);
-        funnelEntity.setFunnel(funnelDTO);
-
-        return createFunnel(httpServletRequest, funnelEntity);
-    }
-
-    /**
-     * Creates a new Funnel.
-     *
-     * @param httpServletRequest request
-     * @param funnelEntity A funnelEntity.
-     * @return A funnelEntity.
-     */
-    @POST
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    @ApiOperation(
-            value = "Creates a funnel",
-            response = FunnelEntity.class,
-            authorizations = {
-                @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
-            }
-    )
-    @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 createFunnel(
-            @Context HttpServletRequest httpServletRequest,
-            @ApiParam(
-                    value = "The funnel configuration details.",
-                    required = true
-            ) FunnelEntity funnelEntity) {
-
-        if (funnelEntity == null || funnelEntity.getFunnel() == null) {
-            throw new IllegalArgumentException("Funnel details must be specified.");
-        }
-
-        if (funnelEntity.getRevision() == null) {
-            throw new IllegalArgumentException("Revision must be specified.");
-        }
-
-        if (funnelEntity.getFunnel().getId() != null) {
-            throw new IllegalArgumentException("Funnel ID cannot be specified.");
-        }
-
-        // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate
-        if (properties.isClusterManager()) {
-
-            // create ID for resource
-            final String id = UUID.randomUUID().toString();
-
-            // set ID for resource
-            funnelEntity.getFunnel().setId(id);
-
-            // convert POST request to PUT request to force entity ID to be the same across nodes
-            URI putUri = null;
-            try {
-                putUri = new URI(getAbsolutePath().toString() + "/" + id);
-            } catch (final URISyntaxException e) {
-                throw new WebApplicationException(e);
-            }
-
-            // change content type to JSON for serializing entity
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
-
-            // replicate put request
-            return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(funnelEntity), getHeaders(headersToOverride)).getResponse();
-        }
-
-        // handle expects request (usually from the cluster manager)
-        final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
-        if (expects != null) {
-            return generateContinueResponse().build();
-        }
-
-        // 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();
-        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);
-
-        // build the response
-        return clusterContext(generateCreatedResponse(URI.create(funnel.getUri()), entity)).build();
-    }
-
-    /**
      * Retrieves the specified funnel.
      *
      * @param clientId Optional client id. If the client id is not specified, a
@@ -316,9 +99,9 @@ public class FunnelResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets a funnel",
             response = FunnelEntity.class,
@@ -355,7 +138,7 @@ public class FunnelResource extends ApplicationResource {
         }
 
         // get the funnel
-        final FunnelDTO funnel = serviceFacade.getFunnel(groupId, id);
+        final FunnelDTO funnel = serviceFacade.getFunnel(id);
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
@@ -370,60 +153,6 @@ public class FunnelResource extends ApplicationResource {
     }
 
     /**
-     * Updates the specified funnel.
-     *
-     * @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 id The id of the funnel to update.
-     * @param parentGroupId The id of the process group to move this funnel to.
-     * @param x The x coordinate for this funnels position.
-     * @param y The y coordinate for this funnels position.
-     * @return A funnelEntity.
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response updateFunnel(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @PathParam("id") String id, @FormParam("parentGroupId") String parentGroupId,
-            @FormParam("x") DoubleParameter x, @FormParam("y") DoubleParameter y) {
-
-        // create the funnel DTO
-        final FunnelDTO funnelDTO = new FunnelDTO();
-        funnelDTO.setId(id);
-        funnelDTO.setParentGroupId(parentGroupId);
-
-        // require both coordinates to be specified
-        if (x != null && y != null) {
-            funnelDTO.setPosition(new PositionDTO(x.getDouble(), y.getDouble()));
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the funnel entity
-        final FunnelEntity funnelEntity = new FunnelEntity();
-        funnelEntity.setRevision(revision);
-        funnelEntity.setFunnel(funnelDTO);
-
-        // update the funnel
-        return updateFunnel(httpServletRequest, id, funnelEntity);
-    }
-
-    /**
      * Creates a new Funnel.
      *
      * @param httpServletRequest request
@@ -432,10 +161,10 @@ public class FunnelResource extends ApplicationResource {
      * @return A funnelEntity.
      */
     @PUT
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a funnel",
             response = FunnelEntity.class,
@@ -498,7 +227,7 @@ public class FunnelResource extends ApplicationResource {
         // update the funnel
         final RevisionDTO revision = funnelEntity.getRevision();
         final ConfigurationSnapshot<FunnelDTO> controllerResponse = serviceFacade.updateFunnel(
-                new Revision(revision.getVersion(), revision.getClientId()), groupId, requestFunnelDTO);
+                new Revision(revision.getVersion(), revision.getClientId()), requestFunnelDTO);
 
         // get the results
         final FunnelDTO responseFunnelDTO = controllerResponse.getConfiguration();
@@ -535,9 +264,9 @@ public class FunnelResource extends ApplicationResource {
      */
     @DELETE
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes a funnel",
             response = FunnelEntity.class,
@@ -580,7 +309,7 @@ public class FunnelResource extends ApplicationResource {
         // handle expects request (usually from the cluster manager)
         final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
         if (expects != null) {
-            serviceFacade.verifyDeleteFunnel(groupId, id);
+            serviceFacade.verifyDeleteFunnel(id);
             return generateContinueResponse().build();
         }
 
@@ -591,7 +320,7 @@ public class FunnelResource extends ApplicationResource {
         }
 
         // delete the specified funnel
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteFunnel(new Revision(clientVersion, clientId.getClientId()), groupId, id);
+        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteFunnel(new Revision(clientVersion, clientId.getClientId()), id);
 
         // get the updated revision
         final RevisionDTO revision = new RevisionDTO();
@@ -610,10 +339,6 @@ public class FunnelResource extends ApplicationResource {
         this.serviceFacade = serviceFacade;
     }
 
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
     public void setClusterManager(WebClusterManager clusterManager) {
         this.clusterManager = clusterManager;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.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/HistoryResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java
index 7462ff8..47c2b17 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java
@@ -16,12 +16,24 @@
  */
 package org.apache.nifi.web.api;
 
-import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiParam;
 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.web.NiFiServiceFacade;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.action.ActionDTO;
+import org.apache.nifi.web.api.dto.action.HistoryDTO;
+import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
+import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ComponentHistoryEntity;
+import org.apache.nifi.web.api.entity.HistoryEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+import org.apache.nifi.web.api.request.DateTimeParameter;
+import org.apache.nifi.web.api.request.IntegerParameter;
+
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
@@ -32,25 +44,11 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.apache.nifi.web.api.entity.ActionEntity;
-import org.apache.nifi.web.api.entity.HistoryEntity;
-import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.nifi.web.api.request.DateTimeParameter;
-import org.apache.nifi.web.api.request.IntegerParameter;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.web.NiFiServiceFacade;
-import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID;
-import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.dto.action.ActionDTO;
-import org.apache.nifi.web.api.dto.action.HistoryDTO;
-import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
-import org.apache.nifi.web.api.entity.ComponentHistoryEntity;
-import org.springframework.security.access.prepost.PreAuthorize;
 
 /**
  * RESTful endpoint for querying the history of this Controller.
  */
-@Api(hidden = true)
+@Path("history")
 public class HistoryResource extends ApplicationResource {
 
     private NiFiServiceFacade serviceFacade;
@@ -85,9 +83,9 @@ public class HistoryResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets configuration history",
             response = HistoryEntity.class,
@@ -234,8 +232,8 @@ public class HistoryResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @Path("{id}")
     @ApiOperation(
             value = "Gets an action",
@@ -299,9 +297,9 @@ public class HistoryResource extends ApplicationResource {
      */
     @DELETE
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasRole('ROLE_ADMIN')")
     @ApiOperation(
             value = "Purges history",
             response = HistoryEntity.class,
@@ -360,9 +358,9 @@ public class HistoryResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/processors/{processorId}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("processors/{processorId}")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets configuration history for a processor",
             response = ComponentHistoryEntity.class,
@@ -417,9 +415,9 @@ public class HistoryResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/controller-services/{controllerServiceId}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("controller-services/{controllerServiceId}")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets configuration history for a controller service",
             response = ComponentHistoryEntity.class,
@@ -474,9 +472,9 @@ public class HistoryResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/reporting-tasks/{reportingTaskId}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("reporting-tasks/{reportingTaskId}")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets configuration history for a reporting task",
             response = ComponentHistoryEntity.class,

http://git-wip-us.apache.org/repos/asf/nifi/blob/153f63ef/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 2f7eed6..568628e 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
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.web.api;
 
-import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiParam;
 import com.wordnik.swagger.annotations.ApiResponse;
@@ -33,54 +32,42 @@ import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.PortDTO;
-import org.apache.nifi.web.api.dto.PositionDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.status.PortStatusDTO;
 import org.apache.nifi.web.api.entity.InputPortEntity;
-import org.apache.nifi.web.api.entity.InputPortsEntity;
 import org.apache.nifi.web.api.entity.PortStatusEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.nifi.web.api.request.DoubleParameter;
-import org.apache.nifi.web.api.request.IntegerParameter;
 import org.apache.nifi.web.api.request.LongParameter;
-import org.springframework.security.access.prepost.PreAuthorize;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
-import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
 import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.core.Response;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 /**
  * RESTful endpoint for managing an Input Port.
  */
-@Api(hidden = true)
+@Path("input-ports")
 public class InputPortResource extends ApplicationResource {
 
     private NiFiServiceFacade serviceFacade;
     private WebClusterManager clusterManager;
     private NiFiProperties properties;
-    private String groupId;
 
     /**
      * Populates the uri for the specified input ports.
@@ -98,220 +85,13 @@ public class InputPortResource extends ApplicationResource {
     /**
      * Populates the uri for the specified input ports.
      */
-    private PortDTO populateRemainingInputPortContent(PortDTO inputPort) {
+    public PortDTO populateRemainingInputPortContent(PortDTO inputPort) {
         // populate the input port uri
-        inputPort.setUri(generateResourceUri("controller", "process-groups", inputPort.getParentGroupId(), "input-ports", inputPort.getId()));
+        inputPort.setUri(generateResourceUri("input-ports", inputPort.getId()));
         return inputPort;
     }
 
     /**
-     * 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
-    @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Gets all input ports",
-            response = InputPortsEntity.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 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) {
-
-        // replicate if cluster manager
-        if (properties.isClusterManager()) {
-            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
-        }
-
-        // get all the input ports
-        final Set<PortDTO> inputPorts = populateRemainingInputPortsContent(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);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
-     * Creates a new input port.
-     *
-     * @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 x The x coordinate for this funnels position.
-     * @param y The y coordinate for this funnels position.
-     * @param name The input ports name.
-     * @return A inputPortEntity.
-     */
-    @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response createInputPort(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @FormParam("x") DoubleParameter x, @FormParam("y") DoubleParameter y,
-            @FormParam("name") String name) {
-
-        // ensure the position has been specified
-        if (x == null || y == null) {
-            throw new IllegalArgumentException("The position (x, y) must be specified");
-        }
-
-        // create the input port DTO
-        final PortDTO inputPortDTO = new PortDTO();
-        inputPortDTO.setPosition(new PositionDTO(x.getDouble(), y.getDouble()));
-        inputPortDTO.setName(name);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the input port entity entity
-        final InputPortEntity portEntity = new InputPortEntity();
-        portEntity.setRevision(revision);
-        portEntity.setInputPort(inputPortDTO);
-
-        // create the input port
-        return createInputPort(httpServletRequest, portEntity);
-    }
-
-    /**
-     * Creates a new input port.
-     *
-     * @param httpServletRequest request
-     * @param portEntity A inputPortEntity.
-     * @return A inputPortEntity.
-     */
-    @POST
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    @ApiOperation(
-            value = "Creates an input port",
-            response = InputPortEntity.class,
-            authorizations = {
-                @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
-            }
-    )
-    @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 createInputPort(
-            @Context HttpServletRequest httpServletRequest,
-            @ApiParam(
-                    value = "The input port configuration details.",
-                    required = true
-            ) InputPortEntity portEntity) {
-
-        if (portEntity == null || portEntity.getInputPort() == null) {
-            throw new IllegalArgumentException("Port details must be specified.");
-        }
-
-        if (portEntity.getRevision() == null) {
-            throw new IllegalArgumentException("Revision must be specified.");
-        }
-
-        if (portEntity.getInputPort().getId() != null) {
-            throw new IllegalArgumentException("Input port ID cannot be specified.");
-        }
-
-        // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate
-        if (properties.isClusterManager()) {
-
-            // create ID for resource
-            final String id = UUID.randomUUID().toString();
-
-            // set ID for resource
-            portEntity.getInputPort().setId(id);
-
-            // convert POST request to PUT request to force entity ID to be the same across nodes
-            URI putUri = null;
-            try {
-                putUri = new URI(getAbsolutePath().toString() + "/" + id);
-            } catch (final URISyntaxException e) {
-                throw new WebApplicationException(e);
-            }
-
-            // change content type to JSON for serializing entity
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
-
-            // replicate put request
-            return clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(portEntity), getHeaders(headersToOverride)).getResponse();
-
-        }
-
-        // handle expects request (usually from the cluster manager)
-        final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
-        if (expects != null) {
-            return generateContinueResponse().build();
-        }
-
-        // 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();
-        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);
-
-        // build the response
-        return clusterContext(generateCreatedResponse(URI.create(port.getUri()), entity)).build();
-    }
-
-    /**
      * 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.
@@ -320,9 +100,9 @@ public class InputPortResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets an input port",
             response = InputPortEntity.class,
@@ -359,7 +139,7 @@ public class InputPortResource extends ApplicationResource {
         }
 
         // get the port
-        final PortDTO port = serviceFacade.getInputPort(groupId, id);
+        final PortDTO port = serviceFacade.getInputPort(id);
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
@@ -382,9 +162,9 @@ public class InputPortResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("/{id}/status")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
         value = "Gets status for an input port",
         response = PortStatusEntity.class,
@@ -458,7 +238,7 @@ public class InputPortResource extends ApplicationResource {
         }
 
         // get the specified input port status
-        final PortStatusDTO portStatus = serviceFacade.getInputPortStatus(groupId, id);
+        final PortStatusDTO portStatus = serviceFacade.getInputPortStatus(id);
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
@@ -477,95 +257,15 @@ public class InputPortResource extends ApplicationResource {
      * Updates the specified input port.
      *
      * @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 id The id of the input port to update.
-     * @param x The x coordinate for this funnels position.
-     * @param y The y coordinate for this funnels position.
-     * @param groupAccessControl The allowed groups for this input port.
-     * @param userAccessControl The allowed users for this input port.
-     * @param comments Any comments about this input port.
-     * @param name The input ports name.
-     * @param state The state of this port.
-     * @param concurrentlySchedulableTaskCount The number of concurrently schedulable tasks.
-     * @param formParams form params
-     * @return A inputPortEntity.
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response updateInputPort(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @PathParam("id") String id,
-            @FormParam("x") DoubleParameter x, @FormParam("y") DoubleParameter y,
-            @FormParam("comments") String comments,
-            @FormParam("groupAccessControl[]") Set<String> groupAccessControl,
-            @FormParam("userAccessControl[]") Set<String> userAccessControl,
-            @FormParam("name") String name,
-            @FormParam("state") String state,
-            @FormParam("concurrentlySchedulableTaskCount") IntegerParameter concurrentlySchedulableTaskCount,
-            MultivaluedMap<String, String> formParams) {
-
-        // create the input port DTO
-        final PortDTO portDTO = new PortDTO();
-        portDTO.setId(id);
-        portDTO.setComments(comments);
-        portDTO.setName(name);
-        portDTO.setState(state);
-
-        if (concurrentlySchedulableTaskCount != null) {
-            portDTO.setConcurrentlySchedulableTaskCount(concurrentlySchedulableTaskCount.getInteger());
-        }
-
-        // require both coordinates to be specified
-        if (x != null && y != null) {
-            portDTO.setPosition(new PositionDTO(x.getDouble(), y.getDouble()));
-        }
-
-        // only set the group access control when applicable
-        if (!groupAccessControl.isEmpty() || formParams.containsKey("groupAccessControl[]")) {
-            portDTO.setGroupAccessControl(groupAccessControl);
-        }
-
-        // only set the user access control when applicable
-        if (!userAccessControl.isEmpty() || formParams.containsKey("userAccessControl[]")) {
-            portDTO.setUserAccessControl(userAccessControl);
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the input port entity
-        final InputPortEntity portEntity = new InputPortEntity();
-        portEntity.setRevision(revision);
-        portEntity.setInputPort(portDTO);
-
-        // update the port
-        return updateInputPort(httpServletRequest, id, portEntity);
-    }
-
-    /**
-     * Updates the specified input port.
-     *
-     * @param httpServletRequest request
      * @param id The id of the input port to update.
      * @param portEntity A inputPortEntity.
      * @return A inputPortEntity.
      */
     @PUT
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates an input port",
             response = InputPortEntity.class,
@@ -622,14 +322,14 @@ public class InputPortResource extends ApplicationResource {
         // handle expects request (usually from the cluster manager)
         final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
         if (expects != null) {
-            serviceFacade.verifyUpdateInputPort(groupId, requestPortDTO);
+            serviceFacade.verifyUpdateInputPort(requestPortDTO);
             return generateContinueResponse().build();
         }
 
         // update the input port
         final RevisionDTO revision = portEntity.getRevision();
         final ConfigurationSnapshot<PortDTO> controllerResponse = serviceFacade.updateInputPort(
-                new Revision(revision.getVersion(), revision.getClientId()), groupId, requestPortDTO);
+                new Revision(revision.getVersion(), revision.getClientId()), requestPortDTO);
 
         // get the results
         final PortDTO responsePortDTO = controllerResponse.getConfiguration();
@@ -663,9 +363,9 @@ public class InputPortResource extends ApplicationResource {
      */
     @DELETE
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes an input port",
             response = InputPortEntity.class,
@@ -708,7 +408,7 @@ public class InputPortResource extends ApplicationResource {
         // handle expects request (usually from the cluster manager)
         final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
         if (expects != null) {
-            serviceFacade.verifyDeleteInputPort(groupId, id);
+            serviceFacade.verifyDeleteInputPort(id);
             return generateContinueResponse().build();
         }
 
@@ -719,7 +419,7 @@ public class InputPortResource extends ApplicationResource {
         }
 
         // delete the specified input port
-        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteInputPort(new Revision(clientVersion, clientId.getClientId()), groupId, id);
+        final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteInputPort(new Revision(clientVersion, clientId.getClientId()), id);
 
         // get the updated revision
         final RevisionDTO revision = new RevisionDTO();
@@ -738,10 +438,6 @@ public class InputPortResource extends ApplicationResource {
         this.serviceFacade = serviceFacade;
     }
 
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
     public void setClusterManager(WebClusterManager clusterManager) {
         this.clusterManager = clusterManager;
     }