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

[03/10] nifi git commit: NIFI-1554: - Introducing new REST endpoints to align with the authorizable resources. - Additionally changes to support the new endpoints. - Addressing comments in PR. - This closes #374.

http://git-wip-us.apache.org/repos/asf/nifi/blob/add29816/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
index 076b3c2..6202561 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
@@ -16,18 +16,16 @@
  */
 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.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
-import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
 import org.apache.nifi.cluster.node.Node;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.ui.extension.UiExtension;
 import org.apache.nifi.ui.extension.UiExtensionMapping;
@@ -41,14 +39,10 @@ import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
 import org.apache.nifi.web.api.entity.ComponentStateEntity;
 import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.ProcessorEntity;
-import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
 import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
-import org.apache.nifi.web.api.entity.StatusHistoryEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 
@@ -70,14 +64,17 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.net.URI;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
 /**
  * RESTful endpoint for managing a Processor.
  */
-@Path("processors")
+@Path("/processors")
+@Api(
+    value = "/processors",
+    description = "Endpoint for managing a Processor."
+)
 public class ProcessorResource extends ApplicationResource {
 
     private static final List<Long> POSSIBLE_RUN_DURATIONS = Arrays.asList(0L, 25L, 50L, 100L, 250L, 500L, 1000L, 2000L);
@@ -197,169 +194,6 @@ public class ProcessorResource extends ApplicationResource {
     }
 
     /**
-     * Retrieves the specified processor status.
-     *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
-     * @param id The id of the processor history to retrieve.
-     * @return A processorStatusEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/status")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-        value = "Gets status for a processor",
-        response = ProcessorStatusEntity.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 getProcessorStatus(
-        @ApiParam(
-            value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-            required = false
-        )
-        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-        @ApiParam(
-            value = "Whether or not to include the breakdown per node. Optional, defaults to false",
-            required = false
-        )
-        @QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
-        @ApiParam(
-            value = "The id of the node where to get the status.",
-            required = false
-        )
-        @QueryParam("clusterNodeId") String clusterNodeId,
-        @ApiParam(
-            value = "The processor id.",
-            required = true
-        )
-        @PathParam("id") String id) {
-
-        // ensure a valid request
-        if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
-            throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
-        }
-
-        if (properties.isClusterManager()) {
-            // determine where this request should be sent
-            if (clusterNodeId == null) {
-                final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
-                final ProcessorStatusEntity entity = (ProcessorStatusEntity) nodeResponse.getUpdatedEntity();
-
-                // ensure there is an updated entity (result of merging) and prune the response as necessary
-                if (entity != null && !nodewise) {
-                    entity.getProcessorStatus().setNodeSnapshots(null);
-                }
-
-                return nodeResponse.getResponse();
-            } else {
-                // get the target node and ensure it exists
-                final Node targetNode = clusterManager.getNode(clusterNodeId);
-                if (targetNode == null) {
-                    throw new UnknownNodeException("The specified cluster node does not exist.");
-                }
-
-                final Set<NodeIdentifier> targetNodes = new HashSet<>();
-                targetNodes.add(targetNode.getNodeId());
-
-                // replicate the request to the specific node
-                return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
-            }
-        }
-
-        // get the specified processor status
-        final ProcessorStatusDTO processorStatus = serviceFacade.getProcessorStatus(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // generate the response entity
-        final ProcessorStatusEntity entity = new ProcessorStatusEntity();
-        entity.setRevision(revision);
-        entity.setProcessorStatus(processorStatus);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
-     * Retrieves the specified processor status history.
-     *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
-     * @param id The id of the processor history to retrieve.
-     * @return A statusHistoryEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/status/history")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Gets status history for a processor",
-            response = StatusHistoryEntity.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 getProcessorStatusHistory(
-            @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
-                    value = "The processor id.",
-                    required = true
-            )
-            @PathParam("id") String id) {
-
-        // replicate if cluster manager
-        if (properties.isClusterManager()) {
-            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
-        }
-
-        // get the specified processor status history
-        final StatusHistoryDTO processorStatusHistory = serviceFacade.getProcessorStatusHistory(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // generate the response entity
-        final StatusHistoryEntity entity = new StatusHistoryEntity();
-        entity.setRevision(revision);
-        entity.setStatusHistory(processorStatusHistory);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
      * Returns the descriptor for the specified property.
      *
      * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.

http://git-wip-us.apache.org/repos/asf/nifi/blob/add29816/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.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/ProvenanceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
index 670b16a..bc9bcc8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
@@ -38,17 +38,14 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
 import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
 import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
-import org.apache.nifi.web.api.dto.provenance.ProvenanceRequestDTO;
 import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO;
 import org.apache.nifi.web.api.dto.provenance.lineage.LineageRequestDTO;
-import org.apache.nifi.web.api.dto.provenance.lineage.LineageRequestDTO.LineageRequestType;
 import org.apache.nifi.web.api.entity.LineageEntity;
 import org.apache.nifi.web.api.entity.ProvenanceEntity;
 import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
 import org.apache.nifi.web.api.entity.ProvenanceOptionsEntity;
+import org.apache.nifi.web.api.entity.SubmitReplayRequestEntity;
 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.nifi.web.api.request.LongParameter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -57,7 +54,6 @@ 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;
@@ -68,7 +64,6 @@ 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 javax.ws.rs.core.StreamingOutput;
 import java.io.IOException;
@@ -78,7 +73,6 @@ import java.net.URI;
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -86,7 +80,11 @@ import java.util.UUID;
 /**
  * RESTful endpoint for querying data provenance.
  */
-@Api(hidden = true)
+@Path("/provenance")
+@Api(
+    value = "/provenance",
+    description = "Endpoint for accessing data flow provenance."
+)
 public class ProvenanceResource extends ApplicationResource {
 
     private static final Logger logger = LoggerFactory.getLogger(ProvenanceResource.class);
@@ -100,7 +98,7 @@ public class ProvenanceResource extends ApplicationResource {
      * Populates the uri for the specified provenance.
      */
     private ProvenanceDTO populateRemainingProvenanceContent(ProvenanceDTO provenance) {
-        provenance.setUri(generateResourceUri("controller", "provenance", provenance.getId()));
+        provenance.setUri(generateResourceUri("provenance", provenance.getId()));
         return provenance;
     }
 
@@ -108,7 +106,7 @@ public class ProvenanceResource extends ApplicationResource {
      * Populates the uri for the specified lineage.
      */
     private LineageDTO populateRemainingLineageContent(LineageDTO lineage) {
-        lineage.setUri(generateResourceUri("controller", "provenance", "lineage", lineage.getId()));
+        lineage.setUri(generateResourceUri("provenance", "lineage", lineage.getId()));
         return lineage;
     }
 
@@ -120,8 +118,8 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/search-options")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("search-options")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Gets the searchable attributes for provenance events",
@@ -170,15 +168,13 @@ public class ProvenanceResource extends ApplicationResource {
      * Creates a new replay request for the content associated with the specified provenance event id.
      *
      * @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 clusterNodeId The id of the node in the cluster that has the specified event. Required if clustered.
-     * @param eventId The provenance event id.
+     * @param replayRequestEntity The replay request
      * @return A provenanceEventEntity
      */
     @POST
-    @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/replays")
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("replays")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE') and hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Replays content from a provenance event",
@@ -197,36 +193,25 @@ public class ProvenanceResource extends ApplicationResource {
             }
     )
     public Response submitReplay(
-            @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 id of the node where the content exists if clustered.",
-                    required = false
-            )
-            @FormParam("clusterNodeId") String clusterNodeId,
-            @ApiParam(
-                    value = "The provenance event id.",
-                    required = true
-            )
-            @FormParam("eventId") LongParameter eventId) {
+        @Context HttpServletRequest httpServletRequest,
+        @ApiParam(
+            value = "The replay request.",
+            required = true
+        ) SubmitReplayRequestEntity replayRequestEntity) {
 
         // ensure the event id is specified
-        if (eventId == null) {
+        if (replayRequestEntity == null || replayRequestEntity.getEventId() == null) {
             throw new IllegalArgumentException("The id of the event must be specified.");
         }
 
         // replicate if cluster manager
         if (properties.isClusterManager()) {
             // determine where this request should be sent
-            if (clusterNodeId == null) {
+            if (replayRequestEntity.getClusterNodeId() == null) {
                 throw new IllegalArgumentException("The id of the node in the cluster is required.");
             } else {
                 // get the target node and ensure it exists
-                final Node targetNode = clusterManager.getNode(clusterNodeId);
+                final Node targetNode = clusterManager.getNode(replayRequestEntity.getClusterNodeId());
                 if (targetNode == null) {
                     throw new UnknownNodeException("The specified cluster node does not exist.");
                 }
@@ -235,7 +220,7 @@ public class ProvenanceResource extends ApplicationResource {
                 targetNodes.add(targetNode.getNodeId());
 
                 // replicate the request to the specific node
-                return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
+                return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), replayRequestEntity, getHeaders(), targetNodes).getResponse();
             }
         }
 
@@ -246,11 +231,12 @@ public class ProvenanceResource extends ApplicationResource {
         }
 
         // submit the provenance replay request
-        final ProvenanceEventDTO event = serviceFacade.submitReplay(eventId.getLong());
+        final RevisionDTO requestRevision = replayRequestEntity.getRevision();
+        final ProvenanceEventDTO event = serviceFacade.submitReplay(replayRequestEntity.getEventId());
 
         // create a revision to return
         final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
+        revision.setClientId(requestRevision.getClientId());
 
         // create a response entity
         final ProvenanceEventEntity entity = new ProvenanceEventEntity();
@@ -258,7 +244,7 @@ public class ProvenanceResource extends ApplicationResource {
         entity.setRevision(revision);
 
         // generate the response
-        URI uri = URI.create(generateResourceUri("controller", "provenance", "events", event.getId()));
+        URI uri = URI.create(generateResourceUri("provenance", "events", event.getId()));
         return clusterContext(generateCreatedResponse(uri, entity)).build();
     }
 
@@ -273,7 +259,7 @@ public class ProvenanceResource extends ApplicationResource {
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.WILDCARD)
-    @Path("/events/{id}/content/input")
+    @Path("events/{id}/content/input")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Gets the input content for a provenance event",
@@ -333,7 +319,7 @@ public class ProvenanceResource extends ApplicationResource {
         }
 
         // get the uri of the request
-        final String uri = generateResourceUri("controller", "provenance", "events", String.valueOf(id.getLong()), "content", "input");
+        final String uri = generateResourceUri("provenance", "events", String.valueOf(id.getLong()), "content", "input");
 
         // get an input stream to the content
         final DownloadableContent content = serviceFacade.getContent(id.getLong(), uri, ContentDirection.INPUT);
@@ -372,7 +358,7 @@ public class ProvenanceResource extends ApplicationResource {
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.WILDCARD)
-    @Path("/events/{id}/content/output")
+    @Path("events/{id}/content/output")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Gets the output content for a provenance event",
@@ -432,7 +418,7 @@ public class ProvenanceResource extends ApplicationResource {
         }
 
         // get the uri of the request
-        final String uri = generateResourceUri("controller", "provenance", "events", String.valueOf(id.getLong()), "content", "output");
+        final String uri = generateResourceUri("provenance", "events", String.valueOf(id.getLong()), "content", "output");
 
         // get an input stream to the content
         final DownloadableContent content = serviceFacade.getContent(id.getLong(), uri, ContentDirection.OUTPUT);
@@ -464,106 +450,12 @@ public class ProvenanceResource extends ApplicationResource {
      * Creates provenance using the specified query criteria.
      *
      * @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 startDate The start date.
-     * @param endDate The end date.
-     * @param minimumFileSize The minimum size of the content after the event.
-     * @param maximumFileSize The maximum size of the content after the event.
-     * @param maxResults The maximum number of results to return.
-     * @param clusterNodeId The id of node in the cluster to search. This is optional and only relevant when clustered. If clustered and it is not specified the entire cluster is searched.
-     * @param formParams Additionally, the search parameters are specified in the form parameters. Because the search parameters differ based on configuration they are specified in a map-like fashion:
-     * <br>
-     * <ul>
-     * <li>search[filename]=myFile.txt</li>
-     * <li>search[eventType]=RECEIVED</li>
-     * </ul>
-     *
-     * @return A provenanceEntity
-     */
-    @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("") // necessary due to bug in swagger
-    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
-    public Response submitProvenanceRequest(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @FormParam("startDate") DateTimeParameter startDate,
-            @FormParam("endDate") DateTimeParameter endDate,
-            @FormParam("minimumFileSize") String minimumFileSize,
-            @FormParam("maximumFileSize") String maximumFileSize,
-            @FormParam("maxResults") IntegerParameter maxResults,
-            @FormParam("clusterNodeId") String clusterNodeId,
-            MultivaluedMap<String, String> formParams) {
-
-        // ensure the max results has been specified
-        if (maxResults == null) {
-            throw new IllegalArgumentException("Max results must be specified.");
-        } else if (maxResults.getInteger() > MAX_MAX_RESULTS) {
-            throw new IllegalArgumentException("The maximum number of results cannot be greater than " + MAX_MAX_RESULTS);
-        }
-
-        // create collections for holding the search terms
-        final Map<String, String> searchTerms = 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("search[")) {
-                    final int startIndex = StringUtils.indexOf(parameterName, "[");
-                    final int endIndex = StringUtils.lastIndexOf(parameterName, "]");
-                    if (startIndex != -1 && endIndex != -1) {
-                        final String searchTerm = StringUtils.substring(parameterName, startIndex + 1, endIndex);
-                        searchTerms.put(searchTerm, formParams.getFirst(parameterName));
-                    }
-                }
-            }
-        }
-
-        // Build request object from all params
-        final ProvenanceRequestDTO request = new ProvenanceRequestDTO();
-        request.setSearchTerms(searchTerms);
-        request.setMinimumFileSize(minimumFileSize);
-        request.setMaximumFileSize(maximumFileSize);
-        request.setMaxResults(maxResults.getInteger());
-
-        // add date range
-        if (startDate != null) {
-            request.setStartDate(startDate.getDateTime());
-        }
-        if (endDate != null) {
-            request.setEndDate(endDate.getDateTime());
-        }
-
-        // build the provenance object
-        final ProvenanceDTO provenanceDto = new ProvenanceDTO();
-        provenanceDto.setClusterNodeId(clusterNodeId);
-        provenanceDto.setRequest(request);
-
-        // create a revision to return
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the request entity
-        final ProvenanceEntity entity = new ProvenanceEntity();
-        entity.setRevision(revision);
-        entity.setProvenance(provenanceDto);
-
-        return submitProvenanceRequest(httpServletRequest, entity);
-    }
-
-    /**
-     * Creates provenance using the specified query criteria.
-     *
-     * @param httpServletRequest request
      * @param provenanceEntity A provenanceEntity
      * @return A provenanceEntity
      */
     @POST
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
     @Path("") // necessary due to bug in swagger
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
@@ -679,8 +571,8 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{id}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Gets a provenance query",
@@ -765,8 +657,8 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @DELETE
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/{id}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Deletes a provenance query",
@@ -854,8 +746,8 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/events/{id}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("events/{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Gets a provenance event",
@@ -943,79 +835,13 @@ public class ProvenanceResource extends ApplicationResource {
      * When querying for the lineage of a flowfile you must specify the uuid. The eventId and eventDirection cannot be specified in this case.
      *
      * @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 eventId The id of an event to get the lineage for. Must also specify the eventDirection and not the uuid.
-     * @param lineageRequest Either 'PARENTS', 'CHILDREN', or 'FLOWFILE'. PARENTS will return the lineage for the flowfiles that are parents of the specified event. CHILDREN will return the lineage of
-     * for the flowfiles that are children of the specified event. FLOWFILE will return the lineage for the specified flowfile.
-     * @param uuid The uuid of the flowfile to get the lineage for. Must not specify the eventId or eventDirection.
-     * @param clusterNodeId The id of node in the cluster that the event/flowfile originated from. This is only required when clustered.
-     * @return A lineageEntity
-     */
-    @POST
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/lineage")
-    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
-    public Response submitLineageRequest(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @FormParam("lineageRequestType") String lineageRequest,
-            @FormParam("eventId") LongParameter eventId,
-            @FormParam("uuid") String uuid,
-            @FormParam("clusterNodeId") String clusterNodeId) {
-
-        // create the lineage request
-        final LineageRequestDTO request = new LineageRequestDTO();
-
-        // ensure the lineage request type is specified
-        try {
-            final LineageRequestType direction = LineageRequestType.valueOf(lineageRequest);
-            request.setLineageRequestType(direction);
-        } catch (final IllegalArgumentException iae) {
-            throw new IllegalArgumentException(String.format("The event direction must be one of %s", StringUtils.join(LineageRequestType.values())));
-        }
-
-        // set the uuid (may be null if based on event)
-        request.setUuid(uuid);
-
-        // set the event id (may be null is based on flowfile)
-        if (eventId != null) {
-            request.setEventId(eventId.getLong());
-        }
-
-        // create the lineage
-        final LineageDTO lineage = new LineageDTO();
-        lineage.setClusterNodeId(clusterNodeId);
-        lineage.setRequest(request);
-
-        // create a revision to return
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create a response entity
-        final LineageEntity entity = new LineageEntity();
-        entity.setLineage(lineage);
-        entity.setRevision(revision);
-
-        return submitLineageRequest(httpServletRequest, entity);
-    }
-
-    /**
-     * Submits a lineage request based on an event or a flowfile uuid.
-     *
-     * When querying for the lineage of an event you must specify the eventId and the eventDirection. The eventDirection must be 'parents' or 'children' and specifies whether we are going up or down
-     * the flowfile ancestry. The uuid cannot be specified in these cases.
-     *
-     * When querying for the lineage of a flowfile you must specify the uuid. The eventId and eventDirection cannot be specified in this case.
-     *
-     * @param httpServletRequest request
      * @param lineageEntity A lineageEntity
      * @return A lineageEntity
      */
     @POST
-    @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/lineage")
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("lineage")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Submits a lineage query",
@@ -1136,8 +962,8 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @GET
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/lineage/{id}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("lineage/{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Gets a lineage query",
@@ -1220,8 +1046,8 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @DELETE
     @Consumes(MediaType.WILDCARD)
-    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @Path("/lineage/{id}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("lineage/{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     @ApiOperation(
             value = "Deletes a lineage query",

http://git-wip-us.apache.org/repos/asf/nifi/blob/add29816/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.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/RemoteProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
index 868e647..d1999d9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
@@ -16,17 +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.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.cluster.node.Node;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.ConfigurationSnapshot;
 import org.apache.nifi.web.NiFiServiceFacade;
@@ -34,22 +31,15 @@ import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
-import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity;
-import org.apache.nifi.web.api.entity.StatusHistoryEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
-import org.apache.nifi.web.api.request.IntegerParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 
 import 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.PUT;
@@ -62,14 +52,17 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.net.URI;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
 /**
  * RESTful endpoint for managing a Remote group.
  */
-@Path("remote-process-groups")
+@Path("/remote-process-groups")
+@Api(
+    value = "/remote-process-groups",
+    description = "Endpoint for managing a Remote Process Group."
+)
 public class RemoteProcessGroupResource extends ApplicationResource {
 
     private static final String VERBOSE_DEFAULT_VALUE = "false";
@@ -178,169 +171,6 @@ public class RemoteProcessGroupResource extends ApplicationResource {
     }
 
     /**
-     * Retrieves the specified remote process group status.
-     *
-     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
-     * @param id The id of the processor history to retrieve.
-     * @return A remoteProcessGroupStatusEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/status")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-        value = "Gets status for a remote process group",
-        response = ProcessorStatusEntity.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 getRemoteProcessGroupStatus(
-        @ApiParam(
-            value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-            required = false
-        )
-        @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-        @ApiParam(
-            value = "Whether or not to include the breakdown per node. Optional, defaults to false",
-            required = false
-        )
-        @QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
-        @ApiParam(
-            value = "The id of the node where to get the status.",
-            required = false
-        )
-        @QueryParam("clusterNodeId") String clusterNodeId,
-        @ApiParam(
-            value = "The remote process group id.",
-            required = true
-        )
-        @PathParam("id") String id) {
-
-        // ensure a valid request
-        if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
-            throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
-        }
-
-        if (properties.isClusterManager()) {
-            // determine where this request should be sent
-            if (clusterNodeId == null) {
-                final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
-                final RemoteProcessGroupStatusEntity entity = (RemoteProcessGroupStatusEntity) nodeResponse.getUpdatedEntity();
-
-                // ensure there is an updated entity (result of merging) and prune the response as necessary
-                if (entity != null && !nodewise) {
-                    entity.getRemoteProcessGroupStatus().setNodeSnapshots(null);
-                }
-
-                return nodeResponse.getResponse();
-            } else {
-                // get the target node and ensure it exists
-                final Node targetNode = clusterManager.getNode(clusterNodeId);
-                if (targetNode == null) {
-                    throw new UnknownNodeException("The specified cluster node does not exist.");
-                }
-
-                final Set<NodeIdentifier> targetNodes = new HashSet<>();
-                targetNodes.add(targetNode.getNodeId());
-
-                // replicate the request to the specific node
-                return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
-            }
-        }
-
-        // get the specified remote process group status
-        final RemoteProcessGroupStatusDTO remoteProcessGroupStatus = serviceFacade.getRemoteProcessGroupStatus(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // generate the response entity
-        final RemoteProcessGroupStatusEntity entity = new RemoteProcessGroupStatusEntity();
-        entity.setRevision(revision);
-        entity.setRemoteProcessGroupStatus(remoteProcessGroupStatus);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
-     * Retrieves the specified remote process groups status history.
-     *
-     * @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 remote process group to retrieve the status fow.
-     * @return A statusHistoryEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/status/history")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Gets the status history",
-            response = StatusHistoryEntity.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 getRemoteProcessGroupStatusHistory(
-            @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
-                    value = "The remote process group id.",
-                    required = true
-            )
-            @PathParam("id") String id) {
-
-        // replicate if cluster manager
-        if (properties.isClusterManager()) {
-            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
-        }
-
-        // get the specified processor status history
-        final StatusHistoryDTO remoteProcessGroupStatusHistory = serviceFacade.getRemoteProcessGroupStatusHistory(id);
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // generate the response entity
-        final StatusHistoryEntity entity = new StatusHistoryEntity();
-        entity.setRevision(revision);
-        entity.setStatusHistory(remoteProcessGroupStatusHistory);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
      * Removes the specified remote process group.
      *
      * @param httpServletRequest request
@@ -352,7 +182,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
     @DELETE
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}")
+    @Path("{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Deletes a remote process group",
@@ -425,61 +255,6 @@ public class RemoteProcessGroupResource extends ApplicationResource {
      * Updates the specified remote process group 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 remote process group to update.
-     * @param portId The id of the input port to update.
-     * @param isTransmitting Whether or not this port is transmitting.
-     * @param isCompressed Whether or not this port should compress.
-     * @param concurrentlySchedulableTaskCount The number of concurrent tasks that should be supported
-     *
-     * @return A remoteProcessGroupPortEntity
-     */
-    @PUT
-    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/input-ports/{port-id}")
-    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
-    public Response updateRemoteProcessGroupInputPort(
-            @Context HttpServletRequest httpServletRequest,
-            @FormParam(VERSION) LongParameter version,
-            @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @PathParam("id") String id,
-            @PathParam("port-id") String portId,
-            @FormParam("transmitting") Boolean isTransmitting,
-            @FormParam("compressed") Boolean isCompressed,
-            @FormParam("concurrentlySchedulableTaskCount") IntegerParameter concurrentlySchedulableTaskCount) {
-
-        // create the remote group port dto
-        final RemoteProcessGroupPortDTO remotePort = new RemoteProcessGroupPortDTO();
-        remotePort.setId(portId);
-        remotePort.setUseCompression(isCompressed);
-        remotePort.setTransmitting(isTransmitting);
-
-        if (concurrentlySchedulableTaskCount != null) {
-            remotePort.setConcurrentlySchedulableTaskCount(concurrentlySchedulableTaskCount.getInteger());
-        }
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        if (version != null) {
-            revision.setVersion(version.getLong());
-        }
-
-        // create the remote group port entity
-        final RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity();
-        entity.setRevision(revision);
-        entity.setRemoteProcessGroupPort(remotePort);
-
-        return updateRemoteProcessGroupInputPort(httpServletRequest, id, portId, entity);
-    }
-
-    /**
-     * Updates the specified remote process group input port.
-     *
-     * @param httpServletRequest request
      * @param id The id of the remote process group to update.
      * @param portId The id of the input port to update.
      * @param remoteProcessGroupPortEntity The remoteProcessGroupPortEntity
@@ -489,7 +264,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
     @PUT
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/input-ports/{port-id}")
+    @Path("{id}/input-ports/{port-id}")
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a remote port",
@@ -578,7 +353,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
     @PUT
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}/output-ports/{port-id}")
+    @Path("{id}/output-ports/{port-id}")
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a remote port",
@@ -665,7 +440,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
     @PUT
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
-    @Path("/{id}")
+    @Path("{id}")
     // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
     @ApiOperation(
             value = "Updates a remote process group",

http://git-wip-us.apache.org/repos/asf/nifi/blob/add29816/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 4a746b3..2a48183 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,14 +16,13 @@
  */
 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.ui.extension.UiExtension;
 import org.apache.nifi.ui.extension.UiExtensionMapping;
@@ -40,7 +39,6 @@ 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;
@@ -62,15 +60,17 @@ 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.
  */
-@Path("reporting-tasks")
+@Path("/reporting-tasks")
+@Api(
+    value = "/reporting-tasks",
+    description = "Endpoint for managing a Reporting Task."
+)
 public class ReportingTaskResource extends ApplicationResource {
 
     private NiFiServiceFacade serviceFacade;
@@ -86,7 +86,7 @@ public class ReportingTaskResource extends ApplicationResource {
      * @param reportingTasks tasks
      * @return tasks
      */
-    private Set<ReportingTaskDTO> populateRemainingReportingTasksContent(final String availability, final Set<ReportingTaskDTO> reportingTasks) {
+    public Set<ReportingTaskDTO> populateRemainingReportingTasksContent(final String availability, final Set<ReportingTaskDTO> reportingTasks) {
         for (ReportingTaskDTO reportingTask : reportingTasks) {
             populateRemainingReportingTaskContent(availability, reportingTask);
         }
@@ -96,7 +96,7 @@ public class ReportingTaskResource extends ApplicationResource {
     /**
      * Populates the uri for the specified reporting task.
      */
-    private ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) {
+    public ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) {
         // populate the reporting task href
         reportingTask.setUri(generateResourceUri("reporting-tasks", availability, reportingTask.getId()));
         reportingTask.setAvailability(availability);
@@ -119,7 +119,7 @@ public class ReportingTaskResource extends ApplicationResource {
      * Parses the availability and ensure that the specified availability makes
      * sense for the given NiFi instance.
      */
-    private Availability parseAvailability(final String availability) {
+    public Availability parseAvailability(final String availability) {
         final Availability avail;
         try {
             avail = Availability.valueOf(availability.toUpperCase());
@@ -136,176 +136,6 @@ public class ReportingTaskResource extends ApplicationResource {
     }
 
     /**
-     * Retrieves all the of reporting tasks in this NiFi.
-     *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
-     * @param availability Whether the 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.
-     * @return A reportingTasksEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("{availability}")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-            value = "Gets all reporting tasks",
-            response = ReportingTasksEntity.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 = 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 getReportingTasks(
-            @ApiParam(
-                    value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
-                    required = false
-            )
-            @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @ApiParam(
-                    value = "Whether the reporting task 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) {
-
-        final Availability avail = parseAvailability(availability);
-
-        // replicate if cluster manager
-        if (properties.isClusterManager() && Availability.NODE.equals(avail)) {
-            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
-        }
-
-        // get all the reporting tasks
-        final Set<ReportingTaskDTO> reportingTasks = populateRemainingReportingTasksContent(availability, serviceFacade.getReportingTasks());
-
-        // create the revision
-        final RevisionDTO revision = new RevisionDTO();
-        revision.setClientId(clientId.getClientId());
-
-        // create the response entity
-        final ReportingTasksEntity entity = new ReportingTasksEntity();
-        entity.setRevision(revision);
-        entity.setReportingTasks(reportingTasks);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
-     * Creates a new Reporting Task.
-     *
-     * @param httpServletRequest request
-     * @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 reportingTaskEntity A reportingTaskEntity.
-     * @return A reportingTaskEntity.
-     */
-    @POST
-    @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,
-            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 = 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 createReportingTask(
-            @Context HttpServletRequest httpServletRequest,
-            @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",
-                    required = true
-            )
-            @PathParam("availability") String availability,
-            @ApiParam(
-                    value = "The reporting task configuration details.",
-                    required = true
-            ) ReportingTaskEntity reportingTaskEntity) {
-
-        final Availability avail = parseAvailability(availability);
-
-        if (reportingTaskEntity == null || reportingTaskEntity.getReportingTask() == null) {
-            throw new IllegalArgumentException("Reporting task details must be specified.");
-        }
-
-        if (reportingTaskEntity.getRevision() == null) {
-            throw new IllegalArgumentException("Revision must be specified.");
-        }
-
-        if (reportingTaskEntity.getReportingTask().getId() != null) {
-            throw new IllegalArgumentException("Reporting task ID cannot be specified.");
-        }
-
-        if (StringUtils.isBlank(reportingTaskEntity.getReportingTask().getType())) {
-            throw new IllegalArgumentException("The type of reporting task to create must be specified.");
-        }
-
-        // get the revision
-        final RevisionDTO revision = reportingTaskEntity.getRevision();
-
-        if (properties.isClusterManager()) {
-            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders()).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();
-        }
-
-        // 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());
-        final ReportingTaskDTO reportingTask = controllerResponse.getConfiguration();
-
-        // get the updated revision
-        final RevisionDTO updatedRevision = new RevisionDTO();
-        updatedRevision.setClientId(revision.getClientId());
-        updatedRevision.setVersion(controllerResponse.getVersion());
-
-        // build the response entity
-        final ReportingTaskEntity entity = new ReportingTaskEntity();
-        entity.setRevision(updatedRevision);
-        entity.setReportingTask(populateRemainingReportingTaskContent(availability, reportingTask));
-
-        // build the response
-        return clusterContext(generateCreatedResponse(URI.create(reportingTask.getUri()), entity)).build();
-    }
-
-    /**
      * Retrieves the specified reporting task.
      *
      * @param clientId Optional client id. If the client id is not specified, a

http://git-wip-us.apache.org/repos/asf/nifi/blob/add29816/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ResourceResource.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/ResourceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ResourceResource.java
new file mode 100644
index 0000000..ccdfda7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ResourceResource.java
@@ -0,0 +1,125 @@
+/*
+ * 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 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.ResourceDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.ResourcesEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.util.List;
+
+/**
+ * RESTful endpoint for retrieving system diagnostics.
+ */
+@Path("/resources")
+@Api(
+    value = "/resources",
+    description = "Provides the resources in this NiFi that can have access/authorization policies."
+)
+public class ResourceResource extends ApplicationResource {
+
+    private NiFiServiceFacade serviceFacade;
+    private WebClusterManager clusterManager;
+    private NiFiProperties properties;
+
+    /**
+     * Gets the available resources that support access/authorization policies.
+     *
+     * @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 resourcesEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @ApiOperation(
+            value = "Gets the available resources that support access/authorization policies",
+            response = ResourcesEntity.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 = 401, message = "Client could not be authenticated."),
+                @ApiResponse(code = 403, message = "Client is not authorized to make this request."),}
+    )
+    public Response getResources(
+            @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 the cluster manager
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
+        }
+
+        // TODO - if unsecure, return no resources?
+        final List<ResourceDTO> resources = serviceFacade.getResources();
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // create the response
+        final ResourcesEntity entity = new ResourcesEntity();
+        entity.setRevision(revision);
+        entity.setResources(resources);
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
+
+    // setters
+    public void setServiceFacade(NiFiServiceFacade serviceFacade) {
+        this.serviceFacade = serviceFacade;
+    }
+
+    public void setClusterManager(WebClusterManager clusterManager) {
+        this.clusterManager = clusterManager;
+    }
+
+    public void setProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/add29816/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.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/SiteToSiteResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
new file mode 100644
index 0000000..055d11c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
@@ -0,0 +1,126 @@
+/*
+ * 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.sun.jersey.api.core.ResourceContext;
+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.manager.impl.WebClusterManager;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.NiFiServiceFacade;
+import org.apache.nifi.web.api.dto.ControllerDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.ControllerEntity;
+import org.apache.nifi.web.api.request.ClientIdParameter;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.Path;
+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;
+
+/**
+ * RESTful endpoint for managing a Flow Controller.
+ */
+@Path("/site-to-site")
+@Api(
+        value = "/site-to-site",
+        description = "Provide access to site to site with this NiFi"
+)
+public class SiteToSiteResource extends ApplicationResource {
+
+    private NiFiServiceFacade serviceFacade;
+    private WebClusterManager clusterManager;
+    private NiFiProperties properties;
+
+    @Context
+    private ResourceContext resourceContext;
+
+    /**
+     * Returns the details of 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 controllerEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    // TODO - @PreAuthorize("hasRole('ROLE_NIFI')")
+    @ApiOperation(
+            value = "Returns the details about this NiFi necessary to communicate via site to site",
+            response = ControllerEntity.class,
+            authorizations = @Authorization(value = "NiFi", type = "ROLE_NIFI")
+    )
+    @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 = 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 getController(
+            @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) {
+
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
+        }
+
+        // get the controller dto
+        final ControllerDTO controller = serviceFacade.getController();
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // build the response entity
+        final ControllerEntity entity = new ControllerEntity();
+        entity.setRevision(revision);
+        entity.setController(controller);
+
+        // generate the response
+        return clusterContext(noCache(Response.ok(entity))).build();
+    }
+
+    // setters
+    public void setServiceFacade(NiFiServiceFacade serviceFacade) {
+        this.serviceFacade = serviceFacade;
+    }
+
+    public void setClusterManager(WebClusterManager clusterManager) {
+        this.clusterManager = clusterManager;
+    }
+
+    public void setProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+}