You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2016/11/30 17:51:10 UTC

nifi git commit: NIFI-3117: - Checking referenced services when performing an action with a snippet that requires it. - Updating the SnippetManager to automatically expire unused snippets. - Making the checking of referenced services consistent across co

Repository: nifi
Updated Branches:
  refs/heads/master e3c761134 -> bc223fa19


NIFI-3117: - Checking referenced services when performing an action with a snippet that requires it.
- Updating the SnippetManager to automatically expire unused snippets.
- Making the checking of referenced services consistent across component removal.
- Adding checking of referenced services for all nested components in a snippet.
- Updating the REST API docs to clarify when a referenced service is authorized.
- Conditionally authorizing transitive service references.

This closes #1277.

Signed-off-by: Bryan Bende <bb...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/bc223fa1
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/bc223fa1
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/bc223fa1

Branch: refs/heads/master
Commit: bc223fa197ae0d49f7f8cead3a4acc6c6a1cb023
Parents: e3c7611
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Nov 29 14:04:57 2016 -0500
Committer: Bryan Bende <bb...@apache.org>
Committed: Wed Nov 30 12:50:25 2016 -0500

----------------------------------------------------------------------
 .../apache/nifi/controller/SnippetManager.java  |  45 +--
 .../AuthorizeControllerServiceReference.java    |  34 ++
 .../ConfigurableComponentAuthorizable.java      |   9 +
 .../authorization/ProcessGroupAuthorizable.java |  70 ++++-
 .../StandardAuthorizableLookup.java             | 307 ++++++++++---------
 .../nifi/web/api/ApplicationResource.java       | 165 ++++++----
 .../nifi/web/api/ControllerServiceResource.java |  12 +-
 .../nifi/web/api/ProcessGroupResource.java      |  25 +-
 .../apache/nifi/web/api/ProcessorResource.java  |  12 +-
 .../nifi/web/api/ReportingTaskResource.java     |  12 +-
 .../apache/nifi/web/api/SnippetResource.java    |  51 ++-
 11 files changed, 475 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/SnippetManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/SnippetManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/SnippetManager.java
index 3a9662e..06f4975 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/SnippetManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/SnippetManager.java
@@ -16,49 +16,52 @@
  */
 package org.apache.nifi.controller;
 
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.nifi.persistence.StandardSnippetDeserializer;
+import org.apache.nifi.persistence.StandardSnippetSerializer;
+import org.apache.nifi.stream.io.ByteArrayInputStream;
+import org.apache.nifi.stream.io.ByteArrayOutputStream;
+import org.apache.nifi.stream.io.DataOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.nifi.stream.io.ByteArrayInputStream;
-import org.apache.nifi.stream.io.ByteArrayOutputStream;
-import org.apache.nifi.stream.io.DataOutputStream;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.persistence.StandardSnippetDeserializer;
-import org.apache.nifi.persistence.StandardSnippetSerializer;
+import java.util.concurrent.TimeUnit;
 
 public class SnippetManager {
 
-    private final ConcurrentMap<String, StandardSnippet> snippetMap = new ConcurrentHashMap<>();
+    private final Cache<String, StandardSnippet> snippetMap = CacheBuilder.newBuilder().expireAfterWrite(1, TimeUnit.MINUTES).build();
 
-    public void addSnippet(final StandardSnippet snippet) {
-        final StandardSnippet oldSnippet = this.snippetMap.putIfAbsent(snippet.getId(), snippet);
-        if (oldSnippet != null) {
+    public synchronized void addSnippet(final StandardSnippet snippet) {
+        if (snippetMap.getIfPresent(snippet.getId()) != null) {
             throw new IllegalStateException("Snippet with ID " + snippet.getId() + " already exists");
         }
+        snippetMap.put(snippet.getId(), snippet);
     }
 
-    public void removeSnippet(final StandardSnippet snippet) {
-        if (!snippetMap.remove(snippet.getId(), snippet)) {
+    public synchronized void removeSnippet(final StandardSnippet snippet) {
+        if (snippetMap.getIfPresent(snippet.getId()) == null) {
             throw new IllegalStateException("Snippet is not contained in this SnippetManager");
         }
+        snippetMap.invalidate(snippet.getId());
     }
 
-    public StandardSnippet getSnippet(final String identifier) {
-        return snippetMap.get(identifier);
+    public synchronized StandardSnippet getSnippet(final String identifier) {
+        return snippetMap.getIfPresent(identifier);
     }
 
-    public Collection<StandardSnippet> getSnippets() {
-        return snippetMap.values();
+    public synchronized Collection<StandardSnippet> getSnippets() {
+        return Collections.unmodifiableCollection(snippetMap.asMap().values());
     }
 
-    public void clear() {
-        snippetMap.clear();
+    public synchronized void clear() {
+        snippetMap.invalidateAll();
     }
 
     public static List<StandardSnippet> parseBytes(final byte[] bytes) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
index a3a4e19..e2ed926 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
@@ -31,6 +31,40 @@ import java.util.Objects;
 public final class AuthorizeControllerServiceReference {
 
     /**
+     * Authorizes any referenced controller services from the specified configurable component.
+     *
+     * @param authorizable authorizable that may reference a controller service
+     * @param authorizer authorizer
+     * @param lookup lookup
+     */
+    public static void authorizeControllerServiceReferences(final ConfigurableComponentAuthorizable authorizable, final Authorizer authorizer,
+                                                            final AuthorizableLookup lookup, final boolean authorizeTransitiveServices) {
+
+        // consider each property when looking for service references
+        authorizable.getPropertyDescriptors().stream().forEach(descriptor -> {
+            // if this descriptor identifies a controller service
+            if (descriptor.getControllerServiceDefinition() != null) {
+                // get the service id
+                final String serviceId = authorizable.getValue(descriptor);
+
+                // authorize the service if configured
+                if (serviceId != null) {
+                    try {
+                        final ConfigurableComponentAuthorizable currentServiceAuthorizable = lookup.getControllerService(serviceId);
+                        currentServiceAuthorizable.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+
+                        if (authorizeTransitiveServices) {
+                            authorizeControllerServiceReferences(currentServiceAuthorizable, authorizer, lookup, authorizeTransitiveServices);
+                        }
+                    } catch (ResourceNotFoundException e) {
+                        // ignore if the resource is not found, if the referenced service was previously deleted, it should not stop this action
+                    }
+                }
+            }
+        });
+    }
+
+    /**
      * Authorizes the proposed properties for the specified authorizable.
      *
      * @param proposedProperties proposed properties

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
index a06c7a0..f352211 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
@@ -19,6 +19,8 @@ package org.apache.nifi.authorization;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.components.PropertyDescriptor;
 
+import java.util.List;
+
 /**
  * Authorizable for a component that references a ControllerService.
  */
@@ -46,6 +48,13 @@ public interface ConfigurableComponentAuthorizable {
     PropertyDescriptor getPropertyDescriptor(String propertyName);
 
     /**
+     * Returns the property descriptors for this configurable component.
+     *
+     * @return property descriptors
+     */
+    List<PropertyDescriptor> getPropertyDescriptors();
+
+    /**
      * Returns the current value of the specified property.
      *
      * @param propertyDescriptor property descriptor

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
index ce7c10b..289b8fa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
@@ -32,9 +32,73 @@ public interface ProcessGroupAuthorizable {
     Authorizable getAuthorizable();
 
     /**
-     * The authorizables for all encapsulated components. Non null
+     * The authorizables for all encapsulated processors. Non null
      *
-     * @return all encapsulated authorizables
+     * @return all encapsulated processors
      */
-    Set<Authorizable> getEncapsulatedAuthorizables();
+    Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors();
+
+    /**
+     * The authorizables for all encapsulated connections. Non null
+     *
+     * @return all encapsulated connections
+     */
+    Set<Authorizable> getEncapsulatedConnections();
+
+    /**
+     * The authorizables for all encapsulated input ports. Non null
+     *
+     * @return all encapsulated input ports
+     */
+    Set<Authorizable> getEncapsulatedInputPorts();
+
+    /**
+     * The authorizables for all encapsulated output ports. Non null
+     *
+     * @return all encapsulated output ports
+     */
+    Set<Authorizable> getEncapsulatedOutputPorts();
+
+    /**
+     * The authorizables for all encapsulated funnels. Non null
+     *
+     * @return all encapsulated funnels
+     */
+    Set<Authorizable> getEncapsulatedFunnels();
+
+    /**
+     * The authorizables for all encapsulated labels. Non null
+     *
+     * @return all encapsulated labels
+     */
+    Set<Authorizable> getEncapsulatedLabels();
+
+    /**
+     * The authorizables for all encapsulated process groups. Non null
+     *
+     * @return all encapsulated process groups
+     */
+    Set<Authorizable> getEncapsulatedProcessGroups();
+
+    /**
+     * The authorizables for all encapsulated remote process groups. Non null
+     *
+     * @return all encapsulated remote process groups
+     */
+    Set<Authorizable> getEncapsulatedRemoteProcessGroups();
+
+    /**
+     * The authorizables for all encapsulated templates. Non null
+     *
+     * @return all encapsulated templates
+     */
+    Set<Authorizable> getEncapsulatedTemplates();
+
+    /**
+     * The authorizables for all encapsulated input ports. Non null
+     *
+     * @return all encapsulated input ports
+     */
+    Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices();
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
index 7aa8321..4045f29 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
@@ -55,9 +55,9 @@ import org.apache.nifi.web.dao.ReportingTaskDAO;
 import org.apache.nifi.web.dao.SnippetDAO;
 import org.apache.nifi.web.dao.TemplateDAO;
 
-import java.util.Collections;
-import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 
 class StandardAuthorizableLookup implements AuthorizableLookup {
@@ -139,54 +139,14 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ConfigurableComponentAuthorizable getProcessor(final String id) {
         final ProcessorNode processorNode = processorDAO.getProcessor(id);
-        return new ConfigurableComponentAuthorizable() {
-            @Override
-            public Authorizable getAuthorizable() {
-                return processorNode;
-            }
-
-            @Override
-            public boolean isRestricted() {
-                return processorNode.isRestricted();
-            }
-
-            @Override
-            public String getValue(PropertyDescriptor propertyDescriptor) {
-                return processorNode.getProperty(propertyDescriptor);
-            }
-
-            @Override
-            public PropertyDescriptor getPropertyDescriptor(String propertyName) {
-                return processorNode.getPropertyDescriptor(propertyName);
-            }
-        };
+        return new ProcessorConfigurableComponentAuthorizable(processorNode);
     }
 
     @Override
     public ConfigurableComponentAuthorizable getProcessorByType(String type) {
         try {
             final ProcessorNode processorNode = controllerFacade.createTemporaryProcessor(type);
-            return new ConfigurableComponentAuthorizable() {
-                @Override
-                public Authorizable getAuthorizable() {
-                    return processorNode;
-                }
-
-                @Override
-                public boolean isRestricted() {
-                    return processorNode.isRestricted();
-                }
-
-                @Override
-                public String getValue(PropertyDescriptor propertyDescriptor) {
-                    return processorNode.getProperty(propertyDescriptor);
-                }
-
-                @Override
-                public PropertyDescriptor getPropertyDescriptor(String propertyName) {
-                    return processorNode.getPropertyDescriptor(propertyName);
-                }
-            };
+            return new ProcessorConfigurableComponentAuthorizable(processorNode);
         } catch (final Exception e) {
             throw new AccessDeniedException("Unable to create processor to verify if it references any Controller Services.");
         }
@@ -288,18 +248,6 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     public ProcessGroupAuthorizable getProcessGroup(final String id) {
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(id);
 
-        final Set<Authorizable> encapsulatedAuthorizables = new HashSet<>();
-        processGroup.findAllProcessors().forEach(processor -> encapsulatedAuthorizables.add(processor));
-        processGroup.findAllConnections().forEach(connection -> encapsulatedAuthorizables.add(connection));
-        processGroup.findAllInputPorts().forEach(inputPort -> encapsulatedAuthorizables.add(inputPort));
-        processGroup.findAllOutputPorts().forEach(outputPort -> encapsulatedAuthorizables.add(outputPort));
-        processGroup.findAllFunnels().forEach(funnel -> encapsulatedAuthorizables.add(funnel));
-        processGroup.findAllLabels().forEach(label -> encapsulatedAuthorizables.add(label));
-        processGroup.findAllProcessGroups().forEach(childGroup -> encapsulatedAuthorizables.add(childGroup));
-        processGroup.findAllRemoteProcessGroups().forEach(remoteProcessGroup -> encapsulatedAuthorizables.add(remoteProcessGroup));
-        processGroup.findAllTemplates().forEach(template -> encapsulatedAuthorizables.add(template));
-        processGroup.findAllControllerServices().forEach(controllerService -> encapsulatedAuthorizables.add(controllerService));
-
         return new ProcessGroupAuthorizable() {
             @Override
             public Authorizable getAuthorizable() {
@@ -307,8 +255,55 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
             }
 
             @Override
-            public Set<Authorizable> getEncapsulatedAuthorizables() {
-                return Collections.unmodifiableSet(encapsulatedAuthorizables);
+            public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() {
+                return processGroup.findAllProcessors().stream().map(
+                        processorNode -> new ProcessorConfigurableComponentAuthorizable(processorNode)).collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedConnections() {
+                return processGroup.findAllConnections().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedInputPorts() {
+                return processGroup.findAllInputPorts().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedOutputPorts() {
+                return processGroup.findAllOutputPorts().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedFunnels() {
+                return processGroup.findAllFunnels().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedLabels() {
+                return processGroup.findAllLabels().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedProcessGroups() {
+                return processGroup.findAllProcessGroups().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedRemoteProcessGroups() {
+                return processGroup.findAllRemoteProcessGroups().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getEncapsulatedTemplates() {
+                return processGroup.findAllTemplates().stream().collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices() {
+                return processGroup.findAllControllerServices().stream().map(
+                        controllerServiceNode -> new ControllerServiceConfigurableComponentAuthorizable(controllerServiceNode)).collect(Collectors.toSet());
             }
         };
     }
@@ -343,54 +338,14 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ConfigurableComponentAuthorizable getControllerService(final String id) {
         final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id);
-        return new ConfigurableComponentAuthorizable() {
-            @Override
-            public Authorizable getAuthorizable() {
-                return controllerService;
-            }
-
-            @Override
-            public boolean isRestricted() {
-                return controllerService.isRestricted();
-            }
-
-            @Override
-            public String getValue(PropertyDescriptor propertyDescriptor) {
-                return controllerService.getProperty(propertyDescriptor);
-            }
-
-            @Override
-            public PropertyDescriptor getPropertyDescriptor(String propertyName) {
-                return controllerService.getControllerServiceImplementation().getPropertyDescriptor(propertyName);
-            }
-        };
+        return new ControllerServiceConfigurableComponentAuthorizable(controllerService);
     }
 
     @Override
     public ConfigurableComponentAuthorizable getControllerServiceByType(String type) {
         try {
             final ControllerServiceNode controllerService = controllerFacade.createTemporaryControllerService(type);
-            return new ConfigurableComponentAuthorizable() {
-                @Override
-                public Authorizable getAuthorizable() {
-                    return controllerService;
-                }
-
-                @Override
-                public boolean isRestricted() {
-                    return controllerService.isRestricted();
-                }
-
-                @Override
-                public String getValue(PropertyDescriptor propertyDescriptor) {
-                    return controllerService.getProperty(propertyDescriptor);
-                }
-
-                @Override
-                public PropertyDescriptor getPropertyDescriptor(String propertyName) {
-                    return controllerService.getControllerServiceImplementation().getPropertyDescriptor(propertyName);
-                }
-            };
+            return new ControllerServiceConfigurableComponentAuthorizable(controllerService);
         } catch (final Exception e) {
             throw new AccessDeniedException("Unable to create controller service to verify if it references any Controller Services.");
         }
@@ -442,54 +397,14 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ConfigurableComponentAuthorizable getReportingTask(final String id) {
         final ReportingTaskNode reportingTaskNode = reportingTaskDAO.getReportingTask(id);
-        return new ConfigurableComponentAuthorizable() {
-            @Override
-            public Authorizable getAuthorizable() {
-                return reportingTaskNode;
-            }
-
-            @Override
-            public boolean isRestricted() {
-                return reportingTaskNode.isRestricted();
-            }
-
-            @Override
-            public String getValue(PropertyDescriptor propertyDescriptor) {
-                return reportingTaskNode.getProperty(propertyDescriptor);
-            }
-
-            @Override
-            public PropertyDescriptor getPropertyDescriptor(String propertyName) {
-                return reportingTaskNode.getReportingTask().getPropertyDescriptor(propertyName);
-            }
-        };
+        return new ReportingTaskConfigurableComponentAuthorizable(reportingTaskNode);
     }
 
     @Override
     public ConfigurableComponentAuthorizable getReportingTaskByType(String type) {
         try {
             final ReportingTaskNode reportingTask = controllerFacade.createTemporaryReportingTask(type);
-            return new ConfigurableComponentAuthorizable() {
-                @Override
-                public Authorizable getAuthorizable() {
-                    return reportingTask;
-                }
-
-                @Override
-                public boolean isRestricted() {
-                    return reportingTask.isRestricted();
-                }
-
-                @Override
-                public String getValue(PropertyDescriptor propertyDescriptor) {
-                    return reportingTask.getProperty(propertyDescriptor);
-                }
-
-                @Override
-                public PropertyDescriptor getPropertyDescriptor(String propertyName) {
-                    return reportingTask.getReportingTask().getPropertyDescriptor(propertyName);
-                }
-            };
+            return new ReportingTaskConfigurableComponentAuthorizable(reportingTask);
         } catch (final Exception e) {
             throw new AccessDeniedException("Unable to create reporting to verify if it references any Controller Services.");
         }
@@ -730,6 +645,114 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         return RESTRICTED_COMPONENTS_AUTHORIZABLE;
     }
 
+    /**
+     * ConfigurableComponentAuthorizable for a ProcessorNode.
+     */
+    private static class ProcessorConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable {
+        private final ProcessorNode processorNode;
+
+        public ProcessorConfigurableComponentAuthorizable(ProcessorNode processorNode) {
+            this.processorNode = processorNode;
+        }
+
+        @Override
+        public Authorizable getAuthorizable() {
+            return processorNode;
+        }
+
+        @Override
+        public boolean isRestricted() {
+            return processorNode.isRestricted();
+        }
+
+        @Override
+        public String getValue(PropertyDescriptor propertyDescriptor) {
+            return processorNode.getProperty(propertyDescriptor);
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String propertyName) {
+            return processorNode.getPropertyDescriptor(propertyName);
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return processorNode.getPropertyDescriptors();
+        }
+    }
+
+    /**
+     * ConfigurableComponentAuthorizable for a ControllerServiceNode.
+     */
+    private static class ControllerServiceConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable {
+        private final ControllerServiceNode controllerServiceNode;
+
+        public ControllerServiceConfigurableComponentAuthorizable(ControllerServiceNode controllerServiceNode) {
+            this.controllerServiceNode = controllerServiceNode;
+        }
+
+        @Override
+        public Authorizable getAuthorizable() {
+            return controllerServiceNode;
+        }
+
+        @Override
+        public boolean isRestricted() {
+            return controllerServiceNode.isRestricted();
+        }
+
+        @Override
+        public String getValue(PropertyDescriptor propertyDescriptor) {
+            return controllerServiceNode.getProperty(propertyDescriptor);
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String propertyName) {
+            return controllerServiceNode.getControllerServiceImplementation().getPropertyDescriptor(propertyName);
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return controllerServiceNode.getControllerServiceImplementation().getPropertyDescriptors();
+        }
+    }
+
+    /**
+     * ConfigurableComponentAuthorizable for a ProcessorNode.
+     */
+    private static class ReportingTaskConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable {
+        private final ReportingTaskNode reportingTaskNode;
+
+        public ReportingTaskConfigurableComponentAuthorizable(ReportingTaskNode reportingTaskNode) {
+            this.reportingTaskNode = reportingTaskNode;
+        }
+
+        @Override
+        public Authorizable getAuthorizable() {
+            return reportingTaskNode;
+        }
+
+        @Override
+        public boolean isRestricted() {
+            return reportingTaskNode.isRestricted();
+        }
+
+        @Override
+        public String getValue(PropertyDescriptor propertyDescriptor) {
+            return reportingTaskNode.getProperty(propertyDescriptor);
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String propertyName) {
+            return reportingTaskNode.getReportingTask().getPropertyDescriptor(propertyName);
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return reportingTaskNode.getReportingTask().getPropertyDescriptors();
+        }
+    }
+
     @Override
     public Authorizable getSystem() {
         return SYSTEM_AUTHORIZABLE;

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.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/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index 02a36c1..3906870 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -16,43 +16,18 @@
  */
 package org.apache.nifi.web.api;
 
-import static javax.ws.rs.core.Response.Status.NOT_FOUND;
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
-
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.core.CacheControl;
-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.Response.ResponseBuilder;
-import javax.ws.rs.core.UriBuilder;
-import javax.ws.rs.core.UriBuilderException;
-import javax.ws.rs.core.UriInfo;
-
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.sun.jersey.api.core.HttpContext;
+import com.sun.jersey.api.representation.Form;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.AuthorizeAccess;
+import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.ProcessGroupAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
@@ -78,7 +53,6 @@ import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.RevisionDTO;
-import org.apache.nifi.web.api.dto.SnippetDTO;
 import org.apache.nifi.web.api.entity.ComponentEntity;
 import org.apache.nifi.web.api.entity.Entity;
 import org.apache.nifi.web.api.entity.TransactionResultEntity;
@@ -87,12 +61,37 @@ import org.apache.nifi.web.security.util.CacheKey;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.sun.jersey.api.core.HttpContext;
-import com.sun.jersey.api.representation.Form;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
-import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.CacheControl;
+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.Response.ResponseBuilder;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriBuilderException;
+import javax.ws.rs.core.UriInfo;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import static javax.ws.rs.core.Response.Status.NOT_FOUND;
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
 
 /**
  * Base class for controllers.
@@ -434,28 +433,60 @@ public abstract class ApplicationResource {
     }
 
     /**
-     * Authorizes the specified Snippet with the specified request action.
+     * Authorizes the specified process group.
      *
-     * @param authorizer authorizer
-     * @param lookup     lookup
-     * @param action     action
+     * @param processGroupAuthorizable      process group
+     * @param authorizer                    authorizer
+     * @param lookup                        lookup
+     * @param action                        action
+     * @param authorizeReferencedServices   whether to authorize referenced services
+     * @param authorizeTemplates            whether to authorize templates
+     * @param authorizeControllerServices   whether to authorize controller services
      */
-    protected void authorizeSnippet(final Snippet snippet, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action) {
+    protected void authorizeProcessGroup(final ProcessGroupAuthorizable processGroupAuthorizable, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action,
+                                         final boolean authorizeReferencedServices, final boolean authorizeTemplates,
+                                         final boolean authorizeControllerServices, final boolean authorizeTransitiveServices) {
+
         final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action, NiFiUserUtils.getNiFiUser());
 
-        snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(processGroupAuthorizable -> {
-            // authorize the process group
-            authorize.accept(processGroupAuthorizable.getAuthorizable());
+        // authorize the process group
+        authorize.accept(processGroupAuthorizable.getAuthorizable());
+
+        // authorize the contents of the group - these methods return all encapsulated components (recursive)
+        processGroupAuthorizable.getEncapsulatedProcessors().forEach(processorAuthorizable -> {
+            // authorize the processor
+            authorize.accept(processorAuthorizable.getAuthorizable());
 
-            // authorize the contents of the group
-            processGroupAuthorizable.getEncapsulatedAuthorizables().forEach(authorize);
+            // authorize any referenced services if necessary
+            if (authorizeReferencedServices) {
+                AuthorizeControllerServiceReference.authorizeControllerServiceReferences(processorAuthorizable, authorizer, lookup, authorizeTransitiveServices);
+            }
         });
-        snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
-        snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id).getAuthorizable()).forEach(authorize);
-        snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).forEach(authorize);
-        snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).forEach(authorize);
-        snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(connAuth -> authorize.accept(connAuth.getAuthorizable()));
-        snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedConnections().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedInputPorts().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedOutputPorts().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedFunnels().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedLabels().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedProcessGroups().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedRemoteProcessGroups().forEach(authorize);
+
+        // authorize templates if necessary
+        if (authorizeTemplates) {
+            processGroupAuthorizable.getEncapsulatedTemplates().forEach(authorize);
+        }
+
+        // authorize controller services if necessary
+        if (authorizeControllerServices) {
+            processGroupAuthorizable.getEncapsulatedControllerServices().forEach(controllerServiceAuthorizable -> {
+                // authorize the controller service
+                authorize.accept(controllerServiceAuthorizable.getAuthorizable());
+
+                // authorize any referenced services if necessary
+                if (authorizeReferencedServices) {
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(controllerServiceAuthorizable, authorizer, lookup, authorizeTransitiveServices);
+                }
+            });
+        }
     }
 
     /**
@@ -465,18 +496,26 @@ public abstract class ApplicationResource {
      * @param lookup     lookup
      * @param action     action
      */
-    protected void authorizeSnippet(final SnippetDTO snippet, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action) {
+    protected void authorizeSnippet(final Snippet snippet, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action,
+                                    final boolean authorizeReferencedServices, final boolean authorizeTransitiveServices) {
         final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action, NiFiUserUtils.getNiFiUser());
 
+        // authorize each component in the specified snippet
         snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(processGroupAuthorizable -> {
-            // authorize the process group
-            authorize.accept(processGroupAuthorizable.getAuthorizable());
-
-            // authorize the contents of the group
-            processGroupAuthorizable.getEncapsulatedAuthorizables().forEach(authorize);
+            // note - we are not authorizing templates or controller services as they are not considered when using this snippet. however,
+            // referenced services are considered so those are explicitly authorized when authorizing a processor
+            authorizeProcessGroup(processGroupAuthorizable, authorizer, lookup, action, authorizeReferencedServices, false, false, authorizeTransitiveServices);
         });
         snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
-        snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id).getAuthorizable()).forEach(authorize);
+        snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).forEach(processorAuthorizable -> {
+            // authorize the processor
+            authorize.accept(processorAuthorizable.getAuthorizable());
+
+            // authorize any referenced services if necessary
+            if (authorizeReferencedServices) {
+                AuthorizeControllerServiceReference.authorizeControllerServiceReferences(processorAuthorizable, authorizer, lookup, authorizeTransitiveServices);
+            }
+        });
         snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).forEach(authorize);
         snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).forEach(authorize);
         snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(connAuth -> authorize.accept(connAuth.getAuthorizable()));

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
index 47a8189..9544ff8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
@@ -570,7 +570,7 @@ public class ControllerServiceResource extends ApplicationResource {
             response = ControllerServiceEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /controller-services/{uuid}", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services if this request changes the reference - /controller-services/{uuid}", type = "")
             }
     )
     @ApiResponses(
@@ -660,7 +660,8 @@ public class ControllerServiceResource extends ApplicationResource {
             value = "Deletes a controller service",
             response = ControllerServiceEntity.class,
             authorizations = {
-                    @Authorization(value = "Write - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Write - /controller-services/{uuid}", type = ""),
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
             }
     )
     @ApiResponses(
@@ -704,8 +705,11 @@ public class ControllerServiceResource extends ApplicationResource {
                 requestControllerServiceEntity,
                 requestRevision,
                 lookup -> {
-                    final Authorizable controllerService = lookup.getControllerService(id).getAuthorizable();
-                    controllerService.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                    final ConfigurableComponentAuthorizable controllerService = lookup.getControllerService(id);
+                    controllerService.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // verify any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(controllerService, authorizer, lookup, false);
                 },
                 () -> serviceFacade.verifyDeleteControllerService(id),
                 (revision, controllerServiceEntity) -> {

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/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 c1ea58c..dbe12db 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
@@ -373,14 +373,9 @@ public class ProcessGroupResource extends ApplicationResource {
                     final NiFiUser user = NiFiUserUtils.getNiFiUser();
                     final ProcessGroupAuthorizable processGroupAuthorizable = lookup.getProcessGroup(id);
 
-                    // ensure write to the process group
-                    final Authorizable processGroup = processGroupAuthorizable.getAuthorizable();
-                    processGroup.authorize(authorizer, RequestAction.WRITE, user);
-
-                    // ensure write to all encapsulated components
-                    processGroupAuthorizable.getEncapsulatedAuthorizables().forEach(encaupsulatedAuthorizable -> {
-                        encaupsulatedAuthorizable.authorize(authorizer, RequestAction.WRITE, user);
-                    });
+                    // ensure write to this process group and all encapsulated components including templates and controller services. additionally, ensure
+                    // read to any referenced services by encapsulated components
+                    authorizeProcessGroup(processGroupAuthorizable, authorizer, lookup, RequestAction.WRITE, true, true, true, false);
                 },
                 () -> serviceFacade.verifyDeleteProcessGroup(id),
                 (revision, processGroupEntity) -> {
@@ -1647,7 +1642,7 @@ public class ProcessGroupResource extends ApplicationResource {
     @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}/snippet-instance")
     @ApiOperation(
-            value = "Copies a snippet",
+            value = "Copies a snippet and discards it.",
             response = FlowSnippetEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /process-groups/{uuid}", type = ""),
@@ -1692,7 +1687,7 @@ public class ProcessGroupResource extends ApplicationResource {
                 serviceFacade,
                 requestCopySnippetEntity,
                 lookup -> {
-                    authorizeSnippetUsage(lookup, groupId, requestCopySnippetEntity.getSnippetId());
+                    authorizeSnippetUsage(lookup, groupId, requestCopySnippetEntity.getSnippetId(), false);
                 },
                 null,
                 copySnippetRequestEntity -> {
@@ -1810,13 +1805,13 @@ public class ProcessGroupResource extends ApplicationResource {
     // templates
     // ---------
 
-    private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId) {
+    private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId, final boolean authorizeTransitiveServices) {
         // ensure write access to the target process group
         lookup.getProcessGroup(groupId).getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
 
-        // ensure read permission to every component in the snippet
+        // ensure read permission to every component in the snippet including referenced services
         final Snippet snippet = lookup.getSnippet(snippetId);
-        authorizeSnippet(snippet, authorizer, lookup, RequestAction.READ);
+        authorizeSnippet(snippet, authorizer, lookup, RequestAction.READ, true, authorizeTransitiveServices);
     }
 
     /**
@@ -1831,7 +1826,7 @@ public class ProcessGroupResource extends ApplicationResource {
     @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}/templates")
     @ApiOperation(
-            value = "Creates a template",
+            value = "Creates a template and discards the specified snippet.",
             response = TemplateEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /process-groups/{uuid}", type = ""),
@@ -1871,7 +1866,7 @@ public class ProcessGroupResource extends ApplicationResource {
                 serviceFacade,
                 requestCreateTemplateRequestEntity,
                 lookup -> {
-                    authorizeSnippetUsage(lookup, groupId, requestCreateTemplateRequestEntity.getSnippetId());
+                    authorizeSnippetUsage(lookup, groupId, requestCreateTemplateRequestEntity.getSnippetId(), true);
                 },
                 () -> serviceFacade.verifyCanAddTemplate(groupId, requestCreateTemplateRequestEntity.getName()),
                 createTemplateRequestEntity -> {

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/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 fcdb99f..9bd02d4 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
@@ -396,7 +396,7 @@ public class ProcessorResource extends ApplicationResource {
             response = ProcessorEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /processors/{uuid}", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services if this request changes the reference - /controller-services/{uuid}", type = "")
             }
     )
     @ApiResponses(
@@ -487,7 +487,8 @@ public class ProcessorResource extends ApplicationResource {
             value = "Deletes a processor",
             response = ProcessorEntity.class,
             authorizations = {
-                    @Authorization(value = "Write - /processors/{uuid}", type = "")
+                    @Authorization(value = "Write - /processors/{uuid}", type = ""),
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
             }
     )
     @ApiResponses(
@@ -530,8 +531,11 @@ public class ProcessorResource extends ApplicationResource {
                 requestProcessorEntity,
                 requestRevision,
                 lookup -> {
-                    final Authorizable processor = lookup.getProcessor(id).getAuthorizable();
-                    processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                    final ConfigurableComponentAuthorizable processor = lookup.getProcessor(id);
+                    processor.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // verify any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(processor, authorizer, lookup, false);
                 },
                 () -> serviceFacade.verifyDeleteProcessor(id),
                 (revision, processorEntity) -> {

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/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 57bc51d..31ceb9f 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
@@ -374,7 +374,7 @@ public class ReportingTaskResource extends ApplicationResource {
             response = ReportingTaskEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /reporting-tasks/{uuid}", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services if this request changes the reference - /controller-services/{uuid}", type = "")
             }
     )
     @ApiResponses(
@@ -464,7 +464,8 @@ public class ReportingTaskResource extends ApplicationResource {
             value = "Deletes a reporting task",
             response = ReportingTaskEntity.class,
             authorizations = {
-                    @Authorization(value = "Write - /reporting-tasks/{uuid}", type = "")
+                    @Authorization(value = "Write - /reporting-tasks/{uuid}", type = ""),
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
             }
     )
     @ApiResponses(
@@ -508,8 +509,11 @@ public class ReportingTaskResource extends ApplicationResource {
                 requestReportingTaskEntity,
                 requestRevision,
                 lookup -> {
-                    final Authorizable reportingTask = lookup.getReportingTask(id).getAuthorizable();
-                    reportingTask.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                    final ConfigurableComponentAuthorizable reportingTask = lookup.getReportingTask(id);
+                    reportingTask.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // verify any referenced services
+                    AuthorizeControllerServiceReference.authorizeControllerServiceReferences(reportingTask, authorizer, lookup, false);
                 },
                 () -> serviceFacade.verifyDeleteReportingTask(id),
                 (revision, reportingTaskEntity) -> {

http://git-wip-us.apache.org/repos/asf/nifi/blob/bc223fa1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
index 050017b..271febe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java
@@ -23,8 +23,10 @@ import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
 import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.AuthorizableLookup;
 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.controller.Snippet;
 import org.apache.nifi.web.NiFiServiceFacade;
@@ -47,6 +49,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.net.URI;
 import java.util.Set;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
 /**
@@ -92,6 +95,32 @@ public class SnippetResource extends ApplicationResource {
     // --------
 
     /**
+     * Authorizes the specified snippet request with the specified request action. This method is used when creating a snippet. Because we do not know what
+     * the snippet will be used for, we just ensure the user has permissions to each selected component. Some actions may require additional permissions
+     * (including referenced services) but those will be enforced when the snippet is used.
+     *
+     * @param authorizer authorizer
+     * @param lookup     lookup
+     * @param action     action
+     */
+    private void authorizeSnippetRequest(final SnippetDTO snippetRequest, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action) {
+        final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action, NiFiUserUtils.getNiFiUser());
+
+        snippetRequest.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(processGroupAuthorizable -> {
+            // note - we are not authorizing templates or controller services as they are not considered when using this snippet. additionally,
+            // we are not checking referenced services since we do not know how this snippet will be used. these checks should be performed
+            // in a subsequent action with this snippet
+            authorizeProcessGroup(processGroupAuthorizable, authorizer, lookup, action, false, false, false, false);
+        });
+        snippetRequest.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
+        snippetRequest.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id).getAuthorizable()).forEach(authorize);
+        snippetRequest.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).forEach(authorize);
+        snippetRequest.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).forEach(authorize);
+        snippetRequest.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(connAuth -> authorize.accept(connAuth.getAuthorizable()));
+        snippetRequest.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).forEach(authorize);
+    }
+
+    /**
      * Creates a snippet based off the specified configuration.
      *
      * @param httpServletRequest request
@@ -102,7 +131,7 @@ public class SnippetResource extends ApplicationResource {
     @Consumes(MediaType.APPLICATION_JSON)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Creates a snippet",
+            value = "Creates a snippet. The snippet will be automatically discarded if not used in a subsequent request after 1 minute.",
             response = SnippetEntity.class,
             authorizations = {
                     @Authorization(value = "Read or Write - /{component-type}/{uuid} - For every component (all Read or all Write) in the Snippet and their descendant components", type = "")
@@ -141,7 +170,7 @@ public class SnippetResource extends ApplicationResource {
                 serviceFacade,
                 requestSnippetEntity,
                 lookup -> {
-                    final SnippetDTO snippet = requestSnippetEntity.getSnippet();
+                    final SnippetDTO snippetRequest = requestSnippetEntity.getSnippet();
 
                     // the snippet being created may be used later for batch component modifications,
                     // copy/paste, or template creation. during those subsequent actions, the snippet
@@ -150,9 +179,9 @@ public class SnippetResource extends ApplicationResource {
                     // read OR write
 
                     try {
-                        authorizeSnippet(snippet, authorizer, lookup, RequestAction.READ);
+                        authorizeSnippetRequest(snippetRequest, authorizer, lookup, RequestAction.READ);
                     } catch (final AccessDeniedException e) {
-                        authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE);
+                        authorizeSnippetRequest(snippetRequest, authorizer, lookup, RequestAction.WRITE);
                     }
                 },
                 null,
@@ -183,7 +212,7 @@ public class SnippetResource extends ApplicationResource {
     @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
     @ApiOperation(
-            value = "Move's the components in this Snippet into a new Process Group and drops the snippet",
+            value = "Move's the components in this Snippet into a new Process Group and discards the snippet",
             response = SnippetEntity.class,
             authorizations = {
                     @Authorization(value = "Write Process Group - /process-groups/{uuid}", type = ""),
@@ -238,9 +267,9 @@ public class SnippetResource extends ApplicationResource {
                         lookup.getProcessGroup(requestSnippetDTO.getParentGroupId()).getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
                     }
 
-                        // ensure write permission to every component in the snippet
-                        final Snippet snippet = lookup.getSnippet(snippetId);
-                        authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE);
+                    // ensure write permission to every component in the snippet excluding referenced services
+                    final Snippet snippet = lookup.getSnippet(snippetId);
+                    authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE, false, false);
                 },
                 () -> serviceFacade.verifyUpdateSnippet(requestSnippetDTO, requestRevisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet())),
                 (revisions, snippetEntity) -> {
@@ -264,7 +293,7 @@ public class SnippetResource extends ApplicationResource {
     @Produces(MediaType.APPLICATION_JSON)
     @Path("{id}")
     @ApiOperation(
-            value = "Deletes the components in a snippet and drops the snippet",
+            value = "Deletes the components in a snippet and discards the snippet",
             response = SnippetEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /{component-type}/{uuid} - For each component in the Snippet and their descendant components", type = "")
@@ -301,9 +330,9 @@ public class SnippetResource extends ApplicationResource {
                 requestEntity,
                 requestRevisions,
                 lookup -> {
-                    // ensure read permission to every component in the snippet
+                    // ensure write permission to every component in the snippet excluding referenced services
                     final Snippet snippet = lookup.getSnippet(snippetId);
-                    authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE);
+                    authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE, true, false);
                 },
                 () -> serviceFacade.verifyDeleteSnippet(snippetId, requestRevisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet())),
                 (revisions, entity) -> {