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

[08/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/ReportingTaskResource.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/ReportingTaskResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
index 802f46f..4a746b3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
@@ -16,36 +16,14 @@
  */
 package org.apache.nifi.web.api;
 
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-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;
-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 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.ui.extension.UiExtension;
 import org.apache.nifi.ui.extension.UiExtensionMapping;
@@ -59,25 +37,40 @@ import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
 import org.apache.nifi.web.api.dto.ReportingTaskDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.entity.ComponentStateEntity;
+import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
 import org.apache.nifi.web.api.entity.ReportingTaskEntity;
 import org.apache.nifi.web.api.entity.ReportingTasksEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 import org.apache.nifi.web.util.Availability;
-import org.springframework.security.access.prepost.PreAuthorize;
 
-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 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.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.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
 
 /**
  * RESTful endpoint for managing a Reporting Task.
  */
-@Api(hidden = true)
+@Path("reporting-tasks")
 public class ReportingTaskResource extends ApplicationResource {
 
     private NiFiServiceFacade serviceFacade;
@@ -105,7 +98,7 @@ public class ReportingTaskResource extends ApplicationResource {
      */
     private ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) {
         // populate the reporting task href
-        reportingTask.setUri(generateResourceUri("controller", "reporting-tasks", availability, reportingTask.getId()));
+        reportingTask.setUri(generateResourceUri("reporting-tasks", availability, reportingTask.getId()));
         reportingTask.setAvailability(availability);
 
         // see if this processor has any ui extensions
@@ -155,9 +148,9 @@ public class ReportingTaskResource 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 reporting tasks",
             response = ReportingTasksEntity.class,
@@ -212,52 +205,6 @@ public class ReportingTaskResource extends ApplicationResource {
     }
 
     /**
-     * Creates a new reporting task.
-     *
-     * @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 reporting task is available on the NCM
-     * only (ncm) or on the nodes only (node). If this instance is not clustered
-     * all tasks should use the node availability.
-     * @param type The type of reporting task to create.
-     * @return A reportingTaskEntity.
-     */
-    @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response createReportingTask(
-            @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 reporting task DTO
-        final ReportingTaskDTO reportingTaskDTO = new ReportingTaskDTO();
-        reportingTaskDTO.setType(type);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the reporting task entity
-        final ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity();
-        reportingTaskEntity.setRevision(revision);
-        reportingTaskEntity.setReportingTask(reportingTaskDTO);
-
-        return createReportingTask(httpServletRequest, availability, reportingTaskEntity);
-    }
-
-    /**
      * Creates a new Reporting Task.
      *
      * @param httpServletRequest request
@@ -268,10 +215,10 @@ public class ReportingTaskResource extends ApplicationResource {
      * @return A reportingTaskEntity.
      */
     @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 reporting task",
             response = ReportingTaskEntity.class,
@@ -321,28 +268,8 @@ public class ReportingTaskResource extends ApplicationResource {
         // get the revision
         final RevisionDTO revision = reportingTaskEntity.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
-            reportingTaskEntity.getReportingTask().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(reportingTaskEntity), getHeaders(headersToOverride)).getResponse();
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders()).getResponse();
         }
 
         // handle expects request (usually from the cluster manager)
@@ -351,6 +278,14 @@ public class ReportingTaskResource extends ApplicationResource {
             return generateContinueResponse().build();
         }
 
+        // set the processor id as appropriate
+        final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
+        if (clusterContext != null) {
+            reportingTaskEntity.getReportingTask().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
+        } else {
+            reportingTaskEntity.getReportingTask().setId(UUID.randomUUID().toString());
+        }
+
         // create the reporting task and generate the json
         final ConfigurationSnapshot<ReportingTaskDTO> controllerResponse = serviceFacade.createReportingTask(
                 new Revision(revision.getVersion(), revision.getClientId()), reportingTaskEntity.getReportingTask());
@@ -384,9 +319,9 @@ public class ReportingTaskResource 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 reporting task",
             response = ReportingTaskEntity.class,
@@ -458,9 +393,9 @@ public class ReportingTaskResource 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 reporting task property descriptor",
             response = PropertyDescriptorEntity.class,
@@ -542,9 +477,9 @@ public class ReportingTaskResource 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 reporting task",
         response = ComponentStateDTO.class,
@@ -605,8 +540,7 @@ public class ReportingTaskResource extends ApplicationResource {
     /**
      * Clears the state for a reporting task.
      *
-     * @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 reporting task 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.
@@ -614,10 +548,10 @@ public class ReportingTaskResource 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 reporting task",
         response = ComponentStateDTO.class,
@@ -637,15 +571,10 @@ public class ReportingTaskResource 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 reporting task is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
             allowableValues = "NCM, NODE",
@@ -672,136 +601,24 @@ public class ReportingTaskResource 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.clearReportingTaskState(new Revision(revision, clientId.getClientId()), id);
+        final RevisionDTO requestRevision = revisionEntity.getRevision();
+        final ConfigurationSnapshot<Void> snapshot = serviceFacade.clearReportingTaskState(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();
     }
 
     /**
-     * Updates the specified reporting task.
-     *
-     * @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 reporting task is available on the NCM
-     * only (ncm) or on the nodes only (node). If this instance is not clustered
-     * all tasks should use the node availability.
-     * @param id The id of the reporting task to update.
-     * @param name The name of the reporting task
-     * @param annotationData The annotation data for the reporting task
-     * @param markedForDeletion Array of property names whose value should be
-     * removed.
-     * @param state The updated scheduled state
-     * @param schedulingStrategy The scheduling strategy for this reporting task
-     * @param schedulingPeriod The scheduling period for this reporting task
-     * @param comments The comments for this reporting task
-     * @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 reportingTaskEntity.
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{availability}/{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response updateReportingTask(
-            @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("markedForDeletion[]") List<String> markedForDeletion,
-            @FormParam("state") String state, @FormParam("schedulingStrategy") String schedulingStrategy,
-            @FormParam("schedulingPeriod") String schedulingPeriod, @FormParam("comments") String comments,
-            MultivaluedMap<String, String> formParams) {
-
-        // create collections for holding the reporting task 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 reporting task DTO
-        final ReportingTaskDTO reportingTaskDTO = new ReportingTaskDTO();
-        reportingTaskDTO.setId(id);
-        reportingTaskDTO.setName(name);
-        reportingTaskDTO.setState(state);
-        reportingTaskDTO.setSchedulingStrategy(schedulingStrategy);
-        reportingTaskDTO.setSchedulingPeriod(schedulingPeriod);
-        reportingTaskDTO.setAnnotationData(annotationData);
-        reportingTaskDTO.setComments(comments);
-
-        // only set the properties when appropriate
-        if (!updatedProperties.isEmpty()) {
-            reportingTaskDTO.setProperties(updatedProperties);
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the reporting task entity
-        final ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity();
-        reportingTaskEntity.setRevision(revision);
-        reportingTaskEntity.setReportingTask(reportingTaskDTO);
-
-        // update the reporting task
-        return updateReportingTask(httpServletRequest, availability, id, reportingTaskEntity);
-    }
-
-    /**
      * Updates the specified a Reporting Task.
      *
      * @param httpServletRequest request
@@ -813,10 +630,10 @@ public class ReportingTaskResource extends ApplicationResource {
      * @return A reportingTaskEntity.
      */
     @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 reporting task",
             response = ReportingTaskEntity.class,
@@ -870,12 +687,7 @@ public class ReportingTaskResource extends ApplicationResource {
 
         // replicate if cluster manager
         if (properties.isClusterManager() && Availability.NODE.equals(avail)) {
-            // change content type to JSON for serializing entity
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
-
-            // replicate the request
-            return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders(headersToOverride)).getResponse();
+            return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders()).getResponse();
         }
 
         // handle expects request (usually from the cluster manager)
@@ -927,9 +739,9 @@ public class ReportingTaskResource 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 reporting task",
             response = ReportingTaskEntity.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/SnippetResource.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/SnippetResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
index 247eac1..5eddcf7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
@@ -23,13 +23,21 @@ 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.HashSet;
-import java.util.List;
-import java.util.Map;
-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.FlowSnippetDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.SnippetDTO;
+import org.apache.nifi.web.api.entity.SnippetEntity;
+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;
@@ -47,21 +55,13 @@ 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.FlowSnippetDTO;
-import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.dto.SnippetDTO;
-import org.apache.nifi.web.api.entity.SnippetEntity;
-import org.apache.nifi.web.api.request.ClientIdParameter;
-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.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
 
 /**
  * RESTful endpoint for managing a Snippet.
@@ -80,94 +80,14 @@ public class SnippetResource extends ApplicationResource {
     private WebClusterManager clusterManager;
     private NiFiProperties properties;
 
-    /**
-     * Get the processor resource within the specified group.
-     *
-     * @return the processor resource within the specified group
-     */
-    private ProcessorResource getProcessorResource(final String groupId) {
-        ProcessorResource processorResource = resourceContext.getResource(ProcessorResource.class);
-        processorResource.setGroupId(groupId);
-        return processorResource;
-    }
-
-    /**
-     * Get the connection sub-resource within the specified group.
-     *
-     * @return the connection sub-resource within the specified group
-     */
-    private ConnectionResource getConnectionResource(final String groupId) {
-        ConnectionResource connectionResource = resourceContext.getResource(ConnectionResource.class);
-        connectionResource.setGroupId(groupId);
-        return connectionResource;
-    }
-
-    /**
-     * Get the input ports sub-resource within the specified group.
-     *
-     * @return the input ports sub-resource within the specified group
-     */
-    private InputPortResource getInputPortResource(final String groupId) {
-        InputPortResource inputPortResource = resourceContext.getResource(InputPortResource.class);
-        inputPortResource.setGroupId(groupId);
-        return inputPortResource;
-    }
-
-    /**
-     * Get the output ports sub-resource within the specified group.
-     *
-     * @return the output ports sub-resource within the specified group
-     */
-    private OutputPortResource getOutputPortResource(final String groupId) {
-        OutputPortResource outputPortResource = resourceContext.getResource(OutputPortResource.class);
-        outputPortResource.setGroupId(groupId);
-        return outputPortResource;
-    }
-
-    /**
-     * Locates the label sub-resource within the specified group.
-     *
-     * @return the label sub-resource within the specified group
-     */
-    private LabelResource getLabelResource(final String groupId) {
-        LabelResource labelResource = resourceContext.getResource(LabelResource.class);
-        labelResource.setGroupId(groupId);
-        return labelResource;
-    }
-
-    /**
-     * Locates the funnel sub-resource within the specified group.
-     *
-     * @return the funnel sub-resource within the specified group
-     */
-    private FunnelResource getFunnelResource(final String groupId) {
-        FunnelResource funnelResource = resourceContext.getResource(FunnelResource.class);
-        funnelResource.setGroupId(groupId);
-        return funnelResource;
-    }
-
-    /**
-     * Locates the remote process group sub-resource within the specified group.
-     *
-     * @return the remote process group sub-resource within the specified group
-     */
-    private RemoteProcessGroupResource getRemoteProcessGroupResource(final String groupId) {
-        RemoteProcessGroupResource remoteProcessGroupResource = resourceContext.getResource(RemoteProcessGroupResource.class);
-        remoteProcessGroupResource.setGroupId(groupId);
-        return remoteProcessGroupResource;
-    }
-
-    /**
-     * Locates the process group sub-resource within the specified group.
-     *
-     * @param groupId group id
-     * @return the process group sub-resource within the specified group
-     */
-    private ProcessGroupResource getProcessGroupResource(final String groupId) {
-        ProcessGroupResource processGroupResource = resourceContext.getResource(ProcessGroupResource.class);
-        processGroupResource.setGroupId(groupId);
-        return processGroupResource;
-    }
+    private ProcessorResource processorResource;
+    private InputPortResource inputPortResource;
+    private OutputPortResource outputPortResource;
+    private FunnelResource funnelResource;
+    private LabelResource labelResource;
+    private RemoteProcessGroupResource remoteProcessGroupResource;
+    private ConnectionResource connectionResource;
+    private ProcessGroupResource processGroupResource;
 
     /**
      * Populates the uri for the specified snippet.
@@ -181,14 +101,14 @@ public class SnippetResource extends ApplicationResource {
 
         // populate the snippet content uris
         if (snippet.getContents() != null) {
-            getProcessorResource(snippetGroupId).populateRemainingProcessorsContent(snippetContents.getProcessors());
-            getConnectionResource(snippetGroupId).populateRemainingConnectionsContent(snippetContents.getConnections());
-            getInputPortResource(snippetGroupId).populateRemainingInputPortsContent(snippetContents.getInputPorts());
-            getOutputPortResource(snippetGroupId).populateRemainingOutputPortsContent(snippetContents.getOutputPorts());
-            getRemoteProcessGroupResource(snippetGroupId).populateRemainingRemoteProcessGroupsContent(snippetContents.getRemoteProcessGroups());
-            getFunnelResource(snippetGroupId).populateRemainingFunnelsContent(snippetContents.getFunnels());
-            getLabelResource(snippetGroupId).populateRemainingLabelsContent(snippetContents.getLabels());
-            getProcessGroupResource(snippetGroupId).populateRemainingProcessGroupsContent(snippetContents.getProcessGroups());
+            processorResource.populateRemainingProcessorsContent(snippetContents.getProcessors());
+            connectionResource.populateRemainingConnectionsContent(snippetContents.getConnections());
+            inputPortResource.populateRemainingInputPortsContent(snippetContents.getInputPorts());
+            outputPortResource.populateRemainingOutputPortsContent(snippetContents.getOutputPorts());
+            remoteProcessGroupResource.populateRemainingRemoteProcessGroupsContent(snippetContents.getRemoteProcessGroups());
+            funnelResource.populateRemainingFunnelsContent(snippetContents.getFunnels());
+            labelResource.populateRemainingLabelsContent(snippetContents.getLabels());
+            processGroupResource.populateRemainingProcessGroupsContent(snippetContents.getProcessGroups());
         }
 
         return snippet;
@@ -222,7 +142,7 @@ public class SnippetResource extends ApplicationResource {
     @POST
     @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     public Response createSnippet(
             @Context HttpServletRequest httpServletRequest,
             @FormParam(VERSION) LongParameter version,
@@ -279,7 +199,7 @@ public class SnippetResource extends ApplicationResource {
     @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')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Creates a snippet",
             response = SnippetEntity.class,
@@ -393,7 +313,7 @@ public class SnippetResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("{id}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets a snippet",
             response = SnippetEntity.class,
@@ -477,7 +397,7 @@ public class SnippetResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     public Response updateSnippet(
             @Context HttpServletRequest httpServletRequest,
             @FormParam(VERSION) LongParameter version,
@@ -523,7 +443,7 @@ public class SnippetResource extends ApplicationResource {
     @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a snippet",
             response = SnippetEntity.class,
@@ -629,7 +549,7 @@ public class SnippetResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes a snippet",
             response = SnippetEntity.class,
@@ -706,6 +626,38 @@ public class SnippetResource extends ApplicationResource {
         this.clusterManager = clusterManager;
     }
 
+    public void setProcessorResource(ProcessorResource processorResource) {
+        this.processorResource = processorResource;
+    }
+
+    public void setInputPortResource(InputPortResource inputPortResource) {
+        this.inputPortResource = inputPortResource;
+    }
+
+    public void setOutputPortResource(OutputPortResource outputPortResource) {
+        this.outputPortResource = outputPortResource;
+    }
+
+    public void setFunnelResource(FunnelResource funnelResource) {
+        this.funnelResource = funnelResource;
+    }
+
+    public void setLabelResource(LabelResource labelResource) {
+        this.labelResource = labelResource;
+    }
+
+    public void setRemoteProcessGroupResource(RemoteProcessGroupResource remoteProcessGroupResource) {
+        this.remoteProcessGroupResource = remoteProcessGroupResource;
+    }
+
+    public void setConnectionResource(ConnectionResource connectionResource) {
+        this.connectionResource = connectionResource;
+    }
+
+    public void setProcessGroupResource(ProcessGroupResource processGroupResource) {
+        this.processGroupResource = processGroupResource;
+    }
+
     public void setProperties(NiFiProperties properties) {
         this.properties = properties;
     }

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/SystemDiagnosticsResource.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/SystemDiagnosticsResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SystemDiagnosticsResource.java
index 1bde7bf..213190a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SystemDiagnosticsResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SystemDiagnosticsResource.java
@@ -34,7 +34,6 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
 import org.apache.nifi.web.api.entity.SystemDiagnosticsEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.springframework.security.access.prepost.PreAuthorize;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DefaultValue;
@@ -74,7 +73,7 @@ public class SystemDiagnosticsResource extends ApplicationResource {
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets the diagnostics for the system NiFi is running on",
             response = SystemDiagnosticsEntity.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/TemplateResource.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/TemplateResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java
index 03debbb..673373a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java
@@ -23,12 +23,18 @@ 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.io.InputStream;
-import java.net.URI;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
+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.NiFiServiceFacade;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.entity.TemplateEntity;
+import org.apache.nifi.web.api.entity.TemplatesEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
@@ -49,19 +55,12 @@ import javax.xml.bind.JAXBElement;
 import javax.xml.bind.JAXBException;
 import javax.xml.bind.Unmarshaller;
 import javax.xml.transform.stream.StreamSource;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.util.NiFiProperties;
-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.TemplateDTO;
-import org.apache.nifi.web.api.entity.TemplateEntity;
-import org.apache.nifi.web.api.entity.TemplatesEntity;
-import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.access.prepost.PreAuthorize;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * RESTful endpoint for managing a Template.
@@ -109,7 +108,7 @@ public class TemplateResource extends ApplicationResource {
     @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')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Gets all templates",
             response = TemplatesEntity.class,
@@ -172,7 +171,7 @@ public class TemplateResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Creates a template",
             response = TemplateEntity.class,
@@ -254,7 +253,7 @@ public class TemplateResource extends ApplicationResource {
     @Consumes(MediaType.MULTIPART_FORM_DATA)
     @Produces(MediaType.APPLICATION_XML)
     @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     public Response importTemplate(
             @Context HttpServletRequest httpServletRequest,
             @FormDataParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@@ -316,7 +315,7 @@ public class TemplateResource extends ApplicationResource {
     @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Produces(MediaType.APPLICATION_XML)
     @Path("") // necessary due to bug in swagger
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     public Response importTemplate(
             @Context HttpServletRequest httpServletRequest,
             TemplateEntity templateEntity) {
@@ -377,7 +376,7 @@ public class TemplateResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_XML)
     @Path("{id}")
-    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
     @ApiOperation(
             value = "Exports a template",
             response = TemplateDTO.class,
@@ -445,7 +444,7 @@ public class TemplateResource extends ApplicationResource {
     @Consumes(MediaType.WILDCARD)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
     @Path("{id}")
-    @PreAuthorize("hasRole('ROLE_DFM')")
+    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes a template",
             response = TemplateEntity.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/UserGroupResource.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/UserGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/UserGroupResource.java
deleted file mode 100644
index 3a0b596..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/UserGroupResource.java
+++ /dev/null
@@ -1,465 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.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.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-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.HttpMethod;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.Response;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.web.api.entity.UserGroupEntity;
-import org.apache.nifi.web.api.request.ClientIdParameter;
-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.UserGroupDTO;
-import org.springframework.security.access.prepost.PreAuthorize;
-
-/**
- * RESTful endpoint for managing this Controller's user groups.
- */
-@Api(hidden = true)
-public class UserGroupResource extends ApplicationResource {
-
-    /*
-     * Developer Note: Clustering assumes a centralized security provider. The
-     * cluster manager will manage user accounts when in clustered mode and
-     * interface with the authorization provider. However, when nodes perform
-     * Site-to-Site, the authorization details of the remote NiFi will be cached
-     * locally. These details need to be invalidated when certain actions are
-     * performed (revoking/deleting accounts, changing user authorities, user
-     * group, etc).
-     */
-    private WebClusterManager clusterManager;
-    private NiFiProperties properties;
-    private NiFiServiceFacade serviceFacade;
-
-    /**
-     * Updates a new user group.
-     *
-     * @param httpServletRequest request
-     * @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 userIds A collection of user ids to include in this group. If a
-     * user already belongs to another group, they will be placed in this group
-     * instead. Existing users in this group will remain in this group.
-     * @param group The name of the group.
-     * @param rawAuthorities Array of authorities to assign to the specified
-     * user.
-     * @param status The status of the specified users account.
-     * @param formParams form params
-     * @return A userGroupEntity.
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{group}")
-    @PreAuthorize("hasRole('ROLE_ADMIN')")
-    public Response updateUserGroup(
-            @Context HttpServletRequest httpServletRequest,
-            @PathParam("group") String group,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @FormParam("userIds[]") Set<String> userIds,
-            @FormParam("authorities[]") Set<String> rawAuthorities,
-            @FormParam("status") String status,
-            MultivaluedMap<String, String> formParams) {
-
-        // get the collection of specified authorities
-        final Set<String> authorities = new HashSet<>();
-        for (String authority : rawAuthorities) {
-            if (StringUtils.isNotBlank(authority)) {
-                authorities.add(authority);
-            }
-        }
-
-        // create the user group dto
-        final UserGroupDTO userGroup = new UserGroupDTO();
-        userGroup.setGroup(group);
-        userGroup.setUserIds(userIds);
-        userGroup.setStatus(status);
-
-        // set the authorities
-        if (!authorities.isEmpty() || formParams.containsKey("authorities")) {
-            userGroup.setAuthorities(authorities);
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the user group entity
-        final UserGroupEntity entity = new UserGroupEntity();
-        entity.setRevision(revision);
-        entity.setUserGroup(userGroup);
-
-        // create the user group
-        return updateUserGroup(httpServletRequest, group, entity);
-    }
-
-    /**
-     * Creates a new user group with the specified users.
-     *
-     * @param httpServletRequest request
-     * @param group The user group.
-     * @param userGroupEntity A userGroupEntity.
-     * @return A userGroupEntity.
-     */
-    @PUT
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{group}")
-    @PreAuthorize("hasRole('ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Updates a user group",
-            response = UserGroupEntity.class,
-            authorizations = {
-                @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 updateUserGroup(
-            @Context HttpServletRequest httpServletRequest,
-            @ApiParam(
-                    value = "The name of the user group.",
-                    required = true
-            )
-            @PathParam("group") String group,
-            @ApiParam(
-                    value = "The user group configuration details.",
-                    required = true
-            )
-            UserGroupEntity userGroupEntity) {
-
-        if (userGroupEntity == null || userGroupEntity.getUserGroup() == null) {
-            throw new IllegalArgumentException("User group details must be specified.");
-        }
-
-        // get the user group
-        UserGroupDTO userGroup = userGroupEntity.getUserGroup();
-
-        // ensure the same id is being used
-        if (!group.equals(userGroup.getGroup())) {
-            throw new IllegalArgumentException(String.format("The user group (%s) in the request body does "
-                    + "not equal the user group of the requested resource (%s).", userGroup.getGroup(), group));
-        }
-
-        // the user group must be specified and cannot be blank
-        if (StringUtils.isBlank(userGroup.getGroup())) {
-            throw new IllegalArgumentException("User group must be specified and cannot be blank.");
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        if (userGroupEntity.getRevision() == null) {
-            revision.setClientId(new ClientIdParameter().getClientId());
-        } else {
-            revision.setClientId(userGroupEntity.getRevision().getClientId());
-        }
-
-        // this user is being modified, replicate to the nodes to invalidate this account
-        // so that it will be re-authorized during the next attempted access - if this wasn't
-        // done the account would remain stale for up to the configured cache duration. this
-        // is acceptable sometimes but when updating a users authorities or groups via the UI
-        // they shouldn't have to wait for the changes to take effect`
-        if (properties.isClusterManager()) {
-            // change content type to JSON for serializing entity
-            final Map<String, String> headersToOverride = new HashMap<>();
-            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
-
-            // identify yourself as the NCM attempting to invalidate the user
-            final Map<String, String> headers = getHeaders(headersToOverride);
-            headers.put(WebClusterManager.CLUSTER_INVALIDATE_USER_GROUP_HEADER, Boolean.TRUE.toString());
-
-            final RevisionDTO invalidateUserRevision = new RevisionDTO();
-            revision.setClientId(revision.getClientId());
-
-            final UserGroupDTO invalidateUserGroup = new UserGroupDTO();
-            invalidateUserGroup.setGroup(group);
-            invalidateUserGroup.setUserIds(userGroup.getUserIds());
-
-            final UserGroupEntity invalidateUserGroupEntity = new UserGroupEntity();
-            invalidateUserGroupEntity.setRevision(invalidateUserRevision);
-            invalidateUserGroupEntity.setUserGroup(invalidateUserGroup);
-
-            // replicate the invalidate request to each node - if this request is not successful return that fact,
-            // otherwise continue with the desired user modification
-            final NodeResponse response = clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), invalidateUserGroupEntity, headers);
-            if (!response.is2xx()) {
-                return response.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();
-        }
-
-        // handle an invalidate request from the NCM
-        final String invalidateRequest = httpServletRequest.getHeader(WebClusterManager.CLUSTER_INVALIDATE_USER_GROUP_HEADER);
-        if (invalidateRequest != null) {
-            serviceFacade.invalidateUserGroup(userGroup.getGroup(), userGroup.getUserIds());
-            return generateOkResponse().build();
-        }
-
-        // create the user group
-        userGroup = serviceFacade.updateUserGroup(userGroup);
-
-        // create the response entity
-        final UserGroupEntity entity = new UserGroupEntity();
-        entity.setRevision(revision);
-        entity.setUserGroup(userGroup);
-
-        // generate the URI for this group and return
-        return generateOkResponse(entity).build();
-    }
-
-    /**
-     * Deletes the user from the specified group. The user will not be removed,
-     * just the fact that they were in this group.
-     *
-     * @param httpServletRequest request
-     * @param group The user group.
-     * @param userId The user id to remove.
-     * @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 userGroupEntity.
-     */
-    @DELETE
-    @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{group}/users/{userId}")
-    @PreAuthorize("hasRole('ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Removes a user from a user group",
-            notes = "Removes a user from a user group. The will not be deleted, jsut the fact that they were in this group.",
-            response = UserGroupEntity.class,
-            authorizations = {
-                @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 removeUserFromGroup(
-            @Context HttpServletRequest httpServletRequest,
-            @ApiParam(
-                    value = "The name of the user group.",
-                    required = true
-            )
-            @PathParam("group") String group,
-            @ApiParam(
-                    value = "The id of the user to remove from the user group.",
-                    required = true
-            )
-            @PathParam("userId") String userId,
-            @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) {
-
-        // this user is being modified, replicate to the nodes to invalidate this account
-        // so that it will be re-authorized during the next attempted access - if this wasn't
-        // done the account would remain stale for up to the configured cache duration. this
-        // is acceptable sometimes but when removing a user via the UI they shouldn't have to
-        // wait for the changes to take effect
-        if (properties.isClusterManager()) {
-            // identify yourself as the NCM attempting to invalidate the user
-            final Map<String, String> headers = getHeaders();
-            headers.put(WebClusterManager.CLUSTER_INVALIDATE_USER_HEADER, Boolean.TRUE.toString());
-
-            // replicate the invalidate request to each node - if this request is not successful return that fact,
-            // otherwise continue with the desired user modification
-            final NodeResponse response = clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), headers);
-            if (!response.is2xx()) {
-                return response.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();
-        }
-
-        // handle an invalidate request from the NCM
-        final String invalidateRequest = httpServletRequest.getHeader(WebClusterManager.CLUSTER_INVALIDATE_USER_HEADER);
-        if (invalidateRequest != null) {
-            serviceFacade.invalidateUser(userId);
-            return generateOkResponse().build();
-        }
-
-        // ungroup the specified user
-        serviceFacade.removeUserFromGroup(userId);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        final UserGroupEntity entity = new UserGroupEntity();
-        entity.setRevision(revision);
-
-        // generate ok response
-        return generateOkResponse(entity).build();
-    }
-
-    /**
-     * Deletes the user group. The users will not be removed, just the fact that
-     * they were grouped.
-     *
-     * @param httpServletRequest request
-     * @param group The user 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.
-     * @return A userGroupEntity.
-     */
-    @DELETE
-    @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{group}")
-    @PreAuthorize("hasRole('ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Deletes a user group",
-            notes = "Deletes a user group. The users will not be removed, just the fact that they were grouped.",
-            response = UserGroupEntity.class,
-            authorizations = {
-                @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 ungroup(
-            @Context HttpServletRequest httpServletRequest,
-            @ApiParam(
-                    value = "The name of the user group.",
-                    required = true
-            )
-            @PathParam("group") String group,
-            @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) {
-
-        // this user is being modified, replicate to the nodes to invalidate this account
-        // so that it will be re-authorized during the next attempted access - if this wasn't
-        // done the account would remain stale for up to the configured cache duration. this
-        // is acceptable sometimes but when removing a user via the UI they shouldn't have to
-        // wait for the changes to take effect
-        if (properties.isClusterManager()) {
-            // identify yourself as the NCM attempting to invalidate the user
-            final Map<String, String> headers = getHeaders();
-            headers.put(WebClusterManager.CLUSTER_INVALIDATE_USER_GROUP_HEADER, Boolean.TRUE.toString());
-
-            // replicate the invalidate request to each node - if this request is not successful return that fact,
-            // otherwise continue with the desired user modification
-            final NodeResponse response = clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), headers);
-            if (!response.is2xx()) {
-                return response.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();
-        }
-
-        // handle an invalidate request from the NCM
-        final String invalidateRequest = httpServletRequest.getHeader(WebClusterManager.CLUSTER_INVALIDATE_USER_GROUP_HEADER);
-        if (invalidateRequest != null) {
-            serviceFacade.invalidateUserGroup(group, null);
-            return generateOkResponse().build();
-        }
-
-        // delete the user group
-        serviceFacade.removeUserGroup(group);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        final UserGroupEntity entity = new UserGroupEntity();
-        entity.setRevision(revision);
-
-        // generate ok response
-        return generateOkResponse(entity).build();
-    }
-
-    /* setters */
-    public void setServiceFacade(NiFiServiceFacade serviceFacade) {
-        this.serviceFacade = serviceFacade;
-    }
-
-    public void setProperties(NiFiProperties properties) {
-        this.properties = properties;
-    }
-
-    public void setClusterManager(WebClusterManager clusterManager) {
-        this.clusterManager = clusterManager;
-    }
-}