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

[3/4] nifi git commit: NIFI-2307: - Enforcing connection permissions based on the source and destination comonent. - Removing connection specific access policies. NIFI-2265: - Filtering out sensitive details in component status and status history when ap

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/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
new file mode 100644
index 0000000..c0805c0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
@@ -0,0 +1,534 @@
+/*
+ * 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.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.resource.AccessPolicyAuthorizable;
+import org.apache.nifi.authorization.resource.Authorizable;
+import org.apache.nifi.authorization.resource.DataTransferAuthorizable;
+import org.apache.nifi.authorization.resource.ProvenanceEventAuthorizable;
+import org.apache.nifi.authorization.resource.ResourceFactory;
+import org.apache.nifi.authorization.resource.ResourceType;
+import org.apache.nifi.authorization.resource.TenantAuthorizable;
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.ConfiguredComponent;
+import org.apache.nifi.controller.Snippet;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceReference;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.controller.ControllerFacade;
+import org.apache.nifi.web.dao.AccessPolicyDAO;
+import org.apache.nifi.web.dao.ConnectionDAO;
+import org.apache.nifi.web.dao.ControllerServiceDAO;
+import org.apache.nifi.web.dao.FunnelDAO;
+import org.apache.nifi.web.dao.LabelDAO;
+import org.apache.nifi.web.dao.PortDAO;
+import org.apache.nifi.web.dao.ProcessGroupDAO;
+import org.apache.nifi.web.dao.ProcessorDAO;
+import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
+import org.apache.nifi.web.dao.ReportingTaskDAO;
+import org.apache.nifi.web.dao.SnippetDAO;
+import org.apache.nifi.web.dao.TemplateDAO;
+
+
+class StandardAuthorizableLookup implements AuthorizableLookup {
+
+    private static final TenantAuthorizable TENANT_AUTHORIZABLE = new TenantAuthorizable();
+    private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getPoliciesResource();
+        }
+    };
+
+    private static final Authorizable PROVENANCE_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getProvenanceResource();
+        }
+    };
+
+    private static final Authorizable COUNTERS_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getCountersResource();
+        }
+    };
+
+    // nifi core components
+    private ControllerFacade controllerFacade;
+
+    // data access objects
+    private ProcessorDAO processorDAO;
+    private ProcessGroupDAO processGroupDAO;
+    private RemoteProcessGroupDAO remoteProcessGroupDAO;
+    private LabelDAO labelDAO;
+    private FunnelDAO funnelDAO;
+    private SnippetDAO snippetDAO;
+    private PortDAO inputPortDAO;
+    private PortDAO outputPortDAO;
+    private ConnectionDAO connectionDAO;
+    private ControllerServiceDAO controllerServiceDAO;
+    private ReportingTaskDAO reportingTaskDAO;
+    private TemplateDAO templateDAO;
+    private AccessPolicyDAO accessPolicyDAO;
+
+    @Override
+    public Authorizable getController() {
+        return controllerFacade;
+    }
+
+    @Override
+    public Authorizable getProcessor(final String id) {
+        return processorDAO.getProcessor(id);
+    }
+
+    @Override
+    public Authorizable getInputPort(final String id) {
+        return inputPortDAO.getPort(id);
+    }
+
+    @Override
+    public Authorizable getOutputPort(final String id) {
+        return outputPortDAO.getPort(id);
+    }
+
+    @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();
+            }
+        };
+    }
+
+    @Override
+    public Authorizable getProcessGroup(final String id) {
+        return processGroupDAO.getProcessGroup(id);
+    }
+
+    @Override
+    public Authorizable getRemoteProcessGroup(final String id) {
+        return remoteProcessGroupDAO.getRemoteProcessGroup(id);
+    }
+
+    @Override
+    public Authorizable getRemoteProcessGroupInputPort(final String remoteProcessGroupId, final String id) {
+        final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
+        return remoteProcessGroup.getInputPort(id);
+    }
+
+    @Override
+    public Authorizable getRemoteProcessGroupOutputPort(final String remoteProcessGroupId, final String id) {
+        final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
+        return remoteProcessGroup.getOutputPort(id);
+    }
+
+    @Override
+    public Authorizable getLabel(final String id) {
+        return labelDAO.getLabel(id);
+    }
+
+    @Override
+    public Authorizable getFunnel(final String id) {
+        return funnelDAO.getFunnel(id);
+    }
+
+    @Override
+    public Authorizable getControllerService(final String id) {
+        return controllerServiceDAO.getControllerService(id);
+    }
+
+    @Override
+    public Authorizable getProvenance() {
+        return PROVENANCE_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getCounters() {
+        return COUNTERS_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getControllerServiceReferencingComponent(String controllerSeriveId, String id) {
+        final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerSeriveId);
+        final ControllerServiceReference referencingComponents = controllerService.getReferences();
+
+        ConfiguredComponent reference = null;
+        for (final ConfiguredComponent component : referencingComponents.getReferencingComponents()) {
+            if (component.getIdentifier().equals(id)) {
+                reference = component;
+                break;
+            }
+        }
+
+        if (reference == null) {
+            throw new ResourceNotFoundException("Unable to find referencing component with id " + id);
+        }
+
+        return reference;
+    }
+
+    @Override
+    public Authorizable getReportingTask(final String id) {
+        return reportingTaskDAO.getReportingTask(id);
+    }
+
+    @Override
+    public Snippet getSnippet(final String id) {
+        return snippetDAO.getSnippet(id);
+    }
+
+    @Override
+    public Authorizable getTenant() {
+        return TENANT_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getPolicies() {
+        return POLICIES_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getAccessPolicyById(final String id) {
+        final AccessPolicy policy = accessPolicyDAO.getAccessPolicy(id);
+        return getAccessPolicyByResource(policy.getResource());
+    }
+
+    @Override
+    public Authorizable getAccessPolicyByResource(final String resource) {
+        try {
+            return new AccessPolicyAuthorizable(getAuthorizableFromResource(resource));
+        } catch (final ResourceNotFoundException e) {
+            // the underlying component has been removed or resource is invalid... require /policies permissions
+            return POLICIES_AUTHORIZABLE;
+        }
+    }
+
+    @Override
+    public Authorizable getAuthorizableFromResource(String resource) {
+        // parse the resource type
+        ResourceType resourceType = null;
+        for (ResourceType type : ResourceType.values()) {
+            if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
+                resourceType = type;
+            }
+        }
+
+        if (resourceType == null) {
+            throw new ResourceNotFoundException("Unrecognized resource: " + resource);
+        }
+
+        // if this is a policy or a provenance event resource, there should be another resource type
+        if (ResourceType.Policy.equals(resourceType) || ResourceType.ProvenanceEvent.equals(resourceType) || ResourceType.DataTransfer.equals(resourceType)) {
+            final ResourceType primaryResourceType = resourceType;
+
+            // get the resource type
+            resource = StringUtils.substringAfter(resource, resourceType.getValue());
+
+            for (ResourceType type : ResourceType.values()) {
+                if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
+                    resourceType = type;
+                }
+            }
+
+            if (resourceType == null) {
+                throw new ResourceNotFoundException("Unrecognized resource: " + resource);
+            }
+
+            // must either be a policy, event, or data transfer
+            if (ResourceType.Policy.equals(primaryResourceType)) {
+                return new AccessPolicyAuthorizable(getAccessPolicy(resourceType, resource));
+            } else if (ResourceType.ProvenanceEvent.equals(primaryResourceType)) {
+                return new ProvenanceEventAuthorizable(getAccessPolicy(resourceType, resource));
+            } else {
+                return new DataTransferAuthorizable(getAccessPolicy(resourceType, resource));
+            }
+        } else {
+            return getAccessPolicy(resourceType, resource);
+        }
+    }
+
+    private Authorizable getAccessPolicy(final ResourceType resourceType, final String resource) {
+        final String slashComponentId = StringUtils.substringAfter(resource, resourceType.getValue());
+        if (slashComponentId.startsWith("/")) {
+            return getAccessPolicyByResource(resourceType, slashComponentId.substring(1));
+        } else {
+            return getAccessPolicyByResource(resourceType);
+        }
+    }
+
+    private Authorizable getAccessPolicyByResource(final ResourceType resourceType, final String componentId) {
+        Authorizable authorizable = null;
+        switch (resourceType) {
+            case ControllerService:
+                authorizable = getControllerService(componentId);
+                break;
+            case Funnel:
+                authorizable = getFunnel(componentId);
+                break;
+            case InputPort:
+                authorizable = getInputPort(componentId);
+                break;
+            case Label:
+                authorizable = getLabel(componentId);
+                break;
+            case OutputPort:
+                authorizable = getOutputPort(componentId);
+                break;
+            case Processor:
+                authorizable = getProcessor(componentId);
+                break;
+            case ProcessGroup:
+                authorizable = getProcessGroup(componentId);
+                break;
+            case RemoteProcessGroup:
+                authorizable = getRemoteProcessGroup(componentId);
+                break;
+            case ReportingTask:
+                authorizable = getReportingTask(componentId);
+                break;
+            case Template:
+                authorizable = getTemplate(componentId);
+                break;
+            case ProvenanceEvent:
+                authorizable = controllerFacade.getProvenanceEventAuthorizable(componentId);
+                break;
+        }
+
+        if (authorizable == null) {
+            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
+        }
+
+        return authorizable;
+    }
+
+    private Authorizable getAccessPolicyByResource(final ResourceType resourceType) {
+        Authorizable authorizable = null;
+        switch (resourceType) {
+            case Controller:
+                authorizable = getController();
+                break;
+            case Counters:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getCountersResource();
+                    }
+                };
+                break;
+            case Flow:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getFlowResource();
+                    }
+                };
+                break;
+            case Provenance:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getProvenanceResource();
+                    }
+                };
+                break;
+            case Proxy:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getProxyResource();
+                    }
+                };
+                break;
+            case Policy:
+                authorizable = POLICIES_AUTHORIZABLE;
+                break;
+            case Resource:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getResourceResource();
+                    }
+                };
+                break;
+            case SiteToSite:
+                // TODO - new site-to-site and port specific site-to-site
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getSiteToSiteResource();
+                    }
+                };
+                break;
+            case System:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getSystemResource();
+                    }
+                };
+                break;
+            case Tenant:
+                authorizable = getTenant();
+                break;
+        }
+
+        if (authorizable == null) {
+            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
+        }
+
+        return authorizable;
+    }
+
+    @Override
+    public Authorizable getTemplate(final String id) {
+        return templateDAO.getTemplate(id);
+    }
+
+    @Override
+    public Authorizable getConnectable(String id) {
+        final ProcessGroup group = processGroupDAO.getProcessGroup(controllerFacade.getRootGroupId());
+        return group.findConnectable(id);
+    }
+
+    public void setProcessorDAO(ProcessorDAO processorDAO) {
+        this.processorDAO = processorDAO;
+    }
+
+    public void setProcessGroupDAO(ProcessGroupDAO processGroupDAO) {
+        this.processGroupDAO = processGroupDAO;
+    }
+
+    public void setRemoteProcessGroupDAO(RemoteProcessGroupDAO remoteProcessGroupDAO) {
+        this.remoteProcessGroupDAO = remoteProcessGroupDAO;
+    }
+
+    public void setLabelDAO(LabelDAO labelDAO) {
+        this.labelDAO = labelDAO;
+    }
+
+    public void setFunnelDAO(FunnelDAO funnelDAO) {
+        this.funnelDAO = funnelDAO;
+    }
+
+    public void setSnippetDAO(SnippetDAO snippetDAO) {
+        this.snippetDAO = snippetDAO;
+    }
+
+    public void setInputPortDAO(PortDAO inputPortDAO) {
+        this.inputPortDAO = inputPortDAO;
+    }
+
+    public void setOutputPortDAO(PortDAO outputPortDAO) {
+        this.outputPortDAO = outputPortDAO;
+    }
+
+    public void setConnectionDAO(ConnectionDAO connectionDAO) {
+        this.connectionDAO = connectionDAO;
+    }
+
+    public void setControllerServiceDAO(ControllerServiceDAO controllerServiceDAO) {
+        this.controllerServiceDAO = controllerServiceDAO;
+    }
+
+    public void setReportingTaskDAO(ReportingTaskDAO reportingTaskDAO) {
+        this.reportingTaskDAO = reportingTaskDAO;
+    }
+
+    public void setTemplateDAO(TemplateDAO templateDAO) {
+        this.templateDAO = templateDAO;
+    }
+
+    public void setAccessPolicyDAO(AccessPolicyDAO accessPolicyDAO) {
+        this.accessPolicyDAO = accessPolicyDAO;
+    }
+
+    public void setControllerFacade(ControllerFacade controllerFacade) {
+        this.controllerFacade = controllerFacade;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizableLookup.java
deleted file mode 100644
index 7408e30..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizableLookup.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web;
-
-import org.apache.nifi.authorization.resource.Authorizable;
-import org.apache.nifi.controller.Snippet;
-
-public interface AuthorizableLookup {
-
-    /**
-     * Get the authorizable Controller.
-     *
-     * @return authorizable
-     */
-    Authorizable getController();
-
-    /**
-     * Get the authorizable Processor.
-     *
-     * @param id processor id
-     * @return authorizable
-     */
-    Authorizable getProcessor(String id);
-
-    /**
-     * Get the authorizable for querying Provenance.
-     *
-     * @return authorizable
-     */
-    Authorizable getProvenance();
-
-    /**
-     * Get the authorizable for viewing/reseting Counters.
-     *
-     * @return authorizable
-     */
-    Authorizable getCounters();
-
-    /**
-     * Get the authorizable InputPort.
-     *
-     * @param id input port id
-     * @return authorizable
-     */
-    Authorizable getInputPort(String id);
-
-    /**
-     * Get the authorizable OutputPort.
-     *
-     * @param id output port id
-     * @return authorizable
-     */
-    Authorizable getOutputPort(String id);
-
-    /**
-     * Get the authorizable Connection.
-     *
-     * @param id connection id
-     * @return authorizable
-     */
-    Authorizable getConnection(String id);
-
-    /**
-     * Get the authorizable ProcessGroup.
-     *
-     * @param id process group id
-     * @return authorizable
-     */
-    Authorizable getProcessGroup(String id);
-
-    /**
-     * Get the authorizable RemoteProcessGroup.
-     *
-     * @param id remote process group id
-     * @return authorizable
-     */
-    Authorizable getRemoteProcessGroup(String id);
-
-    /**
-     * Get the authorizable RemoteProcessGroup input port.
-     *
-     * @param remoteProcessGroupId remote process group id
-     * @param id input port id
-     * @return authorizable
-     */
-    Authorizable getRemoteProcessGroupInputPort(String remoteProcessGroupId, String id);
-
-    /**
-     * Get the authorizable RemoteProcessGroup output port.
-     *
-     * @param remoteProcessGroupId remote process group id
-     * @param id output port id
-     * @return authorizable
-     */
-    Authorizable getRemoteProcessGroupOutputPort(String remoteProcessGroupId, String id);
-
-    /**
-     * Get the authorizable Label.
-     *
-     * @param id label id
-     * @return authorizable
-     */
-    Authorizable getLabel(String id);
-
-    /**
-     * Get the authorizable Funnel.
-     *
-     * @param id funnel id
-     * @return authorizable
-     */
-    Authorizable getFunnel(String id);
-
-    /**
-     * Get the authorizable ControllerService.
-     *
-     * @param id controller service id
-     * @return authorizable
-     */
-    Authorizable getControllerService(String id);
-
-    /**
-     * Get the authorizable referencing component.
-     *
-     * @param controllerSeriveId controller service id
-     * @param id component id
-     * @return authorizable
-     */
-    Authorizable getControllerServiceReferencingComponent(String controllerSeriveId, String id);
-
-    /**
-     * Get the authorizable ReportingTask.
-     *
-     * @param id reporting task id
-     * @return authorizable
-     */
-    Authorizable getReportingTask(String id);
-
-    /**
-     * Get the authorizable Template.
-     *
-     * @param id template id
-     * @return authorizable
-     */
-    Authorizable getTemplate(String id);
-
-    /**
-     * Get the authorizable connectable.
-     *
-     * @param id connectable id
-     * @return authorizable
-     */
-    Authorizable getConnectable(String id);
-
-    /**
-     * Get the snippet of authorizable's.
-     *
-     * @param id snippet id
-     * @return snippet of authorizable's
-     */
-    Snippet getSnippet(String id);
-
-    /**
-     * Get the {@link Authorizable} that represents the resource of users and user groups.
-     * @return authorizable
-     */
-    Authorizable getTenant();
-
-    /**
-     * Get the authorizable for access all policies.
-     *
-     * @return authorizable
-     */
-    Authorizable getPolicies();
-
-    /**
-     * Get the authorizable for the policy of the policy id.
-     *
-     * @param id id
-     * @return authorizable
-     */
-    Authorizable getAccessPolicyById(String id);
-
-    /**
-     * Get the authorizable for the policy of the specified resource.
-     *
-     * @param resource resource
-     * @return authorizable
-     */
-    Authorizable getAccessPolicyByResource(String resource);
-
-    /**
-     * Get the authorizable of the specified resource.
-     *
-     * @param resource resource
-     * @return authorizable
-     */
-    Authorizable getAuthorizableFromResource(final String resource);
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizeAccess.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizeAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizeAccess.java
deleted file mode 100644
index 29fd523..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizeAccess.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web;
-
-public interface AuthorizeAccess {
-    void authorize(AuthorizableLookup lookup);
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index 28485eb..bba2ff2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.web;
 
+import org.apache.nifi.authorization.AuthorizeAccess;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.controller.ScheduledState;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
index 44a55d8..7d548c9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
@@ -16,11 +16,12 @@
  */
 package org.apache.nifi.web;
 
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.aspectj.lang.ProceedingJoinPoint;
 import org.aspectj.lang.annotation.Around;
 import org.aspectj.lang.annotation.Aspect;
 
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 /**
  * Aspect to limit access into the core.
  */
@@ -143,6 +144,17 @@ public class NiFiServiceFacadeLock {
     }
 
     @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
+            + "execution(* schedule*(..))")
+    public Object scheduleLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
+        writeLock.lock();
+        try {
+            return proceedingJoinPoint.proceed();
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
             + "execution(* get*(..))")
     public Object getLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
         readLock.lock();

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardAuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardAuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardAuthorizableLookup.java
deleted file mode 100644
index 7b9da19..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardAuthorizableLookup.java
+++ /dev/null
@@ -1,515 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.authorization.AccessPolicy;
-import org.apache.nifi.authorization.Resource;
-import org.apache.nifi.authorization.resource.AccessPolicyAuthorizable;
-import org.apache.nifi.authorization.resource.Authorizable;
-import org.apache.nifi.authorization.resource.DataTransferAuthorizable;
-import org.apache.nifi.authorization.resource.ProvenanceEventAuthorizable;
-import org.apache.nifi.authorization.resource.ResourceFactory;
-import org.apache.nifi.authorization.resource.ResourceType;
-import org.apache.nifi.authorization.resource.TenantAuthorizable;
-import org.apache.nifi.controller.ConfiguredComponent;
-import org.apache.nifi.controller.Snippet;
-import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.controller.service.ControllerServiceReference;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.web.controller.ControllerFacade;
-import org.apache.nifi.web.dao.AccessPolicyDAO;
-import org.apache.nifi.web.dao.ConnectionDAO;
-import org.apache.nifi.web.dao.ControllerServiceDAO;
-import org.apache.nifi.web.dao.FunnelDAO;
-import org.apache.nifi.web.dao.LabelDAO;
-import org.apache.nifi.web.dao.PortDAO;
-import org.apache.nifi.web.dao.ProcessGroupDAO;
-import org.apache.nifi.web.dao.ProcessorDAO;
-import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
-import org.apache.nifi.web.dao.ReportingTaskDAO;
-import org.apache.nifi.web.dao.SnippetDAO;
-import org.apache.nifi.web.dao.TemplateDAO;
-
-
-class StandardAuthorizableLookup implements AuthorizableLookup {
-
-    private static final TenantAuthorizable TENANT_AUTHORIZABLE = new TenantAuthorizable();
-    private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getPoliciesResource();
-        }
-    };
-
-    private static final Authorizable PROVENANCE_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getProvenanceResource();
-        }
-    };
-
-    private static final Authorizable COUNTERS_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getCountersResource();
-        }
-    };
-
-    // nifi core components
-    private ControllerFacade controllerFacade;
-
-    // data access objects
-    private ProcessorDAO processorDAO;
-    private ProcessGroupDAO processGroupDAO;
-    private RemoteProcessGroupDAO remoteProcessGroupDAO;
-    private LabelDAO labelDAO;
-    private FunnelDAO funnelDAO;
-    private SnippetDAO snippetDAO;
-    private PortDAO inputPortDAO;
-    private PortDAO outputPortDAO;
-    private ConnectionDAO connectionDAO;
-    private ControllerServiceDAO controllerServiceDAO;
-    private ReportingTaskDAO reportingTaskDAO;
-    private TemplateDAO templateDAO;
-    private AccessPolicyDAO accessPolicyDAO;
-
-    @Override
-    public Authorizable getController() {
-        return controllerFacade;
-    }
-
-    @Override
-    public Authorizable getProcessor(final String id) {
-        return processorDAO.getProcessor(id);
-    }
-
-    @Override
-    public Authorizable getInputPort(final String id) {
-        return inputPortDAO.getPort(id);
-    }
-
-    @Override
-    public Authorizable getOutputPort(final String id) {
-        return outputPortDAO.getPort(id);
-    }
-
-    @Override
-    public Authorizable getConnection(final String id) {
-        return connectionDAO.getConnection(id);
-    }
-
-    @Override
-    public Authorizable getProcessGroup(final String id) {
-        return processGroupDAO.getProcessGroup(id);
-    }
-
-    @Override
-    public Authorizable getRemoteProcessGroup(final String id) {
-        return remoteProcessGroupDAO.getRemoteProcessGroup(id);
-    }
-
-    @Override
-    public Authorizable getRemoteProcessGroupInputPort(final String remoteProcessGroupId, final String id) {
-        final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
-        return remoteProcessGroup.getInputPort(id);
-    }
-
-    @Override
-    public Authorizable getRemoteProcessGroupOutputPort(final String remoteProcessGroupId, final String id) {
-        final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
-        return remoteProcessGroup.getOutputPort(id);
-    }
-
-    @Override
-    public Authorizable getLabel(final String id) {
-        return labelDAO.getLabel(id);
-    }
-
-    @Override
-    public Authorizable getFunnel(final String id) {
-        return funnelDAO.getFunnel(id);
-    }
-
-    @Override
-    public Authorizable getControllerService(final String id) {
-        return controllerServiceDAO.getControllerService(id);
-    }
-
-    @Override
-    public Authorizable getProvenance() {
-        return PROVENANCE_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getCounters() {
-        return COUNTERS_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getControllerServiceReferencingComponent(String controllerSeriveId, String id) {
-        final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerSeriveId);
-        final ControllerServiceReference referencingComponents = controllerService.getReferences();
-
-        ConfiguredComponent reference = null;
-        for (final ConfiguredComponent component : referencingComponents.getReferencingComponents()) {
-            if (component.getIdentifier().equals(id)) {
-                reference = component;
-                break;
-            }
-        }
-
-        if (reference == null) {
-            throw new ResourceNotFoundException("Unable to find referencing component with id " + id);
-        }
-
-        return reference;
-    }
-
-    @Override
-    public Authorizable getReportingTask(final String id) {
-        return reportingTaskDAO.getReportingTask(id);
-    }
-
-    @Override
-    public Snippet getSnippet(final String id) {
-        return snippetDAO.getSnippet(id);
-    }
-
-    @Override
-    public Authorizable getTenant() {
-        return TENANT_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getPolicies() {
-        return POLICIES_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getAccessPolicyById(final String id) {
-        final AccessPolicy policy = accessPolicyDAO.getAccessPolicy(id);
-        return getAccessPolicyByResource(policy.getResource());
-    }
-
-    @Override
-    public Authorizable getAccessPolicyByResource(final String resource) {
-        try {
-            return new AccessPolicyAuthorizable(getAuthorizableFromResource(resource));
-        } catch (final ResourceNotFoundException e) {
-            // the underlying component has been removed or resource is invalid... require /policies permissions
-            return POLICIES_AUTHORIZABLE;
-        }
-    }
-
-    @Override
-    public Authorizable getAuthorizableFromResource(String resource) {
-        // parse the resource type
-        ResourceType resourceType = null;
-        for (ResourceType type : ResourceType.values()) {
-            if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
-                resourceType = type;
-            }
-        }
-
-        if (resourceType == null) {
-            throw new ResourceNotFoundException("Unrecognized resource: " + resource);
-        }
-
-        // if this is a policy or a provenance event resource, there should be another resource type
-        if (ResourceType.Policy.equals(resourceType) || ResourceType.ProvenanceEvent.equals(resourceType) || ResourceType.DataTransfer.equals(resourceType)) {
-            final ResourceType primaryResourceType = resourceType;
-
-            // get the resource type
-            resource = StringUtils.substringAfter(resource, resourceType.getValue());
-
-            for (ResourceType type : ResourceType.values()) {
-                if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
-                    resourceType = type;
-                }
-            }
-
-            if (resourceType == null) {
-                throw new ResourceNotFoundException("Unrecognized resource: " + resource);
-            }
-
-            // must either be a policy, event, or data transfer
-            if (ResourceType.Policy.equals(primaryResourceType)) {
-                return new AccessPolicyAuthorizable(getAccessPolicy(resourceType, resource));
-            } else if (ResourceType.ProvenanceEvent.equals(primaryResourceType)) {
-                return new ProvenanceEventAuthorizable(getAccessPolicy(resourceType, resource));
-            } else {
-                return new DataTransferAuthorizable(getAccessPolicy(resourceType, resource));
-            }
-        } else {
-            return getAccessPolicy(resourceType, resource);
-        }
-    }
-
-    private Authorizable getAccessPolicy(final ResourceType resourceType, final String resource) {
-        final String slashComponentId = StringUtils.substringAfter(resource, resourceType.getValue());
-        if (slashComponentId.startsWith("/")) {
-            return getAccessPolicyByResource(resourceType, slashComponentId.substring(1));
-        } else {
-            return getAccessPolicyByResource(resourceType);
-        }
-    }
-
-    private Authorizable getAccessPolicyByResource(final ResourceType resourceType, final String componentId) {
-        Authorizable authorizable = null;
-        switch (resourceType) {
-            case Connection:
-                authorizable = getConnection(componentId);
-                break;
-            case ControllerService:
-                authorizable = getControllerService(componentId);
-                break;
-            case Funnel:
-                authorizable = getFunnel(componentId);
-                break;
-            case InputPort:
-                authorizable = getInputPort(componentId);
-                break;
-            case Label:
-                authorizable = getLabel(componentId);
-                break;
-            case OutputPort:
-                authorizable = getOutputPort(componentId);
-                break;
-            case Processor:
-                authorizable = getProcessor(componentId);
-                break;
-            case ProcessGroup:
-                authorizable = getProcessGroup(componentId);
-                break;
-            case RemoteProcessGroup:
-                authorizable = getRemoteProcessGroup(componentId);
-                break;
-            case ReportingTask:
-                authorizable = getReportingTask(componentId);
-                break;
-            case Template:
-                authorizable = getTemplate(componentId);
-                break;
-            case ProvenanceEvent:
-                authorizable = controllerFacade.getProvenanceEventAuthorizable(componentId);
-                break;
-        }
-
-        if (authorizable == null) {
-            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
-        }
-
-        return authorizable;
-    }
-
-    private Authorizable getAccessPolicyByResource(final ResourceType resourceType) {
-        Authorizable authorizable = null;
-        switch (resourceType) {
-            case Controller:
-                authorizable = getController();
-                break;
-            case Counters:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getCountersResource();
-                    }
-                };
-                break;
-            case Flow:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getFlowResource();
-                    }
-                };
-                break;
-            case Provenance:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getProvenanceResource();
-                    }
-                };
-                break;
-            case Proxy:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getProxyResource();
-                    }
-                };
-                break;
-            case Policy:
-                authorizable = POLICIES_AUTHORIZABLE;
-                break;
-            case Resource:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getResourceResource();
-                    }
-                };
-                break;
-            case SiteToSite:
-                // TODO - new site-to-site and port specific site-to-site
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getSiteToSiteResource();
-                    }
-                };
-                break;
-            case System:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getSystemResource();
-                    }
-                };
-                break;
-            case Tenant:
-                authorizable = getTenant();
-                break;
-        }
-
-        if (authorizable == null) {
-            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
-        }
-
-        return authorizable;
-    }
-
-    @Override
-    public Authorizable getTemplate(final String id) {
-        return templateDAO.getTemplate(id);
-    }
-
-    @Override
-    public Authorizable getConnectable(String id) {
-        final ProcessGroup group = processGroupDAO.getProcessGroup(controllerFacade.getRootGroupId());
-        return group.findConnectable(id);
-    }
-
-    public void setProcessorDAO(ProcessorDAO processorDAO) {
-        this.processorDAO = processorDAO;
-    }
-
-    public void setProcessGroupDAO(ProcessGroupDAO processGroupDAO) {
-        this.processGroupDAO = processGroupDAO;
-    }
-
-    public void setRemoteProcessGroupDAO(RemoteProcessGroupDAO remoteProcessGroupDAO) {
-        this.remoteProcessGroupDAO = remoteProcessGroupDAO;
-    }
-
-    public void setLabelDAO(LabelDAO labelDAO) {
-        this.labelDAO = labelDAO;
-    }
-
-    public void setFunnelDAO(FunnelDAO funnelDAO) {
-        this.funnelDAO = funnelDAO;
-    }
-
-    public void setSnippetDAO(SnippetDAO snippetDAO) {
-        this.snippetDAO = snippetDAO;
-    }
-
-    public void setInputPortDAO(PortDAO inputPortDAO) {
-        this.inputPortDAO = inputPortDAO;
-    }
-
-    public void setOutputPortDAO(PortDAO outputPortDAO) {
-        this.outputPortDAO = outputPortDAO;
-    }
-
-    public void setConnectionDAO(ConnectionDAO connectionDAO) {
-        this.connectionDAO = connectionDAO;
-    }
-
-    public void setControllerServiceDAO(ControllerServiceDAO controllerServiceDAO) {
-        this.controllerServiceDAO = controllerServiceDAO;
-    }
-
-    public void setReportingTaskDAO(ReportingTaskDAO reportingTaskDAO) {
-        this.reportingTaskDAO = reportingTaskDAO;
-    }
-
-    public void setTemplateDAO(TemplateDAO templateDAO) {
-        this.templateDAO = templateDAO;
-    }
-
-    public void setAccessPolicyDAO(AccessPolicyDAO accessPolicyDAO) {
-        this.accessPolicyDAO = accessPolicyDAO;
-    }
-
-    public void setControllerFacade(ControllerFacade controllerFacade) {
-        this.controllerFacade = controllerFacade;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 40faa46..6186bf1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -23,10 +23,13 @@ import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.details.FlowChangePurgeDetails;
 import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
 import org.apache.nifi.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.AccessPolicy;
+import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
+import org.apache.nifi.authorization.AuthorizeAccess;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.Group;
 import org.apache.nifi.authorization.RequestAction;
@@ -747,6 +750,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                             updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId()));
                         }
 
+                        // save
+                        controllerFacade.save();
+
                         // gather details for response
                         final ScheduleComponentsEntity entity = new ScheduleComponentsEntity();
                         entity.setId(processGroupId);
@@ -903,11 +909,22 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                 .map(g -> g.getIdentifier()).map(mapUserGroupIdToTenantEntity()).collect(Collectors.toSet()) : null;
         final Set<AccessPolicySummaryEntity> policyEntities = user != null ? userGroupDAO.getAccessPoliciesForUser(userId).stream()
                 .map(ap -> createAccessPolicySummaryEntity(ap)).collect(Collectors.toSet()) : null;
-        final UserDTO snapshot = deleteComponent(
-                revision,
-                authorizableLookup.getTenant(),
-                () -> userDAO.deleteUser(userId),
-                dtoFactory.createUserDto(user, userGroups, policyEntities));
+
+        final RevisionClaim claim = new StandardRevisionClaim(revision);
+        final NiFiUser nifiUser = NiFiUserUtils.getNiFiUser();
+
+        // perform the deletion
+        final UserDTO snapshot = revisionManager.deleteRevision(claim, nifiUser, () -> {
+            logger.debug("Attempting to delete component {} with claim {}", user, claim);
+
+            userDAO.deleteUser(userId);
+
+            // save the flow
+            controllerFacade.save();
+            logger.debug("Deletion of component {} was successful", user);
+
+            return dtoFactory.createUserDto(user, userGroups, policyEntities);
+        });
 
         return entityFactory.createUserEntity(snapshot, null, null);
     }
@@ -918,11 +935,22 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         final Set<TenantEntity> users = userGroup != null ? userGroup.getUsers().stream()
                 .map(mapUserIdToTenantEntity()).collect(Collectors.toSet()) :
                 null;
-        final UserGroupDTO snapshot = deleteComponent(
-                revision,
-                authorizableLookup.getTenant(),
-                () -> userGroupDAO.deleteUserGroup(userGroupId),
-                dtoFactory.createUserGroupDto(userGroup, users));
+
+        final RevisionClaim claim = new StandardRevisionClaim(revision);
+        final NiFiUser nifiUser = NiFiUserUtils.getNiFiUser();
+
+        // perform the deletion
+        final UserGroupDTO snapshot = revisionManager.deleteRevision(claim, nifiUser, () -> {
+            logger.debug("Attempting to delete component {} with claim {}", userGroup, claim);
+
+            userGroupDAO.deleteUserGroup(userGroupId);
+
+            // save the flow
+            controllerFacade.save();
+            logger.debug("Deletion of component {} was successful", userGroup);
+
+            return dtoFactory.createUserGroupDto(userGroup, users);
+        });
 
         return entityFactory.createUserGroupEntity(snapshot, null, null);
     }
@@ -976,18 +1004,30 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                 controllerFacade.save();
                 logger.debug("Deletion of component {} was successful", authorizable);
 
-                try {
-                    // since the component is being deleted, also delete any relevant access policies
-                    final AccessPolicy readPolicy = accessPolicyDAO.getAccessPolicy(RequestAction.READ, authorizable);
-                    if (authorizable.getResource().getIdentifier().equals(readPolicy.getResource())) {
-                        accessPolicyDAO.deleteAccessPolicy(readPolicy.getIdentifier());
+                // if configured with a policy based authorizer, attempt to remove the corresponding policies
+                if (authorizer instanceof AbstractPolicyBasedAuthorizer) {
+                    try {
+                        // since the component is being deleted, also delete any relevant read access policies
+                        final AccessPolicy readPolicy = accessPolicyDAO.getAccessPolicy(RequestAction.READ, authorizable);
+                        if (authorizable.getResource().getIdentifier().equals(readPolicy.getResource())) {
+                            accessPolicyDAO.deleteAccessPolicy(readPolicy.getIdentifier());
+                        }
+                    } catch (final ResourceNotFoundException e) {
+                        // no policy exists for this component... no worries
+                    } catch (final Exception e) {
+                        logger.warn(String.format("Unable to remove access policy for %s %s after component removal.", RequestAction.READ, authorizable.getResource().getIdentifier()), e);
                     }
-                    final AccessPolicy writePolicy = accessPolicyDAO.getAccessPolicy(RequestAction.WRITE, authorizable);
-                    if (authorizable.getResource().getIdentifier().equals(writePolicy.getResource())) {
-                        accessPolicyDAO.deleteAccessPolicy(writePolicy.getIdentifier());
+                    try {
+                        // since the component is being deleted, also delete any relevant write access policies
+                        final AccessPolicy writePolicy = accessPolicyDAO.getAccessPolicy(RequestAction.WRITE, authorizable);
+                        if (authorizable.getResource().getIdentifier().equals(writePolicy.getResource())) {
+                            accessPolicyDAO.deleteAccessPolicy(writePolicy.getIdentifier());
+                        }
+                    } catch (final ResourceNotFoundException e) {
+                        // no policy exists for this component... no worries
+                    } catch (final Exception e) {
+                        logger.warn(String.format("Unable to remove access policy for %s %s after component removal.", RequestAction.WRITE, authorizable.getResource().getIdentifier()), e);
                     }
-                } catch (final Exception e) {
-                    logger.warn(String.format("Unable to remove access policy for %s after component removal.", authorizable.getResource().getIdentifier()), e);
                 }
 
                 return dto;
@@ -2660,7 +2700,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                     authorizable = authorizableLookup.getFunnel(sourceId);
                     break;
                 case Connection:
-                    authorizable = authorizableLookup.getConnection(sourceId);
+                    authorizable = authorizableLookup.getConnection(sourceId).getAuthorizable();
                     break;
                 case AccessPolicy:
                     authorizable = authorizableLookup.getAccessPolicyById(sourceId);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/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 4aa08c7..a49ed5d 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
@@ -42,8 +42,8 @@ import org.apache.nifi.remote.protocol.ResponseCode;
 import org.apache.nifi.remote.protocol.http.HttpHeaders;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.TypeOneUUIDGenerator;
-import org.apache.nifi.web.AuthorizableLookup;
-import org.apache.nifi.web.AuthorizeAccess;
+import org.apache.nifi.authorization.AuthorizableLookup;
+import org.apache.nifi.authorization.AuthorizeAccess;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.RevisionDTO;
@@ -420,8 +420,7 @@ public abstract class ApplicationResource {
         snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).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(authorize);
-        snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(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);
     }
 
@@ -440,8 +439,7 @@ public abstract class ApplicationResource {
         snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).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(authorize);
-        snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(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);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.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/ConnectionResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
index 4cbd3af..06bba2a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
@@ -24,12 +24,16 @@ import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.ConnectionAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
+import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
+import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
+import org.apache.nifi.web.api.dto.ListingRequestDTO;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
@@ -88,6 +92,38 @@ public class ConnectionResource extends ApplicationResource {
     }
 
     /**
+     * Populate the URIs for the specified flowfile listing.
+     *
+     * @param connectionId connection
+     * @param flowFileListing flowfile listing
+     * @return dto
+     */
+    public ListingRequestDTO populateRemainingFlowFileListingContent(final String connectionId, final ListingRequestDTO flowFileListing) {
+        // uri of the listing
+        flowFileListing.setUri(generateResourceUri("connections", connectionId, "listing-requests", flowFileListing.getId()));
+
+        // uri of each flowfile
+        if (flowFileListing.getFlowFileSummaries() != null) {
+            for (final FlowFileSummaryDTO flowFile : flowFileListing.getFlowFileSummaries()) {
+                populateRemainingFlowFileContent(connectionId, flowFile);
+            }
+        }
+        return flowFileListing;
+    }
+
+    /**
+     * Populate the URIs for the specified flowfile.
+     *
+     * @param connectionId the connection id
+     * @param flowFile the flowfile
+     * @return the dto
+     */
+    public FlowFileSummaryDTO populateRemainingFlowFileContent(final String connectionId, final FlowFileSummaryDTO flowFile) {
+        flowFile.setUri(generateResourceUri("connections", connectionId, "flowfiles", flowFile.getUuid()));
+        return flowFile;
+    }
+
+    /**
      * Retrieves the specified connection.
      *
      * @param id The id of the connection.
@@ -130,8 +166,9 @@ public class ConnectionResource extends ApplicationResource {
 
         // authorize access
         serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable conn = lookup.getConnection(id);
-            conn.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+            // ensure read access to this connection (checks source and destination)
+            final Authorizable authorizable = lookup.getConnection(id).getAuthorizable();
+            authorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
         });
 
         // get the specified relationship
@@ -200,6 +237,10 @@ public class ConnectionResource extends ApplicationResource {
                     + "requested resource (%s).", connection.getId(), id));
         }
 
+        if (connection.getDestination() != null && connection.getDestination().getId() == null) {
+            throw new IllegalArgumentException("When specifying a destination component, the destination id is required.");
+        }
+
         if (isReplicateRequest()) {
             return replicate(HttpMethod.PUT, connectionEntity);
         }
@@ -209,8 +250,20 @@ public class ConnectionResource extends ApplicationResource {
             serviceFacade,
             revision,
             lookup -> {
-                Authorizable authorizable = lookup.getConnection(id);
-                authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                // verifies write access to this connection (this checks the current source and destination)
+                ConnectionAuthorizable connAuth = lookup.getConnection(id);
+                connAuth.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                // if a destination has been specified and is different
+                final Connectable currentDestination = connAuth.getDestination();
+                if (connection.getDestination() != null && currentDestination.getIdentifier().equals(connection.getDestination().getId())) {
+                    // verify access of the new destination (current destination was already authorized as part of the connection check)
+                    final Authorizable newDestinationAuthorizable = lookup.getConnectable(connection.getDestination().getId());
+                    newDestinationAuthorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                    // verify access of the parent group (this is the same check that is performed when creating the connection)
+                    connAuth.getParentGroup().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                }
             },
             () -> serviceFacade.verifyUpdateConnection(connection),
             () -> {
@@ -284,8 +337,9 @@ public class ConnectionResource extends ApplicationResource {
             serviceFacade,
             revision,
             lookup -> {
-                final Authorizable conn = lookup.getConnection(id);
-                conn.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+                // verifies write access to the source and destination
+                final Authorizable authorizable = lookup.getConnection(id).getAuthorizable();
+                authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
             },
             () -> serviceFacade.verifyDeleteConnection(id),
             () -> {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.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/FlowFileQueueResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java
index 18765d0..cea97da 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java
@@ -16,29 +16,12 @@
  */
 package org.apache.nifi.web.api;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.Response.Status;
-import javax.ws.rs.core.StreamingOutput;
-
+import com.wordnik.swagger.annotations.Api;
+import com.wordnik.swagger.annotations.ApiOperation;
+import com.wordnik.swagger.annotations.ApiParam;
+import com.wordnik.swagger.annotations.ApiResponse;
+import com.wordnik.swagger.annotations.ApiResponses;
+import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
@@ -58,12 +41,27 @@ import org.apache.nifi.web.api.entity.FlowFileEntity;
 import org.apache.nifi.web.api.entity.ListingRequestEntity;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 
-import com.wordnik.swagger.annotations.Api;
-import com.wordnik.swagger.annotations.ApiOperation;
-import com.wordnik.swagger.annotations.ApiParam;
-import com.wordnik.swagger.annotations.ApiResponse;
-import com.wordnik.swagger.annotations.ApiResponses;
-import com.wordnik.swagger.annotations.Authorization;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.core.StreamingOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
 
 /**
  * RESTful endpoint for managing a flowfile queue.
@@ -174,7 +172,7 @@ public class FlowFileQueueResource extends ApplicationResource {
 
         // authorize access
         serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable connection = lookup.getConnection(connectionId);
+            final Authorizable connection = lookup.getConnection(connectionId).getAuthorizable();
             connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
         });
 
@@ -259,7 +257,7 @@ public class FlowFileQueueResource extends ApplicationResource {
 
         // authorize access
         serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable connection = lookup.getConnection(connectionId);
+            final Authorizable connection = lookup.getConnection(connectionId).getAuthorizable();
             connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
         });
 
@@ -338,7 +336,7 @@ public class FlowFileQueueResource extends ApplicationResource {
         if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
-                final Authorizable connection = lookup.getConnection(id);
+                final Authorizable connection = lookup.getConnection(id).getAuthorizable();
                 connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
             });
         }
@@ -409,7 +407,7 @@ public class FlowFileQueueResource extends ApplicationResource {
 
         // authorize access
         serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable connection = lookup.getConnection(connectionId);
+            final Authorizable connection = lookup.getConnection(connectionId).getAuthorizable();
             connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
         });
 
@@ -475,7 +473,7 @@ public class FlowFileQueueResource extends ApplicationResource {
         if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
-                final Authorizable connection = lookup.getConnection(connectionId);
+                final Authorizable connection = lookup.getConnection(connectionId).getAuthorizable();
                 connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
             });
         }
@@ -545,7 +543,7 @@ public class FlowFileQueueResource extends ApplicationResource {
         if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
-                final Authorizable connection = lookup.getConnection(id);
+                final Authorizable connection = lookup.getConnection(id).getAuthorizable();
                 connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
             });
         }
@@ -615,7 +613,7 @@ public class FlowFileQueueResource extends ApplicationResource {
 
         // authorize access
         serviceFacade.authorizeAccess(lookup -> {
-            final Authorizable connection = lookup.getConnection(connectionId);
+            final Authorizable connection = lookup.getConnection(connectionId).getAuthorizable();
             connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
         });
 
@@ -681,7 +679,7 @@ public class FlowFileQueueResource extends ApplicationResource {
         if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
-                final Authorizable connection = lookup.getConnection(connectionId);
+                final Authorizable connection = lookup.getConnection(connectionId).getAuthorizable();
                 connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
             });
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/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 70852c1..1ea3736 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,7 +31,7 @@ import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.controller.Snippet;
-import org.apache.nifi.web.AuthorizableLookup;
+import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
@@ -1533,20 +1533,37 @@ public class ProcessGroupResource extends ApplicationResource {
         }
         connectionEntity.getComponent().setParentGroupId(groupId);
 
+        // get the connection
+        final ConnectionDTO connection = connectionEntity.getComponent();
+
+        if (connection.getSource() == null || connection.getSource().getId() == null) {
+            throw new IllegalArgumentException("The source of the connection must be specified.");
+        }
+
+        if (connection.getDestination() == null || connection.getDestination().getId() == null) {
+            throw new IllegalArgumentException("The destination of the connection must be specified.");
+        }
+
         if (isReplicateRequest()) {
             return replicate(HttpMethod.POST, connectionEntity);
         }
 
-        // get the connection
-        final ConnectionDTO connection = connectionEntity.getComponent();
-
         // handle expects request (usually from the cluster manager)
         final boolean validationPhase = isValidationPhase(httpServletRequest);
         if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
+                // ensure write access to the group
                 final Authorizable processGroup = lookup.getProcessGroup(groupId);
                 processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                // ensure write access to the source
+                final Authorizable source = lookup.getConnectable(connection.getSource().getId());
+                source.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+
+                // ensure write access to the destination
+                final Authorizable destination = lookup.getConnectable(connection.getDestination().getId());
+                destination.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
             });
         }
         if (validationPhase) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 20bb819..07d2143 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -631,9 +631,11 @@ public final class DtoFactory {
             return null;
         }
 
+        boolean isAuthorized = connectable.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+
         final ConnectableDTO dto = new ConnectableDTO();
         dto.setId(connectable.getIdentifier());
-        dto.setName(connectable.getName());
+        dto.setName(isAuthorized ? connectable.getName() : connectable.getIdentifier());
         dto.setType(connectable.getConnectableType().name());
 
         if (connectable instanceof RemoteGroupPort) {
@@ -643,11 +645,15 @@ public final class DtoFactory {
             dto.setRunning(remoteGroupPort.isTargetRunning());
             dto.setTransmitting(remoteGroupPort.isRunning());
             dto.setExists(remoteGroupPort.getTargetExists());
-            dto.setComments(remoteGroup.getComments());
+            if (isAuthorized) {
+                dto.setComments(remoteGroup.getComments());
+            }
         } else {
             dto.setGroupId(connectable.getProcessGroup().getIdentifier());
             dto.setRunning(connectable.isRunning());
-            dto.setComments(connectable.getComments());
+            if (isAuthorized) {
+                dto.setComments(connectable.getComments());
+            }
         }
 
         return dto;

http://git-wip-us.apache.org/repos/asf/nifi/blob/4a4d60e6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
index f9a68ae..e50df8f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
@@ -78,6 +78,7 @@ public final class EntityFactory {
             entity.setPermissions(permissions);
             entity.setStatus(status);
             entity.setId(dto.getId());
+            entity.setInputRequirement(dto.getInputRequirement());
             entity.setPosition(dto.getPosition());
             if (permissions != null && permissions.getCanRead()) {
                 entity.setComponent(dto);
@@ -245,10 +246,13 @@ public final class EntityFactory {
             entity.setPosition(dto.getPosition());
             entity.setBends(dto.getBends());
             entity.setLabelIndex(dto.getLabelIndex());
+            entity.setzIndex(dto.getzIndex());
             entity.setSourceId(dto.getSource().getId());
             entity.setSourceGroupId(dto.getSource().getGroupId());
+            entity.setSourceType(dto.getSource().getType());
             entity.setDestinationId(dto.getDestination().getId());
             entity.setDestinationGroupId(dto.getDestination().getGroupId());
+            entity.setDestinationType(dto.getDestination().getType());
             if (permissions != null && permissions.getCanRead()) {
                 entity.setComponent(dto);
             }