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/07/01 20:41:51 UTC

[3/5] nifi git commit: NIFI-1781: - Updating UI according to permissions through out the application. - Shuffling provenance events, template, and cluster search REST APIs according to resources being authorized. - Moving template upload controls. - Remo

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index 5c9bd9a..cc7d5aa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -16,29 +16,13 @@
  */
 package org.apache.nifi.web.api;
 
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-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.WebApplicationException;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-
+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.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.AuthorizationRequest;
@@ -46,20 +30,25 @@ import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
+import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.IllegalClusterResourceRequestException;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.AboutDTO;
 import org.apache.nifi.web.api.dto.BannerDTO;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
 import org.apache.nifi.web.api.dto.BulletinQueryDTO;
+import org.apache.nifi.web.api.dto.ClusterDTO;
+import org.apache.nifi.web.api.dto.NodeDTO;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.dto.action.ActionDTO;
@@ -67,6 +56,7 @@ import org.apache.nifi.web.api.dto.action.HistoryDTO;
 import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
 import org.apache.nifi.web.api.dto.flow.FlowDTO;
 import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
+import org.apache.nifi.web.api.dto.search.NodeSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
 import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
@@ -79,19 +69,19 @@ 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.AboutEntity;
 import org.apache.nifi.web.api.entity.ActionEntity;
-import org.apache.nifi.web.api.entity.AuthorityEntity;
 import org.apache.nifi.web.api.entity.BannerEntity;
 import org.apache.nifi.web.api.entity.BulletinBoardEntity;
+import org.apache.nifi.web.api.entity.ClusterSearchResultsEntity;
 import org.apache.nifi.web.api.entity.ComponentHistoryEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity;
 import org.apache.nifi.web.api.entity.ControllerServicesEntity;
 import org.apache.nifi.web.api.entity.ControllerStatusEntity;
+import org.apache.nifi.web.api.entity.CurrentUserEntity;
 import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.FlowConfigurationEntity;
 import org.apache.nifi.web.api.entity.HistoryEntity;
-import org.apache.nifi.web.api.entity.IdentityEntity;
 import org.apache.nifi.web.api.entity.PortStatusEntity;
 import org.apache.nifi.web.api.entity.PrioritizerTypesEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupEntity;
@@ -106,19 +96,37 @@ import org.apache.nifi.web.api.entity.ReportingTasksEntity;
 import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
 import org.apache.nifi.web.api.entity.SearchResultsEntity;
 import org.apache.nifi.web.api.entity.StatusHistoryEntity;
+import org.apache.nifi.web.api.entity.TemplateEntity;
+import org.apache.nifi.web.api.entity.TemplatesEntity;
 import org.apache.nifi.web.api.request.BulletinBoardPatternParameter;
 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 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 javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+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.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  * RESTful endpoint for managing a Flow.
@@ -215,6 +223,21 @@ public class FlowResource extends ApplicationResource {
         }
     }
 
+    private boolean isAuthorized(final RequestAction action, final Resource resource) {
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        final AuthorizationRequest request = new AuthorizationRequest.Builder()
+                .resource(resource)
+                .identity(user.getIdentity())
+                .anonymous(user.isAnonymous())
+                .accessAttempt(false)
+                .action(action)
+                .build();
+
+        final AuthorizationResult result = authorizer.authorize(request);
+        return Result.Approved.equals(result.getResult());
+    }
+
     // ----
     // flow
     // ----
@@ -299,12 +322,12 @@ public class FlowResource extends ApplicationResource {
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
-    @Path("identity")
+    @Path("current-user")
     @ApiOperation(
             value = "Retrieves the user identity of the user making the request",
-            response = IdentityEntity.class
+            response = CurrentUserEntity.class
     )
-    public Response getIdentity() {
+    public Response getCurrentUser() {
 
         authorizeFlow(RequestAction.READ);
 
@@ -315,59 +338,7 @@ public class FlowResource extends ApplicationResource {
         }
 
         // create the response entity
-        IdentityEntity entity = new IdentityEntity();
-        entity.setUserId(user.getIdentity());
-        entity.setIdentity(user.getUserName());
-        entity.setAnonymous(user.isAnonymous());
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**x
-     * Retrieves the user details, including the authorities, about the user making the request.
-     *
-     * @return A authoritiesEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("authorities")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN', 'ROLE_PROXY', 'ROLE_NIFI', 'ROLE_PROVENANCE')")
-    @ApiOperation(
-            value = "Retrieves the user details, including the authorities, about the user making the request",
-            response = AuthorityEntity.class,
-            authorizations = {
-                    @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
-                    @Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
-                    @Authorization(value = "Administrator", type = "ROLE_ADMIN"),
-                    @Authorization(value = "Proxy", type = "ROLE_PROXY"),
-                    @Authorization(value = "NiFi", type = "ROLE_NIFI"),
-                    @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
-            }
-    )
-    @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 getAuthorities() {
-        // TODO - remove this method once authorities are completely removed
-        authorizeFlow(RequestAction.READ);
-
-        // note that the cluster manager will handle this request directly
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        if (user == null) {
-            throw new WebApplicationException(new Throwable("Unable to access details for current user."));
-        }
-
-        // create the response entity
-        AuthorityEntity entity = new AuthorityEntity();
-        entity.setUserId(user.getIdentity());
-        entity.setAuthorities(new HashSet<>(Arrays.asList("ROLE_MONITOR", "ROLE_DFM", "ROLE_ADMIN", "ROLE_PROXY", "ROLE_NIFI", "ROLE_PROVENANCE")));
+        final CurrentUserEntity entity = serviceFacade.getCurrentUser();
 
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
@@ -1081,7 +1052,7 @@ public class FlowResource extends ApplicationResource {
     // --------------
 
     /**
-     * Retrieves all the of templates in this NiFi.
+     * Retrieves all the of bulletins in this NiFi.
      *
      * @param after Supporting querying for bulletins after a particular
      *            bulletin id.
@@ -1960,7 +1931,7 @@ public class FlowResource extends ApplicationResource {
      * formatted as 'MM/dd/yyyy HH:mm:ss'. This parameter is optional and must
      * be specified in the timezone of the server. The server's timezone can be
      * determined by inspecting the result of a status or history request.
-     * @param userName The user name of the user who's actions are being
+     * @param userIdentity The user name of the user who's actions are being
      * queried. This parameter is optional.
      * @param sourceId The id of the source being queried (usually a processor
      * id). This parameter is optional.
@@ -2023,7 +1994,7 @@ public class FlowResource extends ApplicationResource {
                     value = "Include actions performed by this user.",
                     required = false
             )
-            @QueryParam("userName") String userName,
+            @QueryParam("userIdentity") String userIdentity,
             @ApiParam(
                     value = "Include actions on this component.",
                     required = false
@@ -2082,8 +2053,8 @@ public class FlowResource extends ApplicationResource {
         }
 
         // optionally set the user id
-        if (userName != null) {
-            query.setUserName(userName);
+        if (userIdentity != null) {
+            query.setUserIdentity(userIdentity);
         }
 
         // optionally set the processor id
@@ -2210,6 +2181,138 @@ public class FlowResource extends ApplicationResource {
         return generateOkResponse(entity).build();
     }
 
+    // ---------
+    // templates
+    // ---------
+
+    /**
+     * Retrieves all the of templates in this NiFi.
+     *
+     * @return A templatesEntity.
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("templates")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @ApiOperation(
+            value = "Gets all templates",
+            response = TemplatesEntity.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 getTemplates() {
+
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.GET);
+        }
+
+        // authorize access
+        authorizeFlow(RequestAction.READ);
+
+        // get all the templates
+        final Set<TemplateEntity> templates = serviceFacade.getTemplates();
+        templateResource.populateRemainingTemplateEntitiesContent(templates);
+
+        // create the response entity
+        final TemplatesEntity entity = new TemplatesEntity();
+        entity.setTemplates(templates);
+        entity.setGenerated(new Date());
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
+
+    // --------------------
+    // search cluster nodes
+    // --------------------
+
+    /**
+     * Searches the cluster for a node with a given address.
+     *
+     * @param value Search value that will be matched against a node's address
+     * @return Nodes that match the specified criteria
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("cluster/search-results")
+    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @ApiOperation(
+            value = "Searches the cluster for a node with the specified address",
+            response = ClusterSearchResultsEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
+                    @Authorization(value = "DFM", type = "ROLE_DFM"),
+                    @Authorization(value = "Admin", type = "ROLE_ADMIN")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response searchCluster(
+            @ApiParam(
+                    value = "Node address to search for.",
+                    required = true
+            )
+            @QueryParam("q") @DefaultValue(StringUtils.EMPTY) String value) {
+
+        authorizeFlow(RequestAction.READ);
+
+        // ensure connected to the cluster
+        if (!isConnectedToCluster()) {
+            throw new IllegalClusterResourceRequestException("Only a node connected to a cluster can process the request.");
+        }
+
+        final List<NodeSearchResultDTO> nodeMatches = new ArrayList<>();
+
+        // get the nodes in the cluster
+        final ClusterDTO cluster = serviceFacade.getCluster();
+
+        // check each to see if it matches the search term
+        for (NodeDTO node : cluster.getNodes()) {
+            // ensure the node is connected
+            if (!NodeConnectionState.CONNECTED.name().equals(node.getStatus())) {
+                continue;
+            }
+
+            // determine the current nodes address
+            final String address = node.getAddress() + ":" + node.getApiPort();
+
+            // count the node if there is no search or it matches the address
+            if (StringUtils.isBlank(value) || StringUtils.containsIgnoreCase(address, value)) {
+                final NodeSearchResultDTO nodeMatch = new NodeSearchResultDTO();
+                nodeMatch.setId(node.getNodeId());
+                nodeMatch.setAddress(address);
+                nodeMatches.add(nodeMatch);
+            }
+        }
+
+        // build the response
+        ClusterSearchResultsEntity results = new ClusterSearchResultsEntity();
+        results.setNodeResults(nodeMatches);
+
+        // generate an 200 - OK response
+        return noCache(Response.ok(results)).build();
+    }
+
     // setters
     public void setServiceFacade(NiFiServiceFacade serviceFacade) {
         this.serviceFacade = serviceFacade;

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index 48c241c..0a94fea 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -16,36 +16,14 @@
  */
 package org.apache.nifi.web.api;
 
-import java.io.InputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.Date;
-import java.util.HashMap;
-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.GET;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-
+import com.sun.jersey.api.core.ResourceContext;
+import com.sun.jersey.multipart.FormDataParam;
+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.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
@@ -82,20 +60,38 @@ import org.apache.nifi.web.api.entity.ProcessorsEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
 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.nifi.web.api.request.LongParameter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.sun.jersey.api.core.ResourceContext;
-import com.sun.jersey.multipart.FormDataParam;
-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.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.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * RESTful endpoint for managing a Group.
@@ -1848,62 +1844,6 @@ public class ProcessGroupResource extends ApplicationResource {
     }
 
     /**
-     * Retrieves all the of templates in this NiFi.
-     *
-     * @return A templatesEntity.
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("{id}/templates")
-    // TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
-    @ApiOperation(
-        value = "Gets all templates",
-        response = TemplatesEntity.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 getTemplates(
-        @ApiParam(
-            value = "The process group id.",
-            required = true
-        )
-        @PathParam("id") final String groupId) {
-
-        if (isReplicateRequest()) {
-            return replicate(HttpMethod.GET);
-        }
-
-        // authorize access
-        serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable processGroup = lookup.getProcessGroup(groupId);
-            processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
-        });
-
-        // get all the templates
-        final Set<TemplateDTO> templates = templateResource.populateRemainingTemplatesContent(serviceFacade.getTemplates());
-
-        // create the response entity
-        final TemplatesEntity entity = new TemplatesEntity();
-        entity.setTemplates(templates);
-        entity.setGenerated(new Date());
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
      * Creates a new template based off of the specified template.
      *
      * @param httpServletRequest request

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceEventResource.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/ProvenanceEventResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceEventResource.java
new file mode 100644
index 0000000..8c3a5d6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceEventResource.java
@@ -0,0 +1,391 @@
+/*
+ * 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.nifi.cluster.coordination.http.replication.RequestReplicator;
+import org.apache.nifi.controller.repository.claim.ContentDirection;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.web.DownloadableContent;
+import org.apache.nifi.web.NiFiServiceFacade;
+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.lineage.LineageDTO;
+import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
+import org.apache.nifi.web.api.entity.SubmitReplayRequestEntity;
+import org.apache.nifi.web.api.request.LongParameter;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.StreamingOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+
+
+/**
+ * RESTful endpoint for querying data provenance.
+ */
+@Path("/provenance-events")
+@Api(
+    value = "/provenance-events",
+    description = "Endpoint for accessing data flow provenance."
+)
+public class ProvenanceEventResource extends ApplicationResource {
+
+    private NiFiServiceFacade serviceFacade;
+
+    /**
+     * Populates the uri for the specified provenance.
+     */
+    private ProvenanceDTO populateRemainingProvenanceContent(ProvenanceDTO provenance) {
+        provenance.setUri(generateResourceUri("provenance", provenance.getId()));
+        return provenance;
+    }
+
+    /**
+     * Populates the uri for the specified lineage.
+     */
+    private LineageDTO populateRemainingLineageContent(LineageDTO lineage) {
+        lineage.setUri(generateResourceUri("provenance", "lineage", lineage.getId()));
+        return lineage;
+    }
+
+    /**
+     * Gets the content for the input of the specified event.
+     *
+     * @param clusterNodeId The id of the node within the cluster this content is on. Required if clustered.
+     * @param id The id of the provenance event associated with this content.
+     * @return The content stream
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.WILDCARD)
+    @Path("{id}/content/input")
+    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
+    @ApiOperation(
+            value = "Gets the input content for a provenance event",
+            authorizations = {
+                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
+            }
+    )
+    @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 getInputContent(
+            @ApiParam(
+                    value = "The id of the node where the content exists if clustered.",
+                    required = false
+            )
+            @QueryParam("clusterNodeId") final String clusterNodeId,
+            @ApiParam(
+                    value = "The provenance event id.",
+                    required = true
+            )
+            @PathParam("id") final LongParameter id) {
+
+        // ensure proper input
+        if (id == null) {
+            throw new IllegalArgumentException("The event id must be specified.");
+        }
+
+        // replicate if cluster manager
+        if (isReplicateRequest()) {
+            // determine where this request should be sent
+            if (clusterNodeId == null) {
+                throw new IllegalArgumentException("The id of the node in the cluster is required.");
+            } else {
+                return replicate(HttpMethod.GET, clusterNodeId);
+            }
+        }
+
+        // get the uri of the request
+        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);
+
+        // generate a streaming response
+        final StreamingOutput response = new StreamingOutput() {
+            @Override
+            public void write(OutputStream output) throws IOException, WebApplicationException {
+                try (InputStream is = content.getContent()) {
+                    // stream the content to the response
+                    StreamUtils.copy(is, output);
+
+                    // flush the response
+                    output.flush();
+                }
+            }
+        };
+
+        // use the appropriate content type
+        String contentType = content.getType();
+        if (contentType == null) {
+            contentType = MediaType.APPLICATION_OCTET_STREAM;
+        }
+
+        return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
+    }
+
+    /**
+     * Gets the content for the output of the specified event.
+     *
+     * @param clusterNodeId The id of the node within the cluster this content is on. Required if clustered.
+     * @param id The id of the provenance event associated with this content.
+     * @return The content stream
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.WILDCARD)
+    @Path("{id}/content/output")
+    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
+    @ApiOperation(
+            value = "Gets the output content for a provenance event",
+            authorizations = {
+                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
+            }
+    )
+    @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 getOutputContent(
+            @ApiParam(
+                    value = "The id of the node where the content exists if clustered.",
+                    required = false
+            )
+            @QueryParam("clusterNodeId") final String clusterNodeId,
+            @ApiParam(
+                    value = "The provenance event id.",
+                    required = true
+            )
+            @PathParam("id") final LongParameter id) {
+
+        // ensure proper input
+        if (id == null) {
+            throw new IllegalArgumentException("The event id must be specified.");
+        }
+
+        // replicate if cluster manager
+        if (isReplicateRequest()) {
+            // determine where this request should be sent
+            if (clusterNodeId == null) {
+                throw new IllegalArgumentException("The id of the node in the cluster is required.");
+            } else {
+                return replicate(HttpMethod.GET, clusterNodeId);
+            }
+        }
+
+        // get the uri of the request
+        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);
+
+        // generate a streaming response
+        final StreamingOutput response = new StreamingOutput() {
+            @Override
+            public void write(OutputStream output) throws IOException, WebApplicationException {
+                try (InputStream is = content.getContent()) {
+                    // stream the content to the response
+                    StreamUtils.copy(is, output);
+
+                    // flush the response
+                    output.flush();
+                }
+            }
+        };
+
+        // use the appropriate content type
+        String contentType = content.getType();
+        if (contentType == null) {
+            contentType = MediaType.APPLICATION_OCTET_STREAM;
+        }
+
+        return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
+    }
+
+    /**
+     * Gets the details for a provenance event.
+     *
+     * @param id The id of the event
+     * @param clusterNodeId The id of node in the cluster that the event/flowfile originated from. This is only required when clustered.
+     * @return A provenanceEventEntity
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}")
+    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
+    @ApiOperation(
+            value = "Gets a provenance event",
+            response = ProvenanceEventEntity.class,
+            authorizations = {
+                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
+            }
+    )
+    @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 getProvenanceEvent(
+            @ApiParam(
+                    value = "The id of the node where this event exists if clustered.",
+                    required = false
+            )
+            @QueryParam("clusterNodeId") final String clusterNodeId,
+            @ApiParam(
+                    value = "The provenence event id.",
+                    required = true
+            )
+            @PathParam("id") final LongParameter id) {
+
+        // ensure the id is specified
+        if (id == null) {
+            throw new IllegalArgumentException("Provenance event id must be specified.");
+        }
+
+        // replicate if cluster manager
+        if (isReplicateRequest()) {
+            // since we're cluster we must specify the cluster node identifier
+            if (clusterNodeId == null) {
+                throw new IllegalArgumentException("The cluster node identifier must be specified.");
+            }
+
+            return replicate(HttpMethod.GET, clusterNodeId);
+        }
+
+        // get the provenance event
+        final ProvenanceEventDTO event = serviceFacade.getProvenanceEvent(id.getLong());
+
+        // create a response entity
+        final ProvenanceEventEntity entity = new ProvenanceEventEntity();
+        entity.setProvenanceEvent(event);
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
+
+    /**
+     * Creates a new replay request for the content associated with the specified provenance event id.
+     *
+     * @param httpServletRequest request
+     * @param replayRequestEntity The replay request
+     * @return A provenanceEventEntity
+     */
+    @POST
+    @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",
+            response = ProvenanceEventEntity.class,
+            authorizations = {
+                    @Authorization(value = "Provenance and Data Flow Manager", type = "ROLE_PROVENANCE and ROLE_DFM")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response submitReplay(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The replay request.",
+                    required = true
+            ) final SubmitReplayRequestEntity replayRequestEntity) {
+
+        // ensure the event id is specified
+        if (replayRequestEntity == null || replayRequestEntity.getEventId() == null) {
+            throw new IllegalArgumentException("The id of the event must be specified.");
+        }
+
+        // replicate if cluster manager
+        if (isReplicateRequest()) {
+            // determine where this request should be sent
+            if (replayRequestEntity.getClusterNodeId() == null) {
+                throw new IllegalArgumentException("The id of the node in the cluster is required.");
+            } else {
+                return replicate(HttpMethod.POST, replayRequestEntity, replayRequestEntity.getClusterNodeId());
+            }
+        }
+
+        // handle expects request (usually from the cluster manager)
+        final String expects = httpServletRequest.getHeader(RequestReplicator.REQUEST_VALIDATION_HTTP_HEADER);
+        if (expects != null) {
+            return generateContinueResponse().build();
+        }
+
+        // submit the provenance replay request
+        final ProvenanceEventDTO event = serviceFacade.submitReplay(replayRequestEntity.getEventId());
+
+        // create a response entity
+        final ProvenanceEventEntity entity = new ProvenanceEventEntity();
+        entity.setProvenanceEvent(event);
+
+        // generate the response
+        URI uri = URI.create(generateResourceUri("provenance-events", event.getId()));
+        return clusterContext(generateCreatedResponse(uri, entity)).build();
+    }
+
+    // setters
+
+    public void setServiceFacade(NiFiServiceFacade serviceFacade) {
+        this.serviceFacade = serviceFacade;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/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 f4501f5..70339ea 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
@@ -16,29 +16,12 @@
  */
 package org.apache.nifi.web.api;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.StreamingOutput;
-
+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.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.AuthorizationRequest;
@@ -50,28 +33,31 @@ import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
-import org.apache.nifi.controller.repository.claim.ContentDirection;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.web.DownloadableContent;
 import org.apache.nifi.web.NiFiServiceFacade;
 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.lineage.LineageDTO;
 import org.apache.nifi.web.api.dto.provenance.lineage.LineageRequestDTO;
 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.LongParameter;
 
-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.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+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.util.HashMap;
+import java.util.Map;
 
 
 /**
@@ -166,246 +152,6 @@ public class ProvenanceResource extends ApplicationResource {
     }
 
     /**
-     * Creates a new replay request for the content associated with the specified provenance event id.
-     *
-     * @param httpServletRequest request
-     * @param replayRequestEntity The replay request
-     * @return A provenanceEventEntity
-     */
-    @POST
-    @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",
-            response = ProvenanceEventEntity.class,
-            authorizations = {
-                @Authorization(value = "Provenance and Data Flow Manager", type = "ROLE_PROVENANCE and ROLE_DFM")
-            }
-    )
-    @ApiResponses(
-            value = {
-                @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
-                @ApiResponse(code = 401, message = "Client could not be authenticated."),
-                @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
-                @ApiResponse(code = 404, message = "The specified resource could not be found."),
-                @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
-            }
-    )
-    public Response submitReplay(
-        @Context final HttpServletRequest httpServletRequest,
-        @ApiParam(
-            value = "The replay request.",
-            required = true
-        ) final SubmitReplayRequestEntity replayRequestEntity) {
-
-        authorizeProvenanceRequest();
-
-        // ensure the event id is specified
-        if (replayRequestEntity == null || replayRequestEntity.getEventId() == null) {
-            throw new IllegalArgumentException("The id of the event must be specified.");
-        }
-
-        // replicate if cluster manager
-        if (isReplicateRequest()) {
-            // determine where this request should be sent
-            if (replayRequestEntity.getClusterNodeId() == null) {
-                throw new IllegalArgumentException("The id of the node in the cluster is required.");
-            } else {
-                return replicate(HttpMethod.POST, replayRequestEntity, replayRequestEntity.getClusterNodeId());
-            }
-        }
-
-        // handle expects request (usually from the cluster manager)
-        final String expects = httpServletRequest.getHeader(RequestReplicator.REQUEST_VALIDATION_HTTP_HEADER);
-        if (expects != null) {
-            return generateContinueResponse().build();
-        }
-
-        // submit the provenance replay request
-        final ProvenanceEventDTO event = serviceFacade.submitReplay(replayRequestEntity.getEventId());
-
-        // create a response entity
-        final ProvenanceEventEntity entity = new ProvenanceEventEntity();
-        entity.setProvenanceEvent(event);
-
-        // generate the response
-        URI uri = URI.create(generateResourceUri("provenance", "events", event.getId()));
-        return clusterContext(generateCreatedResponse(uri, entity)).build();
-    }
-
-    /**
-     * Gets the content for the input of the specified event.
-     *
-     * @param clusterNodeId The id of the node within the cluster this content is on. Required if clustered.
-     * @param id The id of the provenance event associated with this content.
-     * @return The content stream
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.WILDCARD)
-    @Path("events/{id}/content/input")
-    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
-    @ApiOperation(
-            value = "Gets the input content for a provenance event",
-            authorizations = {
-                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
-            }
-    )
-    @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 getInputContent(
-            @ApiParam(
-                    value = "The id of the node where the content exists if clustered.",
-                    required = false
-            )
-            @QueryParam("clusterNodeId") final String clusterNodeId,
-            @ApiParam(
-                    value = "The provenance event id.",
-                    required = true
-            )
-            @PathParam("id") final LongParameter id) {
-
-        authorizeProvenanceRequest();
-
-        // ensure proper input
-        if (id == null) {
-            throw new IllegalArgumentException("The event id must be specified.");
-        }
-
-        // replicate if cluster manager
-        if (isReplicateRequest()) {
-            // determine where this request should be sent
-            if (clusterNodeId == null) {
-                throw new IllegalArgumentException("The id of the node in the cluster is required.");
-            } else {
-                return replicate(HttpMethod.GET, clusterNodeId);
-            }
-        }
-
-        // get the uri of the request
-        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);
-
-        // generate a streaming response
-        final StreamingOutput response = new StreamingOutput() {
-            @Override
-            public void write(OutputStream output) throws IOException, WebApplicationException {
-                try (InputStream is = content.getContent()) {
-                    // stream the content to the response
-                    StreamUtils.copy(is, output);
-
-                    // flush the response
-                    output.flush();
-                }
-            }
-        };
-
-        // use the appropriate content type
-        String contentType = content.getType();
-        if (contentType == null) {
-            contentType = MediaType.APPLICATION_OCTET_STREAM;
-        }
-
-        return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
-    }
-
-    /**
-     * Gets the content for the output of the specified event.
-     *
-     * @param clusterNodeId The id of the node within the cluster this content is on. Required if clustered.
-     * @param id The id of the provenance event associated with this content.
-     * @return The content stream
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.WILDCARD)
-    @Path("events/{id}/content/output")
-    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
-    @ApiOperation(
-            value = "Gets the output content for a provenance event",
-            authorizations = {
-                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
-            }
-    )
-    @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 getOutputContent(
-            @ApiParam(
-                    value = "The id of the node where the content exists if clustered.",
-                    required = false
-            )
-            @QueryParam("clusterNodeId") final String clusterNodeId,
-            @ApiParam(
-                    value = "The provenance event id.",
-                    required = true
-            )
-            @PathParam("id") final LongParameter id) {
-
-        authorizeProvenanceRequest();
-
-        // ensure proper input
-        if (id == null) {
-            throw new IllegalArgumentException("The event id must be specified.");
-        }
-
-        // replicate if cluster manager
-        if (isReplicateRequest()) {
-            // determine where this request should be sent
-            if (clusterNodeId == null) {
-                throw new IllegalArgumentException("The id of the node in the cluster is required.");
-            } else {
-                return replicate(HttpMethod.GET, clusterNodeId);
-            }
-        }
-
-        // get the uri of the request
-        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);
-
-        // generate a streaming response
-        final StreamingOutput response = new StreamingOutput() {
-            @Override
-            public void write(OutputStream output) throws IOException, WebApplicationException {
-                try (InputStream is = content.getContent()) {
-                    // stream the content to the response
-                    StreamUtils.copy(is, output);
-
-                    // flush the response
-                    output.flush();
-                }
-            }
-        };
-
-        // use the appropriate content type
-        String contentType = content.getType();
-        if (contentType == null) {
-            contentType = MediaType.APPLICATION_OCTET_STREAM;
-        }
-
-        return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
-    }
-
-    /**
      * Creates provenance using the specified query criteria.
      *
      * @param httpServletRequest request
@@ -635,74 +381,6 @@ public class ProvenanceResource extends ApplicationResource {
     }
 
     /**
-     * Gets the details for a provenance event.
-     *
-     * @param id The id of the event
-     * @param clusterNodeId The id of node in the cluster that the event/flowfile originated from. This is only required when clustered.
-     * @return A provenanceEventEntity
-     */
-    @GET
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @Path("events/{id}")
-    // TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
-    @ApiOperation(
-            value = "Gets a provenance event",
-            response = ProvenanceEventEntity.class,
-            authorizations = {
-                @Authorization(value = "Provenance", type = "ROLE_PROVENANCE")
-            }
-    )
-    @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 getProvenanceEvent(
-            @ApiParam(
-                    value = "The id of the node where this event exists if clustered.",
-                    required = false
-            )
-            @QueryParam("clusterNodeId") final String clusterNodeId,
-            @ApiParam(
-                    value = "The provenence event id.",
-                    required = true
-            )
-            @PathParam("id") final LongParameter id) {
-
-        authorizeProvenanceRequest();
-
-        // ensure the id is specified
-        if (id == null) {
-            throw new IllegalArgumentException("Provenance event id must be specified.");
-        }
-
-        // replicate if cluster manager
-        if (isReplicateRequest()) {
-            // since we're cluster we must specify the cluster node identifier
-            if (clusterNodeId == null) {
-                throw new IllegalArgumentException("The cluster node identifier must be specified.");
-            }
-
-            return replicate(HttpMethod.GET, clusterNodeId);
-        }
-
-        // get the provenance event
-        final ProvenanceEventDTO event = serviceFacade.getProvenanceEvent(id.getLong());
-
-        // create a response entity
-        final ProvenanceEventEntity entity = new ProvenanceEventEntity();
-        entity.setProvenanceEvent(event);
-
-        // generate the response
-        return clusterContext(generateOkResponse(entity)).build();
-    }
-
-    /**
      * 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

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/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 6994815..dcff518 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
@@ -16,7 +16,20 @@
  */
 package org.apache.nifi.web.api;
 
-import java.util.Set;
+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.authorization.Authorizer;
+import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
+import org.apache.nifi.authorization.user.NiFiUserUtils;
+import org.apache.nifi.web.NiFiServiceFacade;
+import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.entity.TemplateEntity;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
@@ -29,22 +42,7 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.RequestAction;
-import org.apache.nifi.authorization.resource.Authorizable;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.web.NiFiServiceFacade;
-import org.apache.nifi.web.api.dto.TemplateDTO;
-import org.apache.nifi.web.api.entity.TemplateEntity;
-
-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.Set;
 
 /**
  * RESTful endpoint for managing a Template.
@@ -60,6 +58,34 @@ public class TemplateResource extends ApplicationResource {
     private Authorizer authorizer;
 
     /**
+     * Populate the uri's for the specified templates.
+     *
+     * @param templateEntities templates
+     * @return templates
+     */
+    public Set<TemplateEntity> populateRemainingTemplateEntitiesContent(Set<TemplateEntity> templateEntities) {
+        for (TemplateEntity templateEntity : templateEntities) {
+            if (templateEntity.getTemplate() != null) {
+                populateRemainingTemplateContent(templateEntity.getTemplate());
+            }
+        }
+        return templateEntities;
+    }
+
+    /**
+     * Populate the uri's for the specified templates.
+     *
+     * @param templateEntity templates
+     * @return templates
+     */
+    public TemplateEntity populateRemainingTemplateEntityContent(TemplateEntity templateEntity) {
+        if (templateEntity.getTemplate() != null) {
+            populateRemainingTemplateContent(templateEntity.getTemplate());
+        }
+        return templateEntity;
+    }
+
+    /**
      * Populates the uri for the specified templates.
      *
      * @param templates templates

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.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/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 10a28c7..cf1bca4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -220,7 +220,6 @@ public final class DtoFactory {
         actionDto.setSourceType(action.getSourceType().name());
         actionDto.setTimestamp(action.getTimestamp());
         actionDto.setUserDn(action.getUserIdentity());
-        actionDto.setUserName(action.getUserName());
         actionDto.setOperation(action.getOperation().name());
         actionDto.setActionDetails(createActionDetailsDto(action.getActionDetails()));
         actionDto.setComponentDetails(createComponentDetailsDto(action.getComponentDetails()));
@@ -305,15 +304,6 @@ public final class DtoFactory {
         final HistoryDTO historyDto = new HistoryDTO();
         historyDto.setTotal(history.getTotal());
         historyDto.setLastRefreshed(history.getLastRefreshed());
-
-        if (history.getActions() != null) {
-            final List<ActionDTO> actionDtos = new ArrayList<>();
-            for (final Action action : history.getActions()) {
-                actionDtos.add(createActionDto(action));
-            }
-            historyDto.setActions(actionDtos);
-        }
-
         return historyDto;
     }
 
@@ -830,6 +820,7 @@ public final class DtoFactory {
 
         final TemplateDTO copy = new TemplateDTO();
         copy.setId(original.getId());
+        copy.setGroupId(template.getProcessGroup().getIdentifier());
         copy.setName(original.getName());
         copy.setDescription(original.getDescription());
         copy.setTimestamp(original.getTimestamp());

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index b72d72c..2c6d948 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -19,6 +19,9 @@ package org.apache.nifi.web.controller;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.AuthorizationResult;
+import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.Resource;
@@ -1138,22 +1141,8 @@ public class ControllerFacade implements Authorizable {
                 attributes = event.getAttributes();
             }
 
-            // calculate the dn chain
-            final List<String> dnChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(user);
-            dnChain.forEach(identity -> {
-                final Authorizable eventAuthorizable = flowController.createProvenanceAuthorizable(event.getComponentId());
-                final NiFiUser chainUser = new StandardNiFiUser(identity) {
-                    private static final long serialVersionUID = 7589311627013017356L;
-
-                    @Override
-                    public boolean isAnonymous() {
-                        // allow current user to drive anonymous flag as anonymous users are never chained... supports single user case
-                        return user.isAnonymous();
-                    }
-                };
-
-                eventAuthorizable.authorize(authorizer, RequestAction.READ, chainUser);
-            });
+            // authorize the event
+            authorizeEvent(event.getComponentId(), attributes);
 
             // get the filename and fall back to the identifier (should never happen)
             String filename = attributes.get(CoreAttributes.FILENAME.key());
@@ -1197,6 +1186,9 @@ public class ControllerFacade implements Authorizable {
             // replay the flow file
             final ProvenanceEventRecord event = flowController.replayFlowFile(originalEvent, user);
 
+            // authorize the replay
+            authorizeReplay(event.getComponentId(), event.getAttributes(), event.getSourceQueueIdentifier());
+
             // convert the event record
             return createProvenanceEventDto(event);
         } catch (final IOException ioe) {
@@ -1205,6 +1197,112 @@ public class ControllerFacade implements Authorizable {
     }
 
     /**
+     * Authorizes access to a provenance event generated by the specified component and containing the specified eventAttributes.
+     *
+     * @param componentId component id
+     * @param eventAttributes event attributes
+     */
+    private AuthorizationResult checkAuthorizationForEvent(final String componentId, final Map<String, String> eventAttributes) {
+        AuthorizationResult result = null;
+
+        // calculate the dn chain
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        final List<String> dnChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(user);
+        for (final String identity : dnChain) {
+            final Authorizable eventAuthorizable = flowController.createProvenanceAuthorizable(componentId);
+            final NiFiUser chainUser = new StandardNiFiUser(identity) {
+                @Override
+                public boolean isAnonymous() {
+                    // allow current user to drive anonymous flag as anonymous users are never chained... supports single user case
+                    return user.isAnonymous();
+                }
+            };
+
+            result = eventAuthorizable.checkAuthorization(authorizer, RequestAction.READ, chainUser, eventAttributes);
+            if (!Result.Approved.equals(result.getResult())) {
+                break;
+            }
+        }
+
+        if (result == null) {
+            result = AuthorizationResult.denied();
+        }
+
+        return result;
+    }
+
+    /**
+     * Authorizes access to a provenance event generated by the specified component and containing the specified eventAttributes.
+     *
+     * @param componentId component id
+     * @param eventAttributes event attributes
+     */
+    private void authorizeEvent(final String componentId, final Map<String, String> eventAttributes) {
+        // calculate the dn chain
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        final List<String> dnChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(user);
+        for (final String identity : dnChain) {
+            final Authorizable eventAuthorizable = flowController.createProvenanceAuthorizable(componentId);
+            final NiFiUser chainUser = new StandardNiFiUser(identity) {
+                @Override
+                public boolean isAnonymous() {
+                    // allow current user to drive anonymous flag as anonymous users are never chained... supports single user case
+                    return user.isAnonymous();
+                }
+            };
+
+            eventAuthorizable.authorize(authorizer, RequestAction.READ, chainUser, eventAttributes);
+        }
+    }
+
+    /**
+     * Authorizes access to replay a specified provenance event.
+     *
+     * @param componentId component id
+     * @param eventAttributes event attributes
+     * @param connectionId connection id
+     */
+    private AuthorizationResult checkAuthorizationForReplay(final String componentId, final Map<String, String> eventAttributes, final String connectionId) {
+        // if the connection id isn't specified, then the replay wouldn't be available anyways and we have nothing to authorize against so deny it`
+        if (connectionId == null) {
+            return AuthorizationResult.denied();
+        }
+
+        final AuthorizationResult result = checkAuthorizationForEvent(componentId, eventAttributes);
+        if (!Result.Approved.equals(result.getResult())) {
+            return result;
+        }
+
+        // authorize write permissions for the queue
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId());
+        final Connection connection = rootGroup.findConnection(connectionId);
+        return connection.checkAuthorization(authorizer, RequestAction.WRITE, user);
+    }
+
+    /**
+     * Authorizes access to replay a specified provenance event.
+     *
+     * @param componentId component id
+     * @param eventAttributes event attributes
+     * @param connectionId connection id
+     */
+    private void authorizeReplay(final String componentId, final Map<String, String> eventAttributes, final String connectionId) {
+        // if the connection id isn't specified, then the replay wouldn't be available anyways and we have nothing to authorize against so deny it`
+        if (connectionId == null) {
+            throw new AccessDeniedException("The connection id is unknown.");
+        }
+
+        authorizeEvent(componentId, eventAttributes);
+
+        // authorize write permissions for the queue
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+        final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId());
+        final Connection connection = rootGroup.findConnection(connectionId);
+        connection.authorize(authorizer, RequestAction.WRITE, user);
+    }
+
+    /**
      * Get the provenance event with the specified event id.
      *
      * @param eventId event id
@@ -1217,6 +1315,10 @@ public class ControllerFacade implements Authorizable {
                 throw new ResourceNotFoundException("Unable to find the specified event.");
             }
 
+            // get the flowfile attributes and authorize the event
+            final Map<String, String> attributes = event.getAttributes();
+            authorizeEvent(event.getComponentId(), attributes);
+
             // convert the event
             return createProvenanceEventDto(event);
         } catch (final IOException ioe) {
@@ -1308,9 +1410,13 @@ public class ControllerFacade implements Authorizable {
             dto.setInputContentClaimFileSize(FormatUtils.formatDataSize(event.getPreviousFileSize()));
         }
 
+        // determine if authorized for event replay
+        final AuthorizationResult replayAuthorized = checkAuthorizationForReplay(event.getComponentId(), event.getAttributes(), event.getSourceQueueIdentifier());
+
         // replay
-        dto.setReplayAvailable(contentAvailability.isReplayable());
-        dto.setReplayExplanation(contentAvailability.getReasonNotReplayable());
+        dto.setReplayAvailable(contentAvailability.isReplayable() && Result.Approved.equals(replayAuthorized.getResult()));
+        dto.setReplayExplanation(contentAvailability.isReplayable()
+                && !Result.Approved.equals(replayAuthorized.getResult()) ? replayAuthorized.getExplanation() : contentAvailability.getReasonNotReplayable());
         dto.setSourceConnectionIdentifier(event.getSourceQueueIdentifier());
 
         // sets the component details if it can find the component still in the flow
@@ -1366,56 +1472,72 @@ public class ControllerFacade implements Authorizable {
     }
 
     private void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final ComponentSearchResultDTO groupMatch = search(search, group);
-        if (groupMatch != null) {
-            results.getProcessGroupResults().add(groupMatch);
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
+            final ComponentSearchResultDTO groupMatch = search(search, group);
+            if (groupMatch != null) {
+                results.getProcessGroupResults().add(groupMatch);
+            }
         }
 
         for (final ProcessorNode procNode : group.getProcessors()) {
-            final ComponentSearchResultDTO match = search(search, procNode);
-            if (match != null) {
-                match.setGroupId(group.getIdentifier());
-                results.getProcessorResults().add(match);
+            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
+                final ComponentSearchResultDTO match = search(search, procNode);
+                if (match != null) {
+                    match.setGroupId(group.getIdentifier());
+                    results.getProcessorResults().add(match);
+                }
             }
         }
 
         for (final Connection connection : group.getConnections()) {
-            final ComponentSearchResultDTO match = search(search, connection);
-            if (match != null) {
-                match.setGroupId(group.getIdentifier());
-                results.getConnectionResults().add(match);
+            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
+                final ComponentSearchResultDTO match = search(search, connection);
+                if (match != null) {
+                    match.setGroupId(group.getIdentifier());
+                    results.getConnectionResults().add(match);
+                }
             }
         }
 
         for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            final ComponentSearchResultDTO match = search(search, remoteGroup);
-            if (match != null) {
-                match.setGroupId(group.getIdentifier());
-                results.getRemoteProcessGroupResults().add(match);
+            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
+                final ComponentSearchResultDTO match = search(search, remoteGroup);
+                if (match != null) {
+                    match.setGroupId(group.getIdentifier());
+                    results.getRemoteProcessGroupResults().add(match);
+                }
             }
         }
 
         for (final Port port : group.getInputPorts()) {
-            final ComponentSearchResultDTO match = search(search, port);
-            if (match != null) {
-                match.setGroupId(group.getIdentifier());
-                results.getInputPortResults().add(match);
+            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
+                final ComponentSearchResultDTO match = search(search, port);
+                if (match != null) {
+                    match.setGroupId(group.getIdentifier());
+                    results.getInputPortResults().add(match);
+                }
             }
         }
 
         for (final Port port : group.getOutputPorts()) {
-            final ComponentSearchResultDTO match = search(search, port);
-            if (match != null) {
-                match.setGroupId(group.getIdentifier());
-                results.getOutputPortResults().add(match);
+            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
+                final ComponentSearchResultDTO match = search(search, port);
+                if (match != null) {
+                    match.setGroupId(group.getIdentifier());
+                    results.getOutputPortResults().add(match);
+                }
             }
         }
 
         for (final Funnel funnel : group.getFunnels()) {
-            final ComponentSearchResultDTO match = search(search, funnel);
-            if (match != null) {
-                match.setGroupId(group.getIdentifier());
-                results.getFunnelResults().add(match);
+            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
+                final ComponentSearchResultDTO match = search(search, funnel);
+                if (match != null) {
+                    match.setGroupId(group.getIdentifier());
+                    results.getFunnelResults().add(match);
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
index 1a2669b..1ac13ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java
@@ -604,7 +604,7 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
                         .accessAttempt(false)
                         .action(RequestAction.WRITE)
                         .resource(connection.getResource())
-                        .eventAttributes(attributes)
+                        .resourceContext(attributes)
                         .build();
 
                 // perform the authorization

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 8faedfb..a1a63a6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -327,6 +327,12 @@
         <property name="requestReplicator" ref="requestReplicator" />
         <property name="authorizer" ref="authorizer"/>
     </bean>
+    <bean id="provenanceEventResource" class="org.apache.nifi.web.api.ProvenanceEventResource" scope="singleton">
+        <property name="serviceFacade" ref="serviceFacade"/>
+        <property name="properties" ref="nifiProperties"/>
+        <property name="clusterCoordinator" ref="clusterCoordinator"/>
+        <property name="requestReplicator" ref="requestReplicator" />
+    </bean>
     <bean id="countersResource" class="org.apache.nifi.web.api.CountersResource" scope="singleton">
         <property name="serviceFacade" ref="serviceFacade"/>
         <property name="properties" ref="nifiProperties"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/ce533033/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
index 2171ea8..ecc725b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
@@ -217,7 +217,7 @@ public class NaiveRevisionManager implements RevisionManager {
 
                 for (final Revision revision : revisionList) {
                     final RevisionLock revisionLock = getRevisionLock(revision);
-                    revisionLock.unlock(revision, revision, user.getUserName());
+                    revisionLock.unlock(revision, revision, user.getIdentity());
                     logger.debug("Relinquished lock for {}", revision);
                 }
 
@@ -300,7 +300,7 @@ public class NaiveRevisionManager implements RevisionManager {
 
                 for (final Revision revision : revisionList) {
                     final Revision updatedRevision = updatedRevisions.get(revision);
-                    getRevisionLock(revision).unlock(revision, updatedRevision, user.getUserName());
+                    getRevisionLock(revision).unlock(revision, updatedRevision, user.getIdentity());
 
                     if (updatedRevision.getVersion() != revision.getVersion()) {
                         logger.debug("Unlocked Revision {} and updated associated Version to {}", revision, updatedRevision.getVersion());