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/12/02 17:55:46 UTC

nifi git commit: NIFI-3135: - Authorizing restricted components on snippet usage. - Updating REST API docs accordingly. - Adding some integration tests to exercise the additional restricted component checks.

Repository: nifi
Updated Branches:
  refs/heads/master 69b23adf1 -> d8d29811f


NIFI-3135: - Authorizing restricted components on snippet usage.
- Updating REST API docs accordingly.
- Adding some integration tests to exercise the additional restricted component checks.

This closes #1287.

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/d8d29811
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d8d29811
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d8d29811

Branch: refs/heads/master
Commit: d8d29811f559b3ae293475a88d52a3c62f9681be
Parents: 69b23ad
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Dec 1 14:54:40 2016 -0500
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Dec 2 12:55:23 2016 -0500

----------------------------------------------------------------------
 .../nifi/authorization/AuthorizableLookup.java  |   5 +-
 .../authorization/ProcessGroupAuthorizable.java |   4 +-
 .../nifi/authorization/SnippetAuthorizable.java |  82 +++++
 .../StandardAuthorizableLookup.java             | 317 ++++++++++++++-----
 .../authorization/TemplateAuthorizable.java     |  48 +++
 .../nifi/web/api/ApplicationResource.java       |  24 +-
 .../apache/nifi/web/api/ControllerResource.java |   6 +-
 .../nifi/web/api/ProcessGroupResource.java      |  67 +++-
 .../apache/nifi/web/api/SnippetResource.java    |  15 +-
 .../apache/nifi/web/api/TemplateResource.java   |   4 +-
 .../accesscontrol/ITProcessorAccessControl.java | 190 +++++++++++
 .../org.apache.nifi.processor.Processor         |   3 +-
 12 files changed, 639 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
index 1d1a828..a24edd9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
@@ -17,7 +17,6 @@
 package org.apache.nifi.authorization;
 
 import org.apache.nifi.authorization.resource.Authorizable;
-import org.apache.nifi.controller.Snippet;
 
 public interface AuthorizableLookup {
 
@@ -201,7 +200,7 @@ public interface AuthorizableLookup {
      * @param id template id
      * @return authorizable
      */
-    Authorizable getTemplate(String id);
+    TemplateAuthorizable getTemplate(String id);
 
     /**
      * Get the authorizable connectable.
@@ -217,7 +216,7 @@ public interface AuthorizableLookup {
      * @param id snippet id
      * @return snippet of authorizable's
      */
-    Snippet getSnippet(String id);
+    SnippetAuthorizable getSnippet(String id);
 
     /**
      * Get the {@link Authorizable} that represents the resource of users and user groups.

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/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 289b8fa..3e2fecc 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
@@ -43,7 +43,7 @@ public interface ProcessGroupAuthorizable {
      *
      * @return all encapsulated connections
      */
-    Set<Authorizable> getEncapsulatedConnections();
+    Set<ConnectionAuthorizable> getEncapsulatedConnections();
 
     /**
      * The authorizables for all encapsulated input ports. Non null
@@ -78,7 +78,7 @@ public interface ProcessGroupAuthorizable {
      *
      * @return all encapsulated process groups
      */
-    Set<Authorizable> getEncapsulatedProcessGroups();
+    Set<ProcessGroupAuthorizable> getEncapsulatedProcessGroups();
 
     /**
      * The authorizables for all encapsulated remote process groups. Non null

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
new file mode 100644
index 0000000..5503f44
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
@@ -0,0 +1,82 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.authorization.resource.Authorizable;
+
+import java.util.Set;
+
+/**
+ * Authorizable for a Snippet.
+ */
+public interface SnippetAuthorizable {
+    /**
+     * The authorizables for selected processors. Non null
+     *
+     * @return processors
+     */
+    Set<ConfigurableComponentAuthorizable> getSelectedProcessors();
+
+    /**
+     * The authorizables for selected connections. Non null
+     *
+     * @return  connections
+     */
+    Set<ConnectionAuthorizable> getSelectedConnections();
+
+    /**
+     * The authorizables for selected input ports. Non null
+     *
+     * @return input ports
+     */
+    Set<Authorizable> getSelectedInputPorts();
+
+    /**
+     * The authorizables for selected output ports. Non null
+     *
+     * @return output ports
+     */
+    Set<Authorizable> getSelectedOutputPorts();
+
+    /**
+     * The authorizables for selected funnels. Non null
+     *
+     * @return funnels
+     */
+    Set<Authorizable> getSelectedFunnels();
+
+    /**
+     * The authorizables for selected labels. Non null
+     *
+     * @return labels
+     */
+    Set<Authorizable> getSelectedLabels();
+
+    /**
+     * The authorizables for selected process groups. Non null
+     *
+     * @return process groups
+     */
+    Set<ProcessGroupAuthorizable> getSelectedProcessGroups();
+
+    /**
+     * The authorizables for selected remote process groups. Non null
+     *
+     * @return remote process groups
+     */
+    Set<Authorizable> getSelectedRemoteProcessGroups();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/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 4045f29..f74931d 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
@@ -34,6 +34,7 @@ import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.Snippet;
+import org.apache.nifi.controller.Template;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceReference;
 import org.apache.nifi.groups.ProcessGroup;
@@ -41,6 +42,8 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.remote.PortAuthorizationResult;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.FlowSnippetDTO;
+import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.dao.AccessPolicyDAO;
 import org.apache.nifi.web.dao.ConnectionDAO;
@@ -55,6 +58,7 @@ import org.apache.nifi.web.dao.ReportingTaskDAO;
 import org.apache.nifi.web.dao.SnippetDAO;
 import org.apache.nifi.web.dao.TemplateDAO;
 
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
@@ -221,91 +225,13 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ConnectionAuthorizable getConnection(final String id) {
         final Connection connection = connectionDAO.getConnection(id);
-        return new ConnectionAuthorizable() {
-            @Override
-            public Authorizable getAuthorizable() {
-                return connection;
-            }
-
-            @Override
-            public Connectable getSource() {
-                return connection.getSource();
-            }
-
-            @Override
-            public Connectable getDestination() {
-                return connection.getDestination();
-            }
-
-            @Override
-            public ProcessGroup getParentGroup() {
-                return connection.getProcessGroup();
-            }
-        };
+        return new StandardConnectionAuthorizable(connection);
     }
 
     @Override
     public ProcessGroupAuthorizable getProcessGroup(final String id) {
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(id);
-
-        return new ProcessGroupAuthorizable() {
-            @Override
-            public Authorizable getAuthorizable() {
-                return processGroup;
-            }
-
-            @Override
-            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());
-            }
-        };
+        return new StandardProcessGroupAuthorizable(processGroup);
     }
 
     @Override
@@ -411,8 +337,75 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     }
 
     @Override
-    public Snippet getSnippet(final String id) {
-        return snippetDAO.getSnippet(id);
+    public SnippetAuthorizable getSnippet(final String id) {
+        final Snippet snippet = snippetDAO.getSnippet(id);
+        final ProcessGroup processGroup = processGroupDAO.getProcessGroup(snippet.getParentGroupId());
+
+        return new SnippetAuthorizable() {
+            @Override
+            public Set<ConfigurableComponentAuthorizable> getSelectedProcessors() {
+                return processGroup.getProcessors().stream()
+                        .filter(processor -> snippet.getProcessors().containsKey(processor.getIdentifier()))
+                        .map(processor -> getProcessor(processor.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<ConnectionAuthorizable> getSelectedConnections() {
+                return processGroup.getConnections().stream()
+                        .filter(connection -> snippet.getConnections().containsKey(connection.getIdentifier()))
+                        .map(connection -> getConnection(connection.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getSelectedInputPorts() {
+                return processGroup.getInputPorts().stream()
+                        .filter(inputPort -> snippet.getInputPorts().containsKey(inputPort.getIdentifier()))
+                        .map(inputPort -> getInputPort(inputPort.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getSelectedOutputPorts() {
+                return processGroup.getOutputPorts().stream()
+                        .filter(outputPort -> snippet.getOutputPorts().containsKey(outputPort.getIdentifier()))
+                        .map(outputPort -> getOutputPort(outputPort.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getSelectedFunnels() {
+                return processGroup.getFunnels().stream()
+                        .filter(funnel -> snippet.getFunnels().containsKey(funnel.getIdentifier()))
+                        .map(funnel -> getFunnel(funnel.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getSelectedLabels() {
+                return processGroup.getLabels().stream()
+                        .filter(label -> snippet.getLabels().containsKey(label.getIdentifier()))
+                        .map(label -> getLabel(label.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<ProcessGroupAuthorizable> getSelectedProcessGroups() {
+                return processGroup.getProcessGroups().stream()
+                        .filter(processGroup -> snippet.getProcessGroups().containsKey(processGroup.getIdentifier()))
+                        .map(processGroup -> getProcessGroup(processGroup.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<Authorizable> getSelectedRemoteProcessGroups() {
+                return processGroup.getRemoteProcessGroups().stream()
+                        .filter(remoteProcessGroup -> snippet.getRemoteProcessGroups().containsKey(remoteProcessGroup.getIdentifier()))
+                        .map(remoteProcessGroup -> getRemoteProcessGroup(remoteProcessGroup.getIdentifier()))
+                        .collect(Collectors.toSet());
+            }
+        };
     }
 
     @Override
@@ -530,7 +523,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
                 authorizable = getReportingTask(componentId).getAuthorizable();
                 break;
             case Template:
-                authorizable = getTemplate(componentId);
+                authorizable = getTemplate(componentId).getAuthorizable();
                 break;
             case Data:
                 authorizable = controllerFacade.getDataAuthorizable(componentId);
@@ -629,9 +622,62 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         return authorizable;
     }
 
+    /**
+     * Creates temporary instances of all processors and controller services found in the specified snippet.
+     *
+     * @param snippet               snippet
+     * @param processors            processors
+     * @param controllerServices    controller services
+     */
+    private void createTemporaryProcessorsAndControllerServices(final FlowSnippetDTO snippet,
+                                                                final Set<ConfigurableComponentAuthorizable> processors,
+                                                                final Set<ConfigurableComponentAuthorizable> controllerServices) {
+
+        if (snippet == null) {
+            return;
+        }
+
+        if (snippet.getProcessors() != null) {
+            processors.addAll(snippet.getProcessors().stream().map(processor -> getProcessorByType(processor.getType())).collect(Collectors.toSet()));
+        }
+
+        if (snippet.getControllerServices() != null) {
+            controllerServices.addAll(snippet.getControllerServices().stream().map(controllerService -> getControllerServiceByType(controllerService.getType())).collect(Collectors.toSet()));
+        }
+
+        if (snippet.getProcessGroups() != null) {
+            snippet.getProcessGroups().stream().forEach(group -> createTemporaryProcessorsAndControllerServices(group.getContents(), processors, controllerServices));
+        }
+    }
+
     @Override
-    public Authorizable getTemplate(final String id) {
-        return templateDAO.getTemplate(id);
+    public TemplateAuthorizable getTemplate(final String id) {
+        final Template template = templateDAO.getTemplate(id);
+        final TemplateDTO contents = template.getDetails();
+
+        // templates are immutable so we can pre-compute all encapsulated processors and controller services
+        final Set<ConfigurableComponentAuthorizable> processors = new HashSet<>();
+        final Set<ConfigurableComponentAuthorizable> controllerServices = new HashSet<>();
+
+        // find all processors and controller services
+        createTemporaryProcessorsAndControllerServices(contents.getSnippet(), processors, controllerServices);
+
+        return new TemplateAuthorizable() {
+            @Override
+            public Authorizable getAuthorizable() {
+                return template;
+            }
+
+            @Override
+            public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() {
+                return processors;
+            }
+
+            @Override
+            public Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices() {
+                return controllerServices;
+            }
+        };
     }
 
     @Override
@@ -645,6 +691,11 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         return RESTRICTED_COMPONENTS_AUTHORIZABLE;
     }
 
+    @Override
+    public Authorizable getSystem() {
+        return SYSTEM_AUTHORIZABLE;
+    }
+
     /**
      * ConfigurableComponentAuthorizable for a ProcessorNode.
      */
@@ -753,9 +804,99 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
     }
 
-    @Override
-    public Authorizable getSystem() {
-        return SYSTEM_AUTHORIZABLE;
+    private static class StandardProcessGroupAuthorizable implements ProcessGroupAuthorizable {
+        private final ProcessGroup processGroup;
+
+        public StandardProcessGroupAuthorizable(ProcessGroup processGroup) {
+            this.processGroup = processGroup;
+        }
+
+        @Override
+        public Authorizable getAuthorizable() {
+            return processGroup;
+        }
+
+        @Override
+        public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() {
+            return processGroup.findAllProcessors().stream().map(
+                    processorNode -> new ProcessorConfigurableComponentAuthorizable(processorNode)).collect(Collectors.toSet());
+        }
+
+        @Override
+        public Set<ConnectionAuthorizable> getEncapsulatedConnections() {
+            return processGroup.findAllConnections().stream().map(
+                    connection -> new StandardConnectionAuthorizable(connection)).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<ProcessGroupAuthorizable> getEncapsulatedProcessGroups() {
+            return processGroup.findAllProcessGroups().stream().map(
+                    group -> new StandardProcessGroupAuthorizable(group)).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());
+        }
+    }
+
+    private static class StandardConnectionAuthorizable implements ConnectionAuthorizable {
+        private final Connection connection;
+
+        public StandardConnectionAuthorizable(Connection connection) {
+            this.connection = connection;
+        }
+
+        @Override
+        public Authorizable getAuthorizable() {
+            return connection;
+        }
+
+        @Override
+        public Connectable getSource() {
+            return connection.getSource();
+        }
+
+        @Override
+        public Connectable getDestination() {
+            return connection.getDestination();
+        }
+
+        @Override
+        public ProcessGroup getParentGroup() {
+            return connection.getProcessGroup();
+        }
     }
 
     public void setProcessorDAO(ProcessorDAO processorDAO) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateAuthorizable.java
new file mode 100644
index 0000000..17a8896
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateAuthorizable.java
@@ -0,0 +1,48 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.authorization.resource.Authorizable;
+
+import java.util.Set;
+
+/**
+ * Authorizable for a Template.
+ */
+public interface TemplateAuthorizable {
+    /**
+     * Returns the authorizable for this template. Non null
+     *
+     * @return authorizable
+     */
+    Authorizable getAuthorizable();
+
+    /**
+     * Returns temporary instances of all encapsulated processors. Non null
+     *
+     * @return temporary instances of all encapsulated processors
+     */
+    Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors();
+
+    /**
+     * Returns temporary instances of all encapsulated controller services. Non null
+     *
+     * @return temporary instances of all encapsulated controller services
+     */
+    Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/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 3906870..79ddc81 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
@@ -29,6 +29,7 @@ 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.SnippetAuthorizable;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -40,7 +41,6 @@ import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
 import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.controller.FlowController;
-import org.apache.nifi.controller.Snippet;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
 import org.apache.nifi.remote.VersionNegotiator;
 import org.apache.nifi.remote.exception.BadRequestException;
@@ -462,12 +462,12 @@ public abstract class ApplicationResource {
                 AuthorizeControllerServiceReference.authorizeControllerServiceReferences(processorAuthorizable, authorizer, lookup, authorizeTransitiveServices);
             }
         });
-        processGroupAuthorizable.getEncapsulatedConnections().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedConnections().stream().map(connection -> connection.getAuthorizable()).forEach(authorize);
         processGroupAuthorizable.getEncapsulatedInputPorts().forEach(authorize);
         processGroupAuthorizable.getEncapsulatedOutputPorts().forEach(authorize);
         processGroupAuthorizable.getEncapsulatedFunnels().forEach(authorize);
         processGroupAuthorizable.getEncapsulatedLabels().forEach(authorize);
-        processGroupAuthorizable.getEncapsulatedProcessGroups().forEach(authorize);
+        processGroupAuthorizable.getEncapsulatedProcessGroups().stream().map(group -> group.getAuthorizable()).forEach(authorize);
         processGroupAuthorizable.getEncapsulatedRemoteProcessGroups().forEach(authorize);
 
         // authorize templates if necessary
@@ -496,18 +496,19 @@ public abstract class ApplicationResource {
      * @param lookup     lookup
      * @param action     action
      */
-    protected void authorizeSnippet(final Snippet snippet, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action,
+    protected void authorizeSnippet(final SnippetAuthorizable 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 -> {
+        snippet.getSelectedProcessGroups().stream().forEach(processGroupAuthorizable -> {
             // 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)).forEach(processorAuthorizable -> {
+        snippet.getSelectedRemoteProcessGroups().stream().forEach(authorize);
+        snippet.getSelectedProcessors().stream().forEach(processorAuthorizable -> {
             // authorize the processor
             authorize.accept(processorAuthorizable.getAuthorizable());
 
@@ -516,10 +517,11 @@ public abstract class ApplicationResource {
                 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()));
-        snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).forEach(authorize);
+        snippet.getSelectedInputPorts().stream().forEach(authorize);
+        snippet.getSelectedOutputPorts().stream().forEach(authorize);
+        snippet.getSelectedConnections().stream().forEach(connAuth -> authorize.accept(connAuth.getAuthorizable()));
+        snippet.getSelectedFunnels().stream().forEach(authorize);
+        snippet.getSelectedLabels().stream().forEach(authorize);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.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/ControllerResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 9d08e22..20b9f87 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -238,7 +238,8 @@ public class ControllerResource extends ApplicationResource {
             response = ReportingTaskEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /controller", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = ""),
+                    @Authorization(value = "Write - if the Reporting Task is restricted - /restricted-components", type = "")
             }
     )
     @ApiResponses(
@@ -330,7 +331,8 @@ public class ControllerResource extends ApplicationResource {
             response = ControllerServiceEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /controller", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = ""),
+                    @Authorization(value = "Write - if the Controller Service is restricted - /restricted-components", type = "")
             }
     )
     @ApiResponses(

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/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 17c65cf..8b9366f 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
@@ -31,10 +31,11 @@ import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
 import org.apache.nifi.authorization.ProcessGroupAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.SnippetAuthorizable;
+import org.apache.nifi.authorization.TemplateAuthorizable;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.controller.Snippet;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.Revision;
@@ -100,6 +101,8 @@ import java.net.URISyntaxException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
 
 /**
  * RESTful endpoint for managing a Group.
@@ -555,7 +558,8 @@ public class ProcessGroupResource extends ApplicationResource {
             response = ProcessorEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /process-groups/{uuid}", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = ""),
+                    @Authorization(value = "Write - if the Processor is restricted - /restricted-components", type = "")
             }
     )
     @ApiResponses(
@@ -1646,7 +1650,8 @@ public class ProcessGroupResource extends ApplicationResource {
             response = FlowSnippetEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /process-groups/{uuid}", type = ""),
-                    @Authorization(value = "Read - /{component-type}/{uuid} - For each component in the snippet and their descendant components", type = "")
+                    @Authorization(value = "Read - /{component-type}/{uuid} - For each component in the snippet and their descendant components", type = ""),
+                    @Authorization(value = "Write - if the snippet contains any restricted Processors - /restricted-components", type = "")
             }
     )
     @ApiResponses(
@@ -1687,7 +1692,24 @@ public class ProcessGroupResource extends ApplicationResource {
                 serviceFacade,
                 requestCopySnippetEntity,
                 lookup -> {
-                    authorizeSnippetUsage(lookup, groupId, requestCopySnippetEntity.getSnippetId(), false);
+                    final NiFiUser user = NiFiUserUtils.getNiFiUser();
+                    final SnippetAuthorizable snippet = authorizeSnippetUsage(lookup, groupId, requestCopySnippetEntity.getSnippetId(), false);
+
+                    // flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda
+                    final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false);
+                    final Consumer<ConfigurableComponentAuthorizable> authorizeRestricted = authorizable -> {
+                        if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) {
+                            lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
+                        }
+                    };
+
+                    // consider each processor. note - this request will not create new controller services so we do not need to check
+                    // for if there are not restricted controller services. it will however, need to authorize the user has access
+                    // to any referenced services and this is done within authorizeSnippetUsage above.
+                    snippet.getSelectedProcessors().stream().forEach(authorizeRestricted);
+                    snippet.getSelectedProcessGroups().stream().forEach(processGroup -> {
+                        processGroup.getEncapsulatedProcessors().forEach(authorizeRestricted);
+                    });
                 },
                 null,
                 copySnippetRequestEntity -> {
@@ -1736,7 +1758,8 @@ public class ProcessGroupResource extends ApplicationResource {
             response = FlowEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /process-groups/{uuid}", type = ""),
-                    @Authorization(value = "Read - /templates/{uuid}", type = "")
+                    @Authorization(value = "Read - /templates/{uuid}", type = ""),
+                    @Authorization(value = "Write - if the template contains any restricted components - /restricted-components", type = "")
             }
     )
     @ApiResponses(
@@ -1773,11 +1796,27 @@ public class ProcessGroupResource extends ApplicationResource {
                 serviceFacade,
                 requestInstantiateTemplateRequestEntity,
                 lookup -> {
+                    final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+                    // ensure write on the group
                     final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
-                    processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                    processGroup.authorize(authorizer, RequestAction.WRITE, user);
 
-                    final Authorizable template = lookup.getTemplate(requestInstantiateTemplateRequestEntity.getTemplateId());
-                    template.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+                    // ensure read on the template
+                    final TemplateAuthorizable template = lookup.getTemplate(requestInstantiateTemplateRequestEntity.getTemplateId());
+                    template.getAuthorizable().authorize(authorizer, RequestAction.READ, user);
+
+                    // flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda
+                    final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false);
+                    final Consumer<ConfigurableComponentAuthorizable> authorizeRestricted = authorizable -> {
+                        if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) {
+                            lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
+                        }
+                    };
+
+                    // ensure restricted access if necessary
+                    template.getEncapsulatedProcessors().forEach(authorizeRestricted);
+                    template.getEncapsulatedControllerServices().forEach(authorizeRestricted);
                 },
                 null,
                 instantiateTemplateRequestEntity -> {
@@ -1805,13 +1844,16 @@ public class ProcessGroupResource extends ApplicationResource {
     // templates
     // ---------
 
-    private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId, final boolean authorizeTransitiveServices) {
+    private SnippetAuthorizable authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId, final boolean authorizeTransitiveServices) {
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
         // ensure write access to the target process group
-        lookup.getProcessGroup(groupId).getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+        lookup.getProcessGroup(groupId).getAuthorizable().authorize(authorizer, RequestAction.WRITE, user);
 
         // ensure read permission to every component in the snippet including referenced services
-        final Snippet snippet = lookup.getSnippet(snippetId);
+        final SnippetAuthorizable snippet = lookup.getSnippet(snippetId);
         authorizeSnippet(snippet, authorizer, lookup, RequestAction.READ, true, authorizeTransitiveServices);
+        return snippet;
     }
 
     /**
@@ -2075,7 +2117,8 @@ public class ProcessGroupResource extends ApplicationResource {
             response = ControllerServiceEntity.class,
             authorizations = {
                     @Authorization(value = "Write - /process-groups/{uuid}", type = ""),
-                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = "")
+                    @Authorization(value = "Read - any referenced Controller Services - /controller-services/{uuid}", type = ""),
+                    @Authorization(value = "Write - if the Controller Service is restricted - /restricted-components", type = "")
             }
     )
     @ApiResponses(

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/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 271febe..f5fc624 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
@@ -26,9 +26,9 @@ 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.SnippetAuthorizable;
 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;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.SnippetDTO;
@@ -106,8 +106,8 @@ public class SnippetResource extends ApplicationResource {
     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());
 
+        // note - we are not authorizing templates or controller services as they are not considered when using this snippet
         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);
@@ -116,8 +116,9 @@ public class SnippetResource extends ApplicationResource {
         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.getConnections().keySet().stream().map(id -> lookup.getConnection(id).getAuthorizable()).forEach(authorize);
         snippetRequest.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).forEach(authorize);
+        snippetRequest.getLabels().keySet().stream().map(id -> lookup.getLabel(id)).forEach(authorize);
     }
 
     /**
@@ -162,6 +163,10 @@ public class SnippetResource extends ApplicationResource {
             throw new IllegalArgumentException("Snippet ID cannot be specified.");
         }
 
+        if (requestSnippetEntity.getSnippet().getParentGroupId() == null) {
+            throw new IllegalArgumentException("The parent Process Group of the snippet must be specified.");
+        }
+
         if (isReplicateRequest()) {
             return replicate(HttpMethod.POST, requestSnippetEntity);
         }
@@ -268,7 +273,7 @@ public class SnippetResource extends ApplicationResource {
                     }
 
                     // ensure write permission to every component in the snippet excluding referenced services
-                    final Snippet snippet = lookup.getSnippet(snippetId);
+                    final SnippetAuthorizable snippet = lookup.getSnippet(snippetId);
                     authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE, false, false);
                 },
                 () -> serviceFacade.verifyUpdateSnippet(requestSnippetDTO, requestRevisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet())),
@@ -331,7 +336,7 @@ public class SnippetResource extends ApplicationResource {
                 requestRevisions,
                 lookup -> {
                     // ensure write permission to every component in the snippet excluding referenced services
-                    final Snippet snippet = lookup.getSnippet(snippetId);
+                    final SnippetAuthorizable snippet = lookup.getSnippet(snippetId);
                     authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE, true, false);
                 },
                 () -> serviceFacade.verifyDeleteSnippet(snippetId, requestRevisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet())),

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/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 f210792..63a37a8 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
@@ -122,7 +122,7 @@ public class TemplateResource extends ApplicationResource {
 
         // authorize access
         serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable template = lookup.getTemplate(id);
+            final Authorizable template = lookup.getTemplate(id).getAuthorizable();
             template.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
         });
 
@@ -196,7 +196,7 @@ public class TemplateResource extends ApplicationResource {
                 serviceFacade,
                 requestTemplateEntity,
                 lookup -> {
-                    final Authorizable template = lookup.getTemplate(id);
+                    final Authorizable template = lookup.getTemplate(id).getAuthorizable();
                     template.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
                 },
                 null,

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITProcessorAccessControl.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITProcessorAccessControl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITProcessorAccessControl.java
index 02817fc..cf7226c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITProcessorAccessControl.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITProcessorAccessControl.java
@@ -21,12 +21,21 @@ import org.apache.nifi.integration.util.NiFiTestAuthorizer;
 import org.apache.nifi.integration.util.NiFiTestUser;
 import org.apache.nifi.integration.util.RestrictedProcessor;
 import org.apache.nifi.integration.util.SourceTestProcessor;
+import org.apache.nifi.util.Tuple;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.SnippetDTO;
 import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.entity.CopySnippetRequestEntity;
+import org.apache.nifi.web.api.entity.CreateTemplateRequestEntity;
+import org.apache.nifi.web.api.entity.FlowEntity;
+import org.apache.nifi.web.api.entity.InstantiateTemplateRequestEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
 import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.SnippetEntity;
+import org.apache.nifi.web.api.entity.TemplateEntity;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -435,6 +444,187 @@ public class ITProcessorAccessControl {
 
         // ensure the request is successful
         assertEquals(201, response.getStatus());
+
+        final ProcessorEntity responseEntity = response.getEntity(ProcessorEntity.class);
+
+        // remove the restricted component
+        deleteRestrictedComponent(responseEntity);
+    }
+
+    /**
+     * Tests attempting to copy/paste a restricted processor.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testCopyPasteRestrictedProcessor() throws Exception {
+        final String copyUrl = helper.getBaseUrl() + "/process-groups/root/snippet-instance";
+        final Tuple<ProcessorEntity, SnippetEntity> tuple = createSnippetWithRestrictedComponent();
+        final SnippetEntity snippetEntity = tuple.getValue();
+
+        // build the copy/paste request
+        final CopySnippetRequestEntity copyRequest = new CopySnippetRequestEntity();
+        copyRequest.setSnippetId(snippetEntity.getSnippet().getId());
+        copyRequest.setOriginX(0.0);
+        copyRequest.setOriginY(0.0);
+
+        // create the snippet
+        ClientResponse response = helper.getReadWriteUser().testPost(copyUrl, copyRequest);
+
+        // ensure the request failed... need privileged users since snippet comprised of the restricted components
+        assertEquals(403, response.getStatus());
+
+        // create the snippet
+        response = helper.getPrivilegedUser().testPost(copyUrl, copyRequest);
+
+        // ensure the request is successful
+        assertEquals(201, response.getStatus());
+
+        final FlowEntity flowEntity = response.getEntity(FlowEntity.class);
+
+        // remove the restricted processors
+        deleteRestrictedComponent(tuple.getKey());
+        deleteRestrictedComponent(flowEntity.getFlow().getProcessors().stream().findFirst().orElse(null));
+    }
+
+    /**
+     * Tests attempting to use a template with a restricted processor.
+     *
+     * @throws Exception ex
+     */
+    @Test
+    public void testTemplateWithRestrictedProcessor() throws Exception {
+        final String createTemplateUrl = helper.getBaseUrl() + "/process-groups/root/templates";
+        final String instantiateTemplateUrl = helper.getBaseUrl() + "/process-groups/root/template-instance";
+        final Tuple<ProcessorEntity, SnippetEntity> tuple = createSnippetWithRestrictedComponent();
+        final SnippetEntity snippetEntity = tuple.getValue();
+
+        // create the template
+        final CreateTemplateRequestEntity createTemplateRequest = new CreateTemplateRequestEntity();
+        createTemplateRequest.setSnippetId(snippetEntity.getSnippet().getId());
+        createTemplateRequest.setName("test");
+
+        // create the snippet
+        ClientResponse response = helper.getWriteUser().testPost(createTemplateUrl, createTemplateRequest);
+
+        // ensure the request failed... need read perms to the components in the snippet
+        assertEquals(403, response.getStatus());
+
+        response = helper.getReadWriteUser().testPost(createTemplateUrl, createTemplateRequest);
+
+        // ensure the request is successfull
+        assertEquals(201, response.getStatus());
+
+        final TemplateEntity templateEntity = response.getEntity(TemplateEntity.class);
+
+        // build the template request
+        final InstantiateTemplateRequestEntity instantiateTemplateRequest = new InstantiateTemplateRequestEntity();
+        instantiateTemplateRequest.setTemplateId(templateEntity.getTemplate().getId());
+        instantiateTemplateRequest.setOriginX(0.0);
+        instantiateTemplateRequest.setOriginY(0.0);
+
+        // create the snippet
+        response = helper.getReadWriteUser().testPost(instantiateTemplateUrl, instantiateTemplateRequest);
+
+        // ensure the request failed... need privileged user since the template is comprised of restricted components
+        assertEquals(403, response.getStatus());
+
+        // create the snippet
+        response = helper.getPrivilegedUser().testPost(instantiateTemplateUrl, instantiateTemplateRequest);
+
+        // ensure the request is successful
+        assertEquals(201, response.getStatus());
+
+        final FlowEntity flowEntity = response.getEntity(FlowEntity.class);
+
+        // clean up the resources created during this test
+        deleteTemplate(templateEntity);
+        deleteRestrictedComponent(tuple.getKey());
+        deleteRestrictedComponent(flowEntity.getFlow().getProcessors().stream().findFirst().orElse(null));
+    }
+
+    private Tuple<ProcessorEntity, SnippetEntity> createSnippetWithRestrictedComponent() throws Exception {
+        final String processorUrl = helper.getBaseUrl() + "/process-groups/root/processors";
+        final String snippetUrl = helper.getBaseUrl() + "/snippets";
+
+        // create the processor
+        ProcessorDTO processor = new ProcessorDTO();
+        processor.setName("restricted");
+        processor.setType(RestrictedProcessor.class.getName());
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(READ_WRITE_CLIENT_ID);
+        revision.setVersion(0L);
+
+        // create the entity body
+        ProcessorEntity entity = new ProcessorEntity();
+        entity.setRevision(revision);
+        entity.setComponent(processor);
+
+        // perform the request as a user with read/write and restricted access
+        ClientResponse response = helper.getPrivilegedUser().testPost(processorUrl, entity);
+
+        // ensure the request is successful
+        assertEquals(201, response.getStatus());
+
+        // get the response
+        final ProcessorEntity responseProcessorEntity = response.getEntity(ProcessorEntity.class);
+
+        // build the snippet for the copy/paste
+        final SnippetDTO snippet = new SnippetDTO();
+        snippet.setParentGroupId(responseProcessorEntity.getComponent().getParentGroupId());
+        snippet.getProcessors().put(responseProcessorEntity.getId(), responseProcessorEntity.getRevision());
+
+        // create the entity body
+        final SnippetEntity snippetEntity = new SnippetEntity();
+        snippetEntity.setSnippet(snippet);
+
+        // create the snippet
+        response = helper.getNoneUser().testPost(snippetUrl, snippetEntity);
+
+        // ensure the request failed... need either read or write to create snippet (not sure what snippet will be used for)
+        assertEquals(403, response.getStatus());
+
+        // create the snippet
+        response = helper.getReadWriteUser().testPost(snippetUrl, snippetEntity);
+
+        // ensure the request is successful
+        assertEquals(201, response.getStatus());
+
+        // get the response
+        return new Tuple<>(responseProcessorEntity, response.getEntity(SnippetEntity.class));
+    }
+
+    private void deleteTemplate(final TemplateEntity entity) throws Exception {
+        // perform the request
+        ClientResponse response = helper.getReadWriteUser().testDelete(entity.getTemplate().getUri());
+
+        // ensure the request is successful
+        assertEquals(200, response.getStatus());
+    }
+
+    private void deleteRestrictedComponent(final ProcessorEntity entity) throws Exception {
+        if (entity == null) {
+            Assert.fail("Failed to get Processor from template or snippet request.");
+            return;
+        }
+
+        // create the entity body
+        final Map<String, String> queryParams = new HashMap<>();
+        queryParams.put("version", String.valueOf(entity.getRevision().getVersion()));
+        queryParams.put("clientId", READ_WRITE_CLIENT_ID);
+
+        // perform the request
+        ClientResponse response = helper.getReadWriteUser().testDelete(entity.getUri(), queryParams);
+
+        // ensure the request fails... needs access to restricted components
+        assertEquals(403, response.getStatus());
+
+        response = helper.getPrivilegedUser().testDelete(entity.getUri(), queryParams);
+
+        // ensure the request is successful
+        assertEquals(200, response.getStatus());
     }
 
     private ProcessorEntity getRandomProcessor(final NiFiTestUser user) throws Exception {

http://git-wip-us.apache.org/repos/asf/nifi/blob/d8d29811/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
index e5de27f..366dc4f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -13,4 +13,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 org.apache.nifi.integration.util.SourceTestProcessor
-org.apache.nifi.integration.util.TerminationTestProcessor
\ No newline at end of file
+org.apache.nifi.integration.util.TerminationTestProcessor
+org.apache.nifi.integration.util.RestrictedProcessor
\ No newline at end of file