From 4a4d60e6af5014342039f751c7b058079d643f7b Mon Sep 17 00:00:00 2001 From: Matt Gilman Date: Thu, 21 Jul 2016 22:53:00 -0400 Subject: [PATCH] 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 appropriate. NIFI-1800: - Adding parent process group id to the Controller Services table. NIFI-2077: - Removing some old un-used icons following the UI refresh. NIFI-2242: - Requiring write permissions for all components in a selection. NIFI-2080: - Updating style of the name in the selection context to handle scroll bars and use available width. NIFI-2331: - Addressing issue when removing a user/group which was causing the tenant policy to be removed. NIFI-2335: - Ensuring the flow is saved after starting/stopping a process group. NIFI-2235: - Ensuring we use consistent conditions between the context menu and the operate palette. - Allowing users with read only access to the tenants page. - Fixing current user integration test. - Ensuring schedule methods are locked appropriately. - Addressing comments from PR. This closes #698 Signed-off-by: jpercivall --- .../authorization/resource/Authorizable.java | 4 +- .../history/ComponentStatusRepository.java | 12 +- .../status/RemoteProcessGroupStatusDTO.java | 21 +- .../RemoteProcessGroupStatusSnapshotDTO.java | 20 +- .../nifi/web/api/entity/ConnectionEntity.java | 49 +++++ .../nifi/web/api/entity/ProcessorEntity.java | 16 ++ .../resource/ResourceFactory.java | 45 ---- .../authorization/resource/ResourceType.java | 3 +- .../endpoints/GroupStatusEndpointMerger.java | 16 -- .../nifi/cluster/manager/StatusMerger.java | 9 - .../nifi/connectable/StandardConnection.java | 55 ++++- .../nifi/controller/FlowController.java | 195 ++++++++++++------ .../VolatileComponentStatusRepository.java | 34 +-- .../AuthorizableLookup.java | 4 +- .../AuthorizeAccess.java | 2 +- .../authorization/ConnectionAuthorizable.java | 54 +++++ .../StandardAuthorizableLookup.java | 35 +++- .../apache/nifi/web/NiFiServiceFacade.java | 1 + .../nifi/web/NiFiServiceFacadeLock.java | 14 +- .../nifi/web/StandardNiFiServiceFacade.java | 82 ++++++-- .../nifi/web/api/ApplicationResource.java | 10 +- .../nifi/web/api/ConnectionResource.java | 66 +++++- .../nifi/web/api/FlowFileQueueResource.java | 72 ++++--- .../nifi/web/api/ProcessGroupResource.java | 25 ++- .../apache/nifi/web/api/dto/DtoFactory.java | 12 +- .../nifi/web/api/dto/EntityFactory.java | 4 + .../nifi/web/controller/ControllerFacade.java | 60 ++++-- .../web/dao/impl/StandardConnectionDAO.java | 16 +- .../main/resources/nifi-web-api-context.xml | 2 +- .../accesscontrol/ITFlowAccessControl.java | 2 +- .../WEB-INF/partials/canvas/canvas-header.jsp | 2 +- .../canvas/drop-request-status-dialog.jsp | 2 +- .../WEB-INF/partials/canvas/navigation.jsp | 14 +- .../WEB-INF/partials/users/users-content.jsp | 2 +- .../nifi-web-ui/src/main/webapp/css/graph.css | 2 +- .../src/main/webapp/css/navigation.css | 4 +- .../src/main/webapp/css/policy-management.css | 2 +- .../src/main/webapp/images/bgButton.png | Bin 234 -> 0 bytes .../src/main/webapp/images/bgButtonOver.png | Bin 234 -> 0 bytes .../main/webapp/images/bgButtonSelected.png | Bin 166 -> 0 bytes .../webapp/images/bgButtonSelectedOver.png | Bin 168 -> 0 bytes .../src/main/webapp/images/bgInputText.png | Bin 139 -> 0 bytes .../src/main/webapp/images/bgShellClose.png | Bin 169 -> 0 bytes .../src/main/webapp/images/bgTabContainer.png | Bin 234 -> 0 bytes .../src/main/webapp/images/bgTableHeader.png | Bin 232 -> 0 bytes .../src/main/webapp/images/buttonRefresh.png | Bin 915 -> 0 bytes .../main/webapp/images/iconAlertDialog.png | Bin 1241 -> 0 bytes .../main/webapp/images/iconAutoRefresh.png | Bin 3102 -> 0 bytes .../src/main/webapp/images/iconBulletin.png | Bin 1066 -> 0 bytes .../src/main/webapp/images/iconCenterView.png | Bin 338 -> 0 bytes .../src/main/webapp/images/iconChart.png | Bin 510 -> 0 bytes .../main/webapp/images/iconClusterSmall.png | Bin 757 -> 0 bytes .../src/main/webapp/images/iconColor.png | Bin 738 -> 0 bytes .../src/main/webapp/images/iconConfigure.png | Bin 696 -> 0 bytes .../src/main/webapp/images/iconConnect.png | Bin 589 -> 0 bytes .../src/main/webapp/images/iconCopy.png | Bin 514 -> 0 bytes .../src/main/webapp/images/iconDisable.png | Bin 764 -> 0 bytes .../src/main/webapp/images/iconDisconnect.png | Bin 569 -> 0 bytes .../src/main/webapp/images/iconEdit.png | Bin 493 -> 0 bytes .../src/main/webapp/images/iconEditButton.png | Bin 915 -> 0 bytes .../src/main/webapp/images/iconEmptyQueue.png | Bin 785 -> 0 bytes .../src/main/webapp/images/iconEnable.png | Bin 472 -> 0 bytes .../src/main/webapp/images/iconExport.png | Bin 453 -> 0 bytes .../src/main/webapp/images/iconFunnel.png | Bin 1223 -> 0 bytes .../src/main/webapp/images/iconGoTo.png | Bin 448 -> 0 bytes .../main/webapp/images/iconInputPortSmall.png | Bin 532 -> 0 bytes .../webapp/images/iconIsolatedProcessor.png | Bin 1781 -> 0 bytes .../src/main/webapp/images/iconLineage.png | Bin 432 -> 0 bytes .../src/main/webapp/images/iconListQueue.png | Bin 1502 -> 0 bytes .../main/webapp/images/iconMoveToParent.png | Bin 215 -> 0 bytes .../src/main/webapp/images/iconNotSecure.png | Bin 137 -> 0 bytes .../webapp/images/iconOutputPortSmall.png | Bin 459 -> 0 bytes .../src/main/webapp/images/iconPaste.png | Bin 601 -> 0 bytes .../webapp/images/iconPortNotTransmitting.png | Bin 231 -> 0 bytes .../webapp/images/iconPortTransmitting.png | Bin 235 -> 0 bytes .../src/main/webapp/images/iconPrimary.png | Bin 647 -> 0 bytes .../src/main/webapp/images/iconProvenance.png | Bin 1104 -> 0 bytes .../src/main/webapp/images/iconRefresh.png | Bin 492 -> 0 bytes .../main/webapp/images/iconRemotePorts.png | Bin 456 -> 0 bytes .../main/webapp/images/iconResetCounter.png | Bin 304 -> 0 bytes .../src/main/webapp/images/iconRevoke.png | Bin 676 -> 0 bytes .../src/main/webapp/images/iconSecure.png | Bin 133 -> 0 bytes .../main/webapp/images/iconSmallProcessor.png | Bin 647 -> 0 bytes .../webapp/images/iconSmallRelationship.png | Bin 770 -> 0 bytes .../src/main/webapp/images/iconStop.png | Bin 402 -> 0 bytes .../src/main/webapp/images/iconToFront.png | Bin 475 -> 0 bytes .../webapp/images/iconTransmissionActive.png | Bin 1330 -> 0 bytes .../images/iconTransmissionInactive.png | Bin 1248 -> 0 bytes .../src/main/webapp/images/iconUsage.png | Bin 470 -> 0 bytes .../src/main/webapp/images/iconViewState.png | Bin 1245 -> 0 bytes .../src/main/webapp/images/loadAnimation.gif | Bin 10789 -> 0 bytes .../src/main/webapp/images/panelBg.jpg | Bin 312 -> 0 bytes .../src/main/webapp/images/starburst.png | Bin 190 -> 0 bytes .../src/main/webapp/images/tabBg.jpg | Bin 320 -> 0 bytes .../images/transmissionSwitchDisabled.png | Bin 1240 -> 0 bytes .../images/transmissionSwitchEnabled.png | Bin 1600 -> 0 bytes .../src/main/webapp/images/ungroup.png | Bin 3409 -> 0 bytes .../nf-ng-canvas-global-menu-controller.js | 2 +- .../nf-ng-canvas-graph-controls-controller.js | 50 +++++ .../main/webapp/js/nf/canvas/nf-actions.js | 26 +-- .../webapp/js/nf/canvas/nf-canvas-utils.js | 184 +++++++++++++---- .../nf/canvas/nf-connection-configuration.js | 70 ++++--- .../main/webapp/js/nf/canvas/nf-connection.js | 111 ++++------ .../webapp/js/nf/canvas/nf-context-menu.js | 67 +----- .../js/nf/canvas/nf-controller-services.js | 27 ++- .../main/webapp/js/nf/canvas/nf-draggable.js | 31 ++- .../js/nf/canvas/nf-policy-management.js | 4 +- .../webapp/js/nf/canvas/nf-queue-listing.js | 5 +- .../webapp/js/nf/nf-connection-details.js | 183 +++++++++++----- .../main/webapp/js/nf/users/nf-users-table.js | 24 ++- .../src/main/webapp/js/nf/users/nf-users.js | 13 +- 111 files changed, 1149 insertions(+), 616 deletions(-) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/{web => authorization}/AuthorizableLookup.java (98%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/{web => authorization}/AuthorizeAccess.java (95%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConnectionAuthorizable.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/{web => authorization}/StandardAuthorizableLookup.java (95%) delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgButton.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgButtonOver.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgButtonSelected.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgButtonSelectedOver.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgInputText.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgShellClose.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgTabContainer.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/bgTableHeader.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/buttonRefresh.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconAlertDialog.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconAutoRefresh.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconBulletin.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconCenterView.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconChart.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconClusterSmall.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconColor.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconConfigure.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconConnect.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconCopy.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconDisable.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconDisconnect.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconEdit.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconEditButton.png delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconEmptyQueue.png delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconEnable.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconExport.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconFunnel.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconGoTo.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconInputPortSmall.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconIsolatedProcessor.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconLineage.png delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconListQueue.png delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconMoveToParent.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconNotSecure.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconOutputPortSmall.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconPaste.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconPortNotTransmitting.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconPortTransmitting.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconPrimary.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconProvenance.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconRefresh.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconRemotePorts.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconResetCounter.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconRevoke.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconSecure.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconSmallProcessor.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconSmallRelationship.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconStop.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconToFront.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconTransmissionActive.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconTransmissionInactive.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconUsage.png delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconViewState.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/loadAnimation.gif delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/panelBg.jpg delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/starburst.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/tabBg.jpg delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/transmissionSwitchDisabled.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/transmissionSwitchEnabled.png delete mode 100755 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/ungroup.png diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/resource/Authorizable.java b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/resource/Authorizable.java index 7cb21ae8eb..09829a9fcd 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/resource/Authorizable.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/resource/Authorizable.java @@ -97,7 +97,7 @@ public interface Authorizable { if (parent == null) { return AuthorizationResult.denied(); } else { - return parent.checkAuthorization(authorizer, action, user); + return parent.checkAuthorization(authorizer, action, user, resourceContext); } } else { return result; @@ -152,7 +152,7 @@ public interface Authorizable { if (parent == null) { throw new AccessDeniedException("Access is denied"); } else { - parent.authorize(authorizer, action, user); + parent.authorize(authorizer, action, user, resourceContext); } } else if (Result.Denied.equals(result.getResult())) { throw new AccessDeniedException(result.getExplanation()); diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java index d273096008..1042c3fbdf 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java @@ -16,16 +16,24 @@ */ package org.apache.nifi.controller.status.history; -import java.util.Date; - import org.apache.nifi.controller.status.ProcessGroupStatus; +import java.util.Date; + /** * A repository for storing and retrieving components' historical status * information */ public interface ComponentStatusRepository { + String COMPONENT_DETAIL_ID = "Id"; + String COMPONENT_DETAIL_GROUP_ID = "Group Id"; + String COMPONENT_DETAIL_NAME = "Name"; + String COMPONENT_DETAIL_TYPE = "Type"; + String COMPONENT_DETAIL_SOURCE_NAME = "Source Name"; + String COMPONENT_DETAIL_DESTINATION_NAME = "Destination Name"; + String COMPONENT_DETAIL_URI = "Uri"; + /** * Captures the status information provided in the given report * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java index 62685b1d9d..faae6e67b7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java @@ -17,14 +17,13 @@ package org.apache.nifi.web.api.dto.status; -import java.util.Date; -import java.util.List; +import com.wordnik.swagger.annotations.ApiModelProperty; +import org.apache.nifi.web.api.dto.util.TimeAdapter; import javax.xml.bind.annotation.XmlType; import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; - -import com.wordnik.swagger.annotations.ApiModelProperty; -import org.apache.nifi.web.api.dto.util.TimeAdapter; +import java.util.Date; +import java.util.List; @XmlType(name = "remoteProcessGroupStatus") public class RemoteProcessGroupStatusDTO { @@ -35,8 +34,6 @@ public class RemoteProcessGroupStatusDTO { private String transmissionStatus; private Date statsLastRefreshed; - private List authorizationIssues; - private RemoteProcessGroupStatusSnapshotDTO aggregateSnapshot; private List nodeSnapshots; @@ -76,16 +73,6 @@ public class RemoteProcessGroupStatusDTO { this.transmissionStatus = transmissionStatus; } - - @ApiModelProperty("Any remote authorization issues for the remote process group.") - public List getAuthorizationIssues() { - return authorizationIssues; - } - - public void setAuthorizationIssues(List authorizationIssues) { - this.authorizationIssues = authorizationIssues; - } - @ApiModelProperty("The URI of the target system.") public String getTargetUri() { return targetUri; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusSnapshotDTO.java index f0f0d7ebf6..85e3a242f5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusSnapshotDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusSnapshotDTO.java @@ -16,13 +16,10 @@ */ package org.apache.nifi.web.api.dto.status; -import java.util.ArrayList; -import java.util.List; +import com.wordnik.swagger.annotations.ApiModelProperty; import javax.xml.bind.annotation.XmlType; -import com.wordnik.swagger.annotations.ApiModelProperty; - /** * The status of a remote process group in this NiFi. */ @@ -36,8 +33,6 @@ public class RemoteProcessGroupStatusSnapshotDTO implements Cloneable { private String transmissionStatus; private Integer activeThreadCount; - private List authorizationIssues; - private Integer flowFilesSent = 0; private Long bytesSent = 0L; private String sent; @@ -118,18 +113,6 @@ public class RemoteProcessGroupStatusSnapshotDTO implements Cloneable { this.activeThreadCount = activeThreadCount; } - /** - * @return any remote authorization issues for this remote process group - */ - @ApiModelProperty("Any remote authorization issues for the remote process group.") - public List getAuthorizationIssues() { - return authorizationIssues; - } - - public void setAuthorizationIssues(List authorizationIssues) { - this.authorizationIssues = authorizationIssues; - } - /** * @return Formatted description of the amount of data sent to this remote process group */ @@ -201,7 +184,6 @@ public class RemoteProcessGroupStatusSnapshotDTO implements Cloneable { other.setTargetUri(getTargetUri()); other.setTransmissionStatus(getTransmissionStatus()); other.setActiveThreadCount(getActiveThreadCount()); - other.setAuthorizationIssues(getAuthorizationIssues() == null ? null : new ArrayList(getAuthorizationIssues())); other.setFlowFilesSent(getFlowFilesSent()); other.setBytesSent(getBytesSent()); other.setFlowFilesReceived(getFlowFilesReceived()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java index 1b48ca404f..86aa1ee892 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java @@ -34,10 +34,13 @@ public class ConnectionEntity extends ComponentEntity { private ConnectionStatusDTO status; private List bends; private Integer labelIndex; + private Long zIndex; private String sourceId; private String sourceGroupId; + private String sourceType; private String destinationId; private String destinationGroupId; + private String destinationType; /** * @return RelationshipDTO that is being serialized @@ -92,6 +95,20 @@ public class ConnectionEntity extends ComponentEntity { this.labelIndex = labelIndex; } + /** + * @return z index for this connection + */ + @ApiModelProperty( + value = "The z index of the connection." + ) + public Long getzIndex() { + return zIndex; + } + + public void setzIndex(Long zIndex) { + this.zIndex = zIndex; + } + /** * @return The identifier of the source of this connection */ @@ -134,6 +151,22 @@ public class ConnectionEntity extends ComponentEntity { this.sourceGroupId = sourceGroupId; } + /** + * @return type of this source connectable component + */ + @ApiModelProperty( + value = "The type of component the source connectable is.", + required = true, + allowableValues = "PROCESSOR, REMOTE_INPUT_PORT, REMOTE_OUTPUT_PORT, INPUT_PORT, OUTPUT_PORT, FUNNEL" + ) + public String getSourceType() { + return sourceType; + } + + public void setSourceType(String sourceType) { + this.sourceType = sourceType; + } + /** * @return The identifier of the group of the destination of this connection */ @@ -147,4 +180,20 @@ public class ConnectionEntity extends ComponentEntity { public void setDestinationGroupId(String destinationGroupId) { this.destinationGroupId = destinationGroupId; } + + /** + * @return type of this destination connectable component + */ + @ApiModelProperty( + value = "The type of component the destination connectable is.", + required = true, + allowableValues = "PROCESSOR, REMOTE_INPUT_PORT, REMOTE_OUTPUT_PORT, INPUT_PORT, OUTPUT_PORT, FUNNEL" + ) + public String getDestinationType() { + return destinationType; + } + + public void setDestinationType(String destinationType) { + this.destinationType = destinationType; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java index 89b784ece6..cda843cf6d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.web.api.entity; +import com.wordnik.swagger.annotations.ApiModelProperty; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO; @@ -28,6 +29,7 @@ import javax.xml.bind.annotation.XmlRootElement; public class ProcessorEntity extends ComponentEntity { private ProcessorDTO component; + private String inputRequirement; private ProcessorStatusDTO status; /** @@ -55,4 +57,18 @@ public class ProcessorEntity extends ComponentEntity { public void setStatus(ProcessorStatusDTO status) { this.status = status; } + + /** + * @return the input requirement of this processor + */ + @ApiModelProperty( + value = "The input requirement for this processor." + ) + public String getInputRequirement() { + return inputRequirement; + } + + public void setInputRequirement(String inputRequirement) { + this.inputRequirement = inputRequirement; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java index 004429acda..baa58ac57e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java @@ -22,18 +22,6 @@ import java.util.Objects; public final class ResourceFactory { - private final static Resource CONNECTION_RESOURCE = new Resource() { - @Override - public String getIdentifier() { - return ResourceType.Connection.getValue(); - } - - @Override - public String getName() { - return "Connection"; - } - }; - private final static Resource CONTROLLER_RESOURCE = new Resource() { @Override public String getIdentifier() { @@ -299,15 +287,6 @@ public final class ResourceFactory { } }; - /** - * Gets the Resource for accessing Connections. - * - * @return The resource for accessing connections - */ - public static Resource getConnectionResource() { - return CONNECTION_RESOURCE; - } - /** * Gets the Resource for accessing the Controller. This includes Controller level configuration, bulletins, reporting tasks, and the cluster. * @@ -602,30 +581,6 @@ public final class ResourceFactory { }; } - /** - * Gets a Resource fo accessing a flowfile queue for the specified connection. - * - * @param connectionIdentifier The identifier of the connection - * @param connectionName The name of the connection - * @return The resource - */ - public static Resource getFlowFileQueueResource(final String connectionIdentifier, final String connectionName) { - Objects.requireNonNull(connectionIdentifier, "The connection identifier must be specified."); - Objects.requireNonNull(connectionName, "The connection name must be specified."); - - return new Resource() { - @Override - public String getIdentifier() { - return String.format("/flowfile-queue/%s", connectionIdentifier); - } - - @Override - public String getName() { - return connectionName + " queue"; - } - }; - } - /** * Prevent outside instantiation. */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java index 177b9a4a29..bce270ad68 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java @@ -17,11 +17,10 @@ package org.apache.nifi.authorization.resource; public enum ResourceType { - Connection("/connections"), Controller("/controller"), ControllerService("/controller-services"), Counters("/counters"), - Funnel("/funnel"), + Funnel("/funnels"), Flow("/flow"), InputPort("/input-ports"), Label("/labels"), diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/GroupStatusEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/GroupStatusEndpointMerger.java index 9440760ebd..88239028ae 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/GroupStatusEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/GroupStatusEndpointMerger.java @@ -21,14 +21,10 @@ import org.apache.nifi.cluster.manager.StatusMerger; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; -import org.apache.nifi.web.api.dto.status.ProcessGroupStatusSnapshotDTO; -import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusSnapshotDTO; import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; import java.net.URI; import java.util.ArrayList; -import java.util.List; -import java.util.ListIterator; import java.util.Map; import java.util.regex.Pattern; @@ -70,18 +66,6 @@ public class GroupStatusEndpointMerger extends AbstractNodeStatusEndpoint nodeAuthorizationIssues = remoteProcessGroupStatus.getAuthorizationIssues(); - if (nodeAuthorizationIssues != null && !nodeAuthorizationIssues.isEmpty()) { - for (final ListIterator iter = nodeAuthorizationIssues.listIterator(); iter.hasNext();) { - final String Issue = iter.next(); - iter.set("[" + nodeId.getApiAddress() + ":" + nodeId.getApiPort() + "] -- " + Issue); - } - remoteProcessGroupStatus.setAuthorizationIssues(nodeAuthorizationIssues); - } - } - StatusMerger.merge(mergedProcessGroupStatus, nodeProcessGroupStatus, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java index 04acbdaaa3..c886747052 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java @@ -378,15 +378,6 @@ public class StatusMerger { target.setActiveThreadCount(target.getActiveThreadCount() + toMerge.getActiveThreadCount()); - final List authIssues = new ArrayList<>(); - if (target.getAuthorizationIssues() != null) { - authIssues.addAll(target.getAuthorizationIssues()); - } - if (toMerge.getAuthorizationIssues() != null) { - authIssues.addAll(toMerge.getAuthorizationIssues()); - } - target.setAuthorizationIssues(authIssues); - target.setFlowFilesSent(target.getFlowFilesSent() + toMerge.getFlowFilesSent()); target.setBytesSent(target.getBytesSent() + toMerge.getBytesSent()); target.setFlowFilesReceived(target.getFlowFilesReceived() + toMerge.getFlowFilesReceived()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java index 0843743817..3745d7d97e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java @@ -20,10 +20,14 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.AuthorizationResult.Result; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.Resource; import org.apache.nifi.authorization.resource.Authorizable; -import org.apache.nifi.authorization.resource.ResourceFactory; -import org.apache.nifi.authorization.resource.ResourceType; +import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.StandardFlowFileQueue; import org.apache.nifi.controller.queue.FlowFileQueue; @@ -42,6 +46,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -103,23 +108,51 @@ public final class StandardConnection implements Connection { @Override public Authorizable getParentAuthorizable() { - return getSource(); + return null; } @Override public Resource getResource() { - String name = getName(); + return new Resource() { + @Override + public String getIdentifier() { + return "/connections/" + StandardConnection.this.getIdentifier(); + } - final Collection relationships = getRelationships(); - if (name == null && CollectionUtils.isNotEmpty(relationships)) { - name = StringUtils.join(relationships.stream().map(relationship -> relationship.getName()).collect(Collectors.toSet()), ", "); + @Override + public String getName() { + String name = StandardConnection.this.getName(); + + final Collection relationships = getRelationships(); + if (name == null && CollectionUtils.isNotEmpty(relationships)) { + name = StringUtils.join(relationships.stream().map(relationship -> relationship.getName()).collect(Collectors.toSet()), ", "); + } + + if (name == null) { + name = "Connection"; + } + + return name; + } + }; + } + + @Override + public AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map resourceContext) { + // check the source + final AuthorizationResult sourceResult = getSource().checkAuthorization(authorizer, action, user, resourceContext); + if (Result.Denied.equals(sourceResult.getResult())) { + return sourceResult; } - if (name == null) { - name = "Connection"; - } + // check the destination + return getDestination().checkAuthorization(authorizer, action, user, resourceContext); + } - return ResourceFactory.getComponentResource(ResourceType.Connection, getIdentifier(), name); + @Override + public void authorize(Authorizer authorizer, RequestAction action, NiFiUser user, Map resourceContext) throws AccessDeniedException { + getSource().authorize(authorizer, action, user, resourceContext); + getDestination().authorize(authorizer, action, user, resourceContext); } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 4670605a54..abe5e27883 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -16,39 +16,8 @@ */ package org.apache.nifi.controller; -import static java.util.Objects.requireNonNull; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.text.DateFormat; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.LockSupport; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -import javax.net.ssl.SSLContext; - +import com.sun.jersey.api.client.ClientHandlerException; +import org.apache.commons.collections4.Predicate; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.action.Action; import org.apache.nifi.admin.service.AuditService; @@ -59,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange; import org.apache.nifi.annotation.notification.PrimaryNodeState; import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.Resource; import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.ProvenanceEventAuthorizable; @@ -183,10 +153,10 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessorInitializationContext; import org.apache.nifi.processor.StandardValidationContextFactory; -import org.apache.nifi.provenance.ProvenanceRepository; +import org.apache.nifi.provenance.ProvenanceAuthorizableFactory; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventType; -import org.apache.nifi.provenance.ProvenanceAuthorizableFactory; +import org.apache.nifi.provenance.ProvenanceRepository; import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.apache.nifi.remote.HttpRemoteSiteListener; import org.apache.nifi.remote.RemoteGroupPort; @@ -236,7 +206,37 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.sun.jersey.api.client.ClientHandlerException; +import javax.net.ssl.SSLContext; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import static java.util.Objects.requireNonNull; public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider { @@ -2128,28 +2128,85 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R return root == null ? null : root.findProcessGroup(searchId); } + /** + * Returns the status of all components in the controller. This request is not in the context of a user so the results will be unfiltered. + * + * @return the component status + */ @Override public ProcessGroupStatus getControllerStatus() { return getGroupStatus(getRootGroupId()); } + /** + * Returns the status of all components in the specified group. This request is not in the context of a user so the results will be unfiltered. + * + * @param groupId group id + * @return the component status + */ public ProcessGroupStatus getGroupStatus(final String groupId) { return getGroupStatus(groupId, getProcessorStats()); } - public ProcessGroupStatus getGroupStatus(final String groupId, final RepositoryStatusReport statusReport) { - final ProcessGroup group = getGroup(groupId); - return getGroupStatus(group, statusReport); + /** + * Returns the status for components in the specified group. This request is made by the specified user so the results will be filtered accordingly. + * + * @param groupId group id + * @param user user making request + * @return the component status + */ + public ProcessGroupStatus getGroupStatus(final String groupId, final NiFiUser user) { + return getGroupStatus(groupId, getProcessorStats(), user); } - public ProcessGroupStatus getGroupStatus(final ProcessGroup group, final RepositoryStatusReport statusReport) { + /** + * Returns the status for the components in the specified group with the specified report. This request is not in the context of a user so the results + * will be unfiltered. + * + * @param groupId group id + * @param statusReport report + * @return the component status + */ + public ProcessGroupStatus getGroupStatus(final String groupId, final RepositoryStatusReport statusReport) { + final ProcessGroup group = getGroup(groupId); + + // this was invoked with no user context so the results will be unfiltered... necessary for aggregating status history + return getGroupStatus(group, statusReport, authorizable -> true); + } + + /** + * Returns the status for the components in the specified group with the specified report. This request is made by the specified user + * so the results will be filtered accordingly. + * + * @param groupId group id + * @param statusReport report + * @param user user making request + * @return the component status + */ + public ProcessGroupStatus getGroupStatus(final String groupId, final RepositoryStatusReport statusReport, final NiFiUser user) { + final ProcessGroup group = getGroup(groupId); + + // on demand status request for a specific user... require authorization per component and filter results as appropriate + return getGroupStatus(group, statusReport, authorizable -> authorizable.isAuthorized(authorizer, RequestAction.READ, user)); + } + + /** + * Returns the status for the components in the specified group with the specified report. The results will be filtered by executing + * the specified predicate. + * + * @param group group id + * @param statusReport report + * @param isAuthorized is authorized check + * @return the component status + */ + public ProcessGroupStatus getGroupStatus(final ProcessGroup group, final RepositoryStatusReport statusReport, final Predicate isAuthorized) { if (group == null) { return null; } final ProcessGroupStatus status = new ProcessGroupStatus(); status.setId(group.getIdentifier()); - status.setName(group.getName()); + status.setName(isAuthorized.evaluate(group) ? group.getName() : group.getIdentifier()); int activeGroupThreads = 0; long bytesRead = 0L; long bytesWritten = 0L; @@ -2170,7 +2227,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final Collection processorStatusCollection = new ArrayList<>(); status.setProcessorStatus(processorStatusCollection); for (final ProcessorNode procNode : group.getProcessors()) { - final ProcessorStatus procStat = getProcessorStatus(statusReport, procNode); + final ProcessorStatus procStat = getProcessorStatus(statusReport, procNode, isAuthorized); processorStatusCollection.add(procStat); activeGroupThreads += procStat.getActiveThreadCount(); bytesRead += procStat.getBytesRead(); @@ -2186,7 +2243,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final Collection localChildGroupStatusCollection = new ArrayList<>(); status.setProcessGroupStatus(localChildGroupStatusCollection); for (final ProcessGroup childGroup : group.getProcessGroups()) { - final ProcessGroupStatus childGroupStatus = getGroupStatus(childGroup, statusReport); + final ProcessGroupStatus childGroupStatus = getGroupStatus(childGroup, statusReport, isAuthorized); localChildGroupStatusCollection.add(childGroupStatus); activeGroupThreads += childGroupStatus.getActiveThreadCount(); bytesRead += childGroupStatus.getBytesRead(); @@ -2207,7 +2264,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final Collection remoteProcessGroupStatusCollection = new ArrayList<>(); status.setRemoteProcessGroupStatus(remoteProcessGroupStatusCollection); for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) { - final RemoteProcessGroupStatus remoteStatus = createRemoteGroupStatus(remoteGroup, statusReport); + final RemoteProcessGroupStatus remoteStatus = createRemoteGroupStatus(remoteGroup, statusReport, isAuthorized); if (remoteStatus != null) { remoteProcessGroupStatusCollection.add(remoteStatus); @@ -2224,13 +2281,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R // get the connection and remote port status for (final Connection conn : group.getConnections()) { + final boolean isConnectionAuthorized = isAuthorized.evaluate(conn); + final boolean isSourceAuthorized = isAuthorized.evaluate(conn.getSource()); + final boolean isDestinationAuthorized = isAuthorized.evaluate(conn.getDestination()); + final ConnectionStatus connStatus = new ConnectionStatus(); connStatus.setId(conn.getIdentifier()); connStatus.setGroupId(conn.getProcessGroup().getIdentifier()); connStatus.setSourceId(conn.getSource().getIdentifier()); - connStatus.setSourceName(conn.getSource().getName()); + connStatus.setSourceName(isSourceAuthorized ? conn.getSource().getName() : conn.getSource().getIdentifier()); connStatus.setDestinationId(conn.getDestination().getIdentifier()); - connStatus.setDestinationName(conn.getDestination().getName()); + connStatus.setDestinationName(isDestinationAuthorized ? conn.getDestination().getName() : conn.getDestination().getIdentifier()); connStatus.setBackPressureDataSizeThreshold(conn.getFlowFileQueue().getBackPressureDataSizeThreshold()); connStatus.setBackPressureObjectThreshold(conn.getFlowFileQueue().getBackPressureObjectThreshold()); @@ -2245,14 +2306,18 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R bytesTransferred += connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut(); } - if (StringUtils.isNotBlank(conn.getName())) { - connStatus.setName(conn.getName()); - } else if (conn.getRelationships() != null && !conn.getRelationships().isEmpty()) { - final Collection relationships = new ArrayList<>(conn.getRelationships().size()); - for (final Relationship relationship : conn.getRelationships()) { - relationships.add(relationship.getName()); + if (isConnectionAuthorized) { + if (StringUtils.isNotBlank(conn.getName())) { + connStatus.setName(conn.getName()); + } else if (conn.getRelationships() != null && !conn.getRelationships().isEmpty()) { + final Collection relationships = new ArrayList<>(conn.getRelationships().size()); + for (final Relationship relationship : conn.getRelationships()) { + relationships.add(relationship.getName()); + } + connStatus.setName(StringUtils.join(relationships, ", ")); } - connStatus.setName(StringUtils.join(relationships, ", ")); + } else { + connStatus.setName(conn.getIdentifier()); } final QueueSize queueSize = conn.getFlowFileQueue().size(); @@ -2285,10 +2350,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final Set inputPorts = group.getInputPorts(); for (final Port port : inputPorts) { + final boolean isInputPortAuthorized = isAuthorized.evaluate(port); + final PortStatus portStatus = new PortStatus(); portStatus.setId(port.getIdentifier()); portStatus.setGroupId(port.getProcessGroup().getIdentifier()); - portStatus.setName(port.getName()); + portStatus.setName(isInputPortAuthorized ? port.getName() : port.getIdentifier()); portStatus.setActiveThreadCount(processScheduler.getActiveThreadCount(port)); // determine the run status @@ -2343,10 +2410,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final Set outputPorts = group.getOutputPorts(); for (final Port port : outputPorts) { + final boolean isOutputPortAuthorized = isAuthorized.evaluate(port); + final PortStatus portStatus = new PortStatus(); portStatus.setId(port.getIdentifier()); portStatus.setGroupId(port.getProcessGroup().getIdentifier()); - portStatus.setName(port.getName()); + portStatus.setName(isOutputPortAuthorized ? port.getName() : port.getIdentifier()); portStatus.setActiveThreadCount(processScheduler.getActiveThreadCount(port)); // determine the run status @@ -2419,7 +2488,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R return status; } - private RemoteProcessGroupStatus createRemoteGroupStatus(final RemoteProcessGroup remoteGroup, final RepositoryStatusReport statusReport) { + private RemoteProcessGroupStatus createRemoteGroupStatus(final RemoteProcessGroup remoteGroup, final RepositoryStatusReport statusReport, final Predicate isAuthorized) { + final boolean isRemoteProcessGroupAuthorized = isAuthorized.evaluate(remoteGroup); + int receivedCount = 0; long receivedContentSize = 0L; int sentCount = 0; @@ -2430,8 +2501,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final RemoteProcessGroupStatus status = new RemoteProcessGroupStatus(); status.setGroupId(remoteGroup.getProcessGroup().getIdentifier()); - status.setName(remoteGroup.getName()); - status.setTargetUri(remoteGroup.getTargetUri().toString()); + status.setName(isRemoteProcessGroupAuthorized ? remoteGroup.getName() : remoteGroup.getIdentifier()); + status.setTargetUri(isRemoteProcessGroupAuthorized ? remoteGroup.getTargetUri().toString() : null); long lineageMillis = 0L; int flowFilesRemoved = 0; @@ -2499,12 +2570,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R return status; } - private ProcessorStatus getProcessorStatus(final RepositoryStatusReport report, final ProcessorNode procNode) { + private ProcessorStatus getProcessorStatus(final RepositoryStatusReport report, final ProcessorNode procNode, final Predicate isAuthorized) { + final boolean isProcessorAuthorized = isAuthorized.evaluate(procNode); + final ProcessorStatus status = new ProcessorStatus(); status.setId(procNode.getIdentifier()); status.setGroupId(procNode.getProcessGroup().getIdentifier()); - status.setName(procNode.getName()); - status.setType(procNode.getComponentType()); + status.setName(isProcessorAuthorized ? procNode.getName() : procNode.getIdentifier()); + status.setType(isProcessorAuthorized ? procNode.getComponentType() : "Processor"); final FlowFileEvent entry = report.getReportEntries().get(procNode.getIdentifier()); if (entry == null) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java index bbd9210a0b..828bdfec10 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java @@ -16,8 +16,6 @@ */ package org.apache.nifi.controller.status.history; -import java.util.Date; - import org.apache.nifi.controller.status.ConnectionStatus; import org.apache.nifi.controller.status.ProcessGroupStatus; import org.apache.nifi.controller.status.ProcessorStatus; @@ -30,6 +28,8 @@ import org.apache.nifi.util.RingBuffer.ForEachEvaluator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Date; + public class VolatileComponentStatusRepository implements ComponentStatusRepository { public static final String NUM_DATA_POINTS_PROPERTY = "nifi.components.status.repository.buffer.size"; @@ -69,7 +69,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit @Override public StatusHistory getProcessorStatusHistory(final String processorId, final Date start, final Date end, final int preferredDataPoints) { final StandardStatusHistory history = new StandardStatusHistory(); - history.setComponentDetail("Id", processorId); + history.setComponentDetail(COMPONENT_DETAIL_ID, processorId); captures.forEach(new ForEachEvaluator() { @Override @@ -80,9 +80,9 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit return true; } - history.setComponentDetail("Group Id", status.getGroupId()); - history.setComponentDetail("Name", status.getName()); - history.setComponentDetail("Type", status.getType()); + history.setComponentDetail(COMPONENT_DETAIL_GROUP_ID, status.getGroupId()); + history.setComponentDetail(COMPONENT_DETAIL_NAME, status.getName()); + history.setComponentDetail(COMPONENT_DETAIL_TYPE, status.getType()); final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(); snapshot.setTimestamp(capture.getCaptureDate()); @@ -102,7 +102,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit @Override public StatusHistory getConnectionStatusHistory(final String connectionId, final Date start, final Date end, final int preferredDataPoints) { final StandardStatusHistory history = new StandardStatusHistory(); - history.setComponentDetail("Id", connectionId); + history.setComponentDetail(COMPONENT_DETAIL_ID, connectionId); captures.forEach(new ForEachEvaluator() { @Override @@ -113,10 +113,10 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit return true; } - history.setComponentDetail("Group Id", status.getGroupId()); - history.setComponentDetail("Name", status.getName()); - history.setComponentDetail("Source Name", status.getSourceName()); - history.setComponentDetail("Destination Name", status.getDestinationName()); + history.setComponentDetail(COMPONENT_DETAIL_GROUP_ID, status.getGroupId()); + history.setComponentDetail(COMPONENT_DETAIL_NAME, status.getName()); + history.setComponentDetail(COMPONENT_DETAIL_SOURCE_NAME, status.getSourceName()); + history.setComponentDetail(COMPONENT_DETAIL_DESTINATION_NAME, status.getDestinationName()); final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(); snapshot.setTimestamp(capture.getCaptureDate()); @@ -136,7 +136,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit @Override public StatusHistory getProcessGroupStatusHistory(final String processGroupId, final Date start, final Date end, final int preferredDataPoints) { final StandardStatusHistory history = new StandardStatusHistory(); - history.setComponentDetail("Id", processGroupId); + history.setComponentDetail(COMPONENT_DETAIL_ID, processGroupId); captures.forEach(new ForEachEvaluator() { @Override @@ -147,7 +147,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit return true; } - history.setComponentDetail("Name", status.getName()); + history.setComponentDetail(COMPONENT_DETAIL_NAME, status.getName()); final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(); snapshot.setTimestamp(capture.getCaptureDate()); @@ -167,7 +167,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit @Override public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroupId, final Date start, final Date end, final int preferredDataPoints) { final StandardStatusHistory history = new StandardStatusHistory(); - history.setComponentDetail("Id", remoteGroupId); + history.setComponentDetail(COMPONENT_DETAIL_ID, remoteGroupId); captures.forEach(new ForEachEvaluator() { @Override @@ -178,9 +178,9 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit return true; } - history.setComponentDetail("Group Id", status.getGroupId()); - history.setComponentDetail("Name", status.getName()); - history.setComponentDetail("Uri", status.getTargetUri()); + history.setComponentDetail(COMPONENT_DETAIL_GROUP_ID, status.getGroupId()); + history.setComponentDetail(COMPONENT_DETAIL_NAME, status.getName()); + history.setComponentDetail(COMPONENT_DETAIL_URI, status.getTargetUri()); final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(); snapshot.setTimestamp(capture.getCaptureDate()); 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/authorization/AuthorizableLookup.java similarity index 98% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizableLookup.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java index 7408e30dad..e0faafbb49 100644 --- 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/authorization/AuthorizableLookup.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.web; +package org.apache.nifi.authorization; import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.controller.Snippet; @@ -72,7 +72,7 @@ public interface AuthorizableLookup { * @param id connection id * @return authorizable */ - Authorizable getConnection(String id); + ConnectionAuthorizable getConnection(String id); /** * Get the authorizable ProcessGroup. 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/authorization/AuthorizeAccess.java similarity index 95% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/AuthorizeAccess.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeAccess.java index 29fd523d0c..4e6661aafa 100644 --- 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/authorization/AuthorizeAccess.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.web; +package org.apache.nifi.authorization; public interface AuthorizeAccess { void authorize(AuthorizableLookup lookup); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConnectionAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConnectionAuthorizable.java new file mode 100644 index 0000000000..76a88331fe --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConnectionAuthorizable.java @@ -0,0 +1,54 @@ +/* + * 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 org.apache.nifi.connectable.Connectable; +import org.apache.nifi.groups.ProcessGroup; + +/** + * Authorizable for a Connection and its Group, Source, and Destination. + */ +public interface ConnectionAuthorizable { + /** + * Returns the authorizable for this connection. Non null + * + * @return authorizable + */ + Authorizable getAuthorizable(); + + /** + * Returns the source. + * + * @return source + */ + Connectable getSource(); + + /** + * Returns the destination. + * + * @return destination + */ + Connectable getDestination(); + + /** + * Returns the parent process group. + * + * @return parent + */ + ProcessGroup getParentGroup(); +} 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/authorization/StandardAuthorizableLookup.java similarity index 95% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardAuthorizableLookup.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java index 7b9da19790..c0805c06cc 100644 --- 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/authorization/StandardAuthorizableLookup.java @@ -14,11 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.web; +package org.apache.nifi.authorization; 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; @@ -26,12 +24,15 @@ 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; @@ -125,8 +126,29 @@ class StandardAuthorizableLookup implements AuthorizableLookup { } @Override - public Authorizable getConnection(final String id) { - return connectionDAO.getConnection(id); + 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 @@ -288,9 +310,6 @@ class StandardAuthorizableLookup implements AuthorizableLookup { 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; 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 28485eb094..bba2ff2e41 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; 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 44a55d873d..7d548c9d1d 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. */ @@ -142,6 +143,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 { 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 40faa46234..6186bf1ef2 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 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 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); 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 4aa08c7510..a49ed5d251 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); } 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 4cbd3affdb..06bba2a02a 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; @@ -87,6 +91,38 @@ public class ConnectionResource extends ApplicationResource { return connectionEntity; } + /** + * 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. * @@ -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), () -> { 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 18765d015b..cea97da9c5 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()); }); } 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 70852c1b47..1ea373658a 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) { 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 20bb81963f..07d21431b6 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; 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 f9a68ae8b8..e50df8fbd3 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); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index 4161657a94..8299181571 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -57,6 +57,7 @@ import org.apache.nifi.controller.status.PortStatus; import org.apache.nifi.controller.status.ProcessGroupStatus; import org.apache.nifi.controller.status.ProcessorStatus; import org.apache.nifi.controller.status.RemoteProcessGroupStatus; +import org.apache.nifi.controller.status.history.ComponentStatusRepository; import org.apache.nifi.diagnostics.SystemDiagnostics; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -268,7 +269,15 @@ public class ControllerFacade implements Authorizable { throw new ResourceNotFoundException(String.format("Unable to locate processor with id '%s'.", processorId)); } - return flowController.getProcessorStatusHistory(processorId); + final StatusHistoryDTO statusHistory = flowController.getProcessorStatusHistory(processorId); + + // if not authorized + if (!processor.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) { + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, processorId); + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_TYPE, "Processor"); + } + + return statusHistory; } /** @@ -286,7 +295,16 @@ public class ControllerFacade implements Authorizable { throw new ResourceNotFoundException(String.format("Unable to locate connection with id '%s'.", connectionId)); } - return flowController.getConnectionStatusHistory(connectionId); + final StatusHistoryDTO statusHistory = flowController.getConnectionStatusHistory(connectionId); + + // if not authorized + if (!connection.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) { + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, connectionId); + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_SOURCE_NAME, connection.getSource().getIdentifier()); + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_DESTINATION_NAME, connection.getDestination().getIdentifier()); + } + + return statusHistory; } /** @@ -305,7 +323,14 @@ public class ControllerFacade implements Authorizable { throw new ResourceNotFoundException(String.format("Unable to locate process group with id '%s'.", groupId)); } - return flowController.getProcessGroupStatusHistory(groupId); + final StatusHistoryDTO statusHistory = flowController.getProcessGroupStatusHistory(groupId); + + // if not authorized + if (!group.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) { + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, groupId); + } + + return statusHistory; } /** @@ -323,7 +348,15 @@ public class ControllerFacade implements Authorizable { throw new ResourceNotFoundException(String.format("Unable to locate remote process group with id '%s'.", remoteProcessGroupId)); } - return flowController.getRemoteProcessGroupStatusHistory(remoteProcessGroupId); + final StatusHistoryDTO statusHistory = flowController.getRemoteProcessGroupStatusHistory(remoteProcessGroupId); + + // if not authorized + if (!remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) { + statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, remoteProcessGroupId); + statusHistory.getComponentDetails().remove(ComponentStatusRepository.COMPONENT_DETAIL_URI); + } + + return statusHistory; } /** @@ -512,10 +545,11 @@ public class ControllerFacade implements Authorizable { * @return the status for the specified process group */ public ProcessGroupStatus getProcessGroupStatus(final String groupId) { - final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId); + final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId, NiFiUserUtils.getNiFiUser()); if (processGroupStatus == null) { throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } + return processGroupStatus; } @@ -536,7 +570,7 @@ public class ControllerFacade implements Authorizable { // calculate the process group status final String groupId = processor.getProcessGroup().getIdentifier(); - final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId); + final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId, NiFiUserUtils.getNiFiUser()); if (processGroupStatus == null) { throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } @@ -566,7 +600,7 @@ public class ControllerFacade implements Authorizable { // calculate the process group status final String groupId = connection.getProcessGroup().getIdentifier(); - final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId); + final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId, NiFiUserUtils.getNiFiUser()); if (processGroupStatus == null) { throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } @@ -595,7 +629,7 @@ public class ControllerFacade implements Authorizable { } final String groupId = port.getProcessGroup().getIdentifier(); - final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId); + final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId, NiFiUserUtils.getNiFiUser()); if (processGroupStatus == null) { throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } @@ -624,7 +658,7 @@ public class ControllerFacade implements Authorizable { } final String groupId = port.getProcessGroup().getIdentifier(); - final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId); + final ProcessGroupStatus processGroupStatus = flowController.getGroupStatus(groupId, NiFiUserUtils.getNiFiUser()); if (processGroupStatus == null) { throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } @@ -653,7 +687,7 @@ public class ControllerFacade implements Authorizable { } final String groupId = remoteProcessGroup.getProcessGroup().getIdentifier(); - final ProcessGroupStatus groupStatus = flowController.getGroupStatus(groupId); + final ProcessGroupStatus groupStatus = flowController.getGroupStatus(groupId, NiFiUserUtils.getNiFiUser()); if (groupStatus == null) { throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } @@ -736,12 +770,6 @@ public class ControllerFacade implements Authorizable { resources.add(ResourceFactory.getProvenanceEventResource(processor.getResource())); } - // add each connection - for (final Connection connection : root.findAllConnections()) { - resources.add(ResourceFactory.getComponentResource(ResourceType.Connection, connection.getIdentifier(), connection.getName())); - resources.add(ResourceFactory.getFlowFileQueueResource(connection.getIdentifier(), connection.getName())); - } - // add each label for (final Label label : root.findAllLabels()) { resources.add(ResourceFactory.getComponentResource(ResourceType.Label, label.getIdentifier(), label.getValue())); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java index 1ec171d206..a337854751 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java @@ -18,7 +18,6 @@ package org.apache.nifi.web.dao.impl; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.authorization.AccessDeniedException; -import org.apache.nifi.authorization.AuthorizationRequest; import org.apache.nifi.authorization.AuthorizationResult; import org.apache.nifi.authorization.AuthorizationResult.Result; import org.apache.nifi.authorization.Authorizer; @@ -26,6 +25,7 @@ import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.UserContextKeys; import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.authorization.user.NiFiUserUtils; +import org.apache.nifi.authorization.user.StandardNiFiUser; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -611,18 +611,8 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO userContext = null; } - final AuthorizationRequest request = new AuthorizationRequest.Builder() - .identity(identity) - .anonymous(user.isAnonymous()) - .accessAttempt(false) - .action(RequestAction.WRITE) - .resource(connection.getResource()) - .resourceContext(attributes) - .userContext(userContext) - .build(); - - // perform the authorization - final AuthorizationResult result = authorizer.authorize(request); + final NiFiUser chainUser = new StandardNiFiUser(identity, user.getClientAddress()); + final AuthorizationResult result = connection.checkAuthorization(authorizer, RequestAction.WRITE, chainUser, attributes); if (!Result.Approved.equals(result.getResult())) { throw new AccessDeniedException(result.getExplanation()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml index 6e8de79d82..1f6508481a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml @@ -119,7 +119,7 @@ - + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITFlowAccessControl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITFlowAccessControl.java index 9202c42fbc..9ebbc75444 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITFlowAccessControl.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITFlowAccessControl.java @@ -66,7 +66,7 @@ public class ITFlowAccessControl { */ @Test public void testGetIdentity() throws Exception { - helper.testGenericGetUri(helper.getBaseUrl() + "/flow/identity"); + helper.testGenericGetUri(helper.getBaseUrl() + "/flow/current-user"); } /** diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp index d4d7aedf34..7fc08a00b6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp @@ -151,7 +151,7 @@ + ng-class="{disabled: !appCtrl.nf.Common.canAccessTenants()}"> Users diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/drop-request-status-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/drop-request-status-dialog.jsp index 9dd35ba728..f2c71d45c3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/drop-request-status-dialog.jsp +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/drop-request-status-dialog.jsp @@ -15,7 +15,7 @@ limitations under the License. --%> <%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> -