You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by mcgilman <gi...@git.apache.org> on 2016/05/23 15:53:50 UTC

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

GitHub user mcgilman opened a pull request:

    https://github.com/apache/nifi/pull/461

    NIFI-1781: UI authorization updates

    NIFI-1781:
    - Including access policies in the breadcrumb's trail.
    - Updating toolbox according to group access policies.
    - Updating actions in palette based on selection access policies.
    NIFI-1554:
    - Introducing authorization during two phase commit.
    - Introducing snippet authorization according to the encapsulated components and the action performed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mcgilman/nifi NIFI-1781

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/461.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #461
    
----

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64262070
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java ---
    @@ -2024,6 +1843,24 @@ public Response instantiateTemplate(
         // templates
         // ---------
     
    +    private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId) {
    +        // ensure write access to the target process group
    +        lookup.getProcessGroup(groupId).authorize(authorizer, RequestAction.WRITE);
    +
    +        // ensure read permission to every component in the snippet
    +        final Snippet snippet = lookup.getSnippet(snippetId);
    +        final Set<Authorizable> authorizables = new HashSet<>();
    +        authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    --- End diff --
    
    Awesome. Will do!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64248023
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java ---
    @@ -46,6 +48,26 @@
     public interface ProcessGroup extends Authorizable {
     
         /**
    +     * Predicate for filtering schedulable Processors.
    +     */
    +    Predicate<ProcessorNode> SCHEDULABLE_PROCESSORS = node -> !node.isRunning() && node.getScheduledState() != ScheduledState.DISABLED;
    +
    +    /**
    +     * Predicate for filtering unschedulable Processors.
    +     */
    +    Predicate<ProcessorNode> UNSCHEDULABLE_PROCESSORS = node -> node.isRunning();
    +
    +    /**
    +     * Predicate for filtering schedulable Ports
    +     */
    +    Predicate<Port> SCHEDULABLE_PORTS = port -> port.getScheduledState() != ScheduledState.DISABLED;
    +
    +    /**
    +     * Predicate for filtering schedulable Ports
    +     */
    +    Predicate<Port> UNSCHEDULABLE_PORTS = port -> port.getScheduledState() == ScheduledState.RUNNING;
    --- End diff --
    
    Shouldn't this be "port.isRunning()" instead, like we do for Processor? It seems like we could actually just have a SCHEDULABLE_TRIGGERABLE and an UNSCHEDULABLE_TRIGGERABLE rather than separate ones for ports & processors?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64259092
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java ---
    @@ -0,0 +1,397 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.nifi.web.api;
    +
    +import com.wordnik.swagger.annotations.Api;
    +import com.wordnik.swagger.annotations.ApiOperation;
    +import com.wordnik.swagger.annotations.ApiParam;
    +import com.wordnik.swagger.annotations.ApiResponse;
    +import com.wordnik.swagger.annotations.ApiResponses;
    +import com.wordnik.swagger.annotations.Authorization;
    +import org.apache.nifi.authorization.Authorizer;
    +import org.apache.nifi.authorization.RequestAction;
    +import org.apache.nifi.authorization.resource.Authorizable;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.authorization.user.NiFiUserUtils;
    +import org.apache.nifi.cluster.manager.impl.WebClusterManager;
    +import org.apache.nifi.controller.Snippet;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.apache.nifi.web.NiFiServiceFacade;
    +import org.apache.nifi.web.Revision;
    +import org.apache.nifi.web.api.dto.SnippetDTO;
    +import org.apache.nifi.web.api.entity.SnippetEntity;
    +
    +import javax.servlet.http.HttpServletRequest;
    +import javax.ws.rs.Consumes;
    +import javax.ws.rs.DELETE;
    +import javax.ws.rs.HttpMethod;
    +import javax.ws.rs.POST;
    +import javax.ws.rs.PUT;
    +import javax.ws.rs.Path;
    +import javax.ws.rs.PathParam;
    +import javax.ws.rs.Produces;
    +import javax.ws.rs.core.Context;
    +import javax.ws.rs.core.MediaType;
    +import javax.ws.rs.core.Response;
    +import java.net.URI;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +/**
    + * RESTful endpoint for querying dataflow snippets.
    + */
    +@Path("/snippets")
    +@Api(
    +    value = "/snippets",
    +    description = "Endpoint for accessing dataflow snippets."
    +)
    +public class SnippetResource extends ApplicationResource {
    +
    +    private NiFiServiceFacade serviceFacade;
    +    private WebClusterManager clusterManager;
    +    private NiFiProperties properties;
    +    private Authorizer authorizer;
    +
    +    /**
    +     * Populate the uri's for the specified snippet.
    +     *
    +     * @param entity processors
    +     * @return dtos
    +     */
    +    private SnippetEntity populateRemainingSnippetEntityContent(SnippetEntity entity) {
    +        if (entity.getSnippet() != null) {
    +            populateRemainingSnippetContent(entity.getSnippet());
    +        }
    +        return entity;
    +    }
    +
    +    /**
    +     * Populates the uri for the specified snippet.
    +     */
    +    private SnippetDTO populateRemainingSnippetContent(SnippetDTO snippet) {
    +        String snippetGroupId = snippet.getParentGroupId();
    +
    +        // populate the snippet href
    +        snippet.setUri(generateResourceUri("process-groups", snippetGroupId, "snippets", snippet.getId()));
    +
    +        return snippet;
    +    }
    +
    +    // --------
    +    // snippets
    +    // --------
    +
    +    /**
    +     * Creates a snippet based off the specified configuration.
    +     *
    +     * @param httpServletRequest request
    +     * @param snippetEntity A snippetEntity
    +     * @return A snippetEntity
    +     */
    +    @POST
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Creates a snippet",
    +        response = SnippetEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
    +            @Authorization(value = "Administrator", type = "ROLE_ADMIN")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response createSnippet(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The snippet configuration details.",
    +            required = true
    +        )
    +        final SnippetEntity snippetEntity) {
    +
    +        if (snippetEntity == null || snippetEntity.getSnippet() == null) {
    +            throw new IllegalArgumentException("Snippet details must be specified.");
    +        }
    +
    +        if (snippetEntity.getSnippet().getId() != null) {
    +            throw new IllegalArgumentException("Snippet ID cannot be specified.");
    +        }
    +
    +        if (properties.isClusterManager()) {
    +            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), snippetEntity, getHeaders()).getResponse();
    +        }
    +
    +        // handle expects request (usually from the cluster manager)
    +        final boolean validationPhase = isValidationPhase(httpServletRequest);
    +        if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
    +            // authorize access
    +            serviceFacade.authorizeAccess(lookup -> {
    +                final SnippetDTO snippet = snippetEntity.getSnippet();
    +
    +                // ensure read permission to every component in the snippet
    +                final Set<Authorizable> authorizables = new HashSet<>();
    +                authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).collect(Collectors.toSet()));
    +                authorizables.stream().forEach(authorizable -> authorizable.authorize(authorizer, RequestAction.READ));
    +            });
    +        }
    +        if (validationPhase) {
    +            return generateContinueResponse().build();
    +        }
    +
    +        // set the processor id as appropriate
    +        snippetEntity.getSnippet().setId(generateUuid());
    +
    +        // create the snippet
    +        final SnippetEntity entity = serviceFacade.createSnippet(snippetEntity.getSnippet());
    +        populateRemainingSnippetEntityContent(entity);
    +
    +        // build the response
    +        return clusterContext(generateCreatedResponse(URI.create(entity.getSnippet().getUri()), entity)).build();
    +    }
    +
    +    /**
    +     * Updates the specified snippet. The contents of the snippet (component
    +     * ids) cannot be updated once the snippet is created.
    +     *
    +     * @param httpServletRequest request
    +     * @param snippetId The id of the snippet.
    +     * @param snippetEntity A snippetEntity
    +     * @return A snippetEntity
    +     */
    +    @PUT
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @Path("{id}")
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Updates a snippet",
    +        response = SnippetEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response updateSnippet(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The snippet id.",
    +            required = true
    +        )
    +        @PathParam("id") String snippetId,
    +        @ApiParam(
    +            value = "The snippet configuration details.",
    +            required = true
    +        ) final SnippetEntity snippetEntity) {
    +
    +        if (snippetEntity == null || snippetEntity.getSnippet() == null) {
    +            throw new IllegalArgumentException("Snippet details must be specified.");
    +        }
    +
    +        // ensure the ids are the same
    +        final SnippetDTO requestSnippetDTO = snippetEntity.getSnippet();
    +        if (!snippetId.equals(requestSnippetDTO.getId())) {
    +            throw new IllegalArgumentException(String.format("The snippet id (%s) in the request body does not equal the "
    +                + "snippet id of the requested resource (%s).", requestSnippetDTO.getId(), snippetId));
    +        }
    +
    +        // replicate if cluster manager
    +        if (properties.isClusterManager()) {
    +            return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), snippetEntity, getHeaders()).getResponse();
    +        }
    +
    +        final NiFiUser user = NiFiUserUtils.getNiFiUser();
    +
    +        // get the revision from this snippet
    +        final Set<Revision> revisions = serviceFacade.getRevisionsFromSnippet(snippetId);
    +
    +        // 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 target process group
    +                if (requestSnippetDTO.getParentGroupId() != null) {
    +                    lookup.getProcessGroup(requestSnippetDTO.getParentGroupId()).authorize(authorizer, RequestAction.WRITE);
    +                }
    +
    +                // ensure read permission to every component in the snippet
    +                final Snippet snippet = lookup.getSnippet(snippetId);
    +                final Set<Authorizable> authorizables = new HashSet<>();
    +                authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    --- End diff --
    
    I'd make the same recommendation here as I made above, where we are iterating over each of these types of things and doing authorization. Rather than having the 'authorizables' set we could just define the lambda expression above and then call it via 'forEach' on each Stream.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64248276
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java ---
    @@ -283,38 +283,21 @@ public boolean isRootGroup() {
         public void startProcessing() {
             readLock.lock();
             try {
    -            for (final ProcessorNode node : processors.values()) {
    +            findAllProcessors().stream().filter(SCHEDULABLE_PROCESSORS).forEach(node -> {
    --- End diff --
    
    It seems like here, rather than finding all processors and putting them into a set, then converting to a stream and filtering, we should allow a Predicate to be passed into the findAllProcessors() method? Same for input/output ports


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64249465
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java ---
    @@ -568,164 +699,50 @@ public void verifyDeleteReportingTask(String reportingTaskId) {
     
     
         @Override
    -    public void verifyUpdateSnippet(SnippetDTO snippetDto) {
    +    public void verifyUpdateSnippet(SnippetDTO snippetDto, final Set<String> affectedComponentIds) {
             try {
                 // if snippet does not exist, then the update request is likely creating it
                 // so we don't verify since it will fail
                 if (snippetDAO.hasSnippet(snippetDto.getId())) {
                     snippetDAO.verifyUpdate(snippetDto);
                 }
             } catch (final Exception e) {
    -            revisionManager.cancelClaim(snippetDto.getId());
    +            affectedComponentIds.forEach(id -> revisionManager.cancelClaim(snippetDto.getId()));
                 throw e;
             }
         }
     
    -    private Set<Revision> getRevisionsForGroup(final String groupId) {
    -        final Set<Revision> revisions = new HashSet<>();
    -
    -        revisions.add(revisionManager.getRevision(groupId));
    -        final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
    -        if (processGroup == null) {
    -            throw new IllegalArgumentException("Snippet contains a reference to Process Group with ID " + groupId + " but no Process Group exists with that ID");
    -        }
    -
    -        processGroup.getConnections().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getFunnels().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getInputPorts().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getOutputPorts().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getLabels().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getProcessors().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getRemoteProcessGroups().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getProcessGroups().stream().map(c -> c.getIdentifier()).forEach(id -> revisions.addAll(getRevisionsForGroup(id)));
    -
    -        return revisions;
    -    }
    -
    -    private Set<Revision> getRevisionsForSnippet(final SnippetDTO snippetDto) {
    -        final Set<Revision> requiredRevisions = new HashSet<>();
    -        requiredRevisions.add(revisionManager.getRevision(snippetDto.getId()));
    -        snippetDto.getConnections().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getFunnels().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getInputPorts().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getOutputPorts().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getLabels().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getProcessors().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getRemoteProcessGroups().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        for (final String groupId : snippetDto.getProcessGroups().keySet()) {
    -            requiredRevisions.addAll(getRevisionsForGroup(groupId));
    -        }
    -
    -        return requiredRevisions;
    -    }
    -
    -    private ProcessGroup getGroup(final String groupId) {
    -        return revisionManager.get(groupId, rev -> processGroupDAO.getProcessGroup(groupId));
    -    }
    -
         @Override
    -    public UpdateResult<SnippetEntity> updateSnippet(final Revision revision, final SnippetDTO snippetDto) {
    -        // if label does not exist, then create new label
    -        if (snippetDAO.hasSnippet(snippetDto.getId()) == false) {
    -            return new UpdateResult<>(createSnippet(snippetDto), true);
    -        }
    -
    -        final Set<Revision> requiredRevisions = getRevisionsForSnippet(snippetDto);
    -
    -        // if the parent group is specified in the request, ensure write access to it as it could be moving the components in the snippet
    -        final String requestProcessGroupIdentifier = snippetDto.getParentGroupId();
    -        if (requestProcessGroupIdentifier != null) {
    -            final ProcessGroup requestProcessGroup = processGroupDAO.getProcessGroup(requestProcessGroupIdentifier);
    -            requestProcessGroup.authorize(authorizer, RequestAction.WRITE);
    -        }
    -
    +    public SnippetEntity updateSnippet(final Set<Revision> revisions, final SnippetDTO snippetDto) {
    --- End diff --
    
    This name is a bit confusing, as it is not actually updating a snippet but rather updating the components referenced by the snippet. Perhaps we should rename it "updateSnippetComponents"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64258489
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java ---
    @@ -2024,6 +1843,24 @@ public Response instantiateTemplate(
         // templates
         // ---------
     
    +    private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId) {
    +        // ensure write access to the target process group
    +        lookup.getProcessGroup(groupId).authorize(authorizer, RequestAction.WRITE);
    +
    +        // ensure read permission to every component in the snippet
    +        final Snippet snippet = lookup.getSnippet(snippetId);
    +        final Set<Authorizable> authorizables = new HashSet<>();
    +        authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    --- End diff --
    
    Or better yet, to get rid of the 'authorizables' Set all together and define the final authorization lambda as an object and just call it on each of these streams. E.g.,
    
    ```
    final Consumer<Authorizable> auth = authorizable -> authorizable.authorize(authorizer, RequestAciton.READ);
    snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(auth);
    snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).forEach(auth);
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64259949
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java ---
    @@ -321,6 +320,157 @@ public Response getControllerServices(
             return clusterContext(generateOkResponse(entity)).build();
         }
     
    +    /**
    +     * Updates the specified process group.
    +     *
    +     * @param httpServletRequest request
    +     * @param id The id of the process group.
    +     * @param scheduleComponentsEntity A scheduleComponentsEntity.
    +     * @return A processGroupEntity.
    +     */
    +    @PUT
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @Path("process-groups/{id}")
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Updates a process group",
    +        response = ScheduleComponentsEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response scheduleComponents(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The process group id.",
    +            required = true
    +        )
    +        @PathParam("id") String id,
    +        ScheduleComponentsEntity scheduleComponentsEntity) {
    +
    +        authorizeFlow();
    +
    +        // ensure the same id is being used
    +        if (!id.equals(scheduleComponentsEntity.getId())) {
    +            throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
    +                + "not equal the process group id of the requested resource (%s).", scheduleComponentsEntity.getId(), id));
    +        }
    +
    +        final ScheduledState state;
    +        if (scheduleComponentsEntity.getState() == null) {
    +            throw new IllegalArgumentException("The scheduled state must be specified.");
    +        } else {
    +            try {
    +                state = ScheduledState.valueOf(scheduleComponentsEntity.getState());
    +            } catch (final IllegalArgumentException iae) {
    +                throw new IllegalArgumentException(String.format("The scheduled must be one of [].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
    +            }
    +        }
    +
    +        // ensure its a supported scheduled state
    +        if (ScheduledState.DISABLED.equals(state) || ScheduledState.STARTING.equals(state) || ScheduledState.STOPPING.equals(state)) {
    +            throw new IllegalArgumentException(String.format("The scheduled must be one of [].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
    +        }
    +
    +        // if the components are not specified, gather all components and their current revision
    +        if (scheduleComponentsEntity.getComponents() == null) {
    +            // TODO - this will break while clustered until nodes are able to process/replicate requests
    +            // get the current revisions for the components being updated
    +            final Set<Revision> revisions = serviceFacade.getRevisionsFromGroup(id, group -> {
    +                final Set<String> componentIds = new HashSet<>();
    +
    +                // ensure authorized for each processor we will attempt to schedule
    +                group.findAllProcessors().stream().filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS).forEach(processor -> {
    +                    if (processor.isAuthorized(authorizer, RequestAction.WRITE)) {
    --- End diff --
    
    Will do!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64256472
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java ---
    @@ -321,6 +320,157 @@ public Response getControllerServices(
             return clusterContext(generateOkResponse(entity)).build();
         }
     
    +    /**
    +     * Updates the specified process group.
    +     *
    +     * @param httpServletRequest request
    +     * @param id The id of the process group.
    +     * @param scheduleComponentsEntity A scheduleComponentsEntity.
    +     * @return A processGroupEntity.
    +     */
    +    @PUT
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @Path("process-groups/{id}")
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Updates a process group",
    +        response = ScheduleComponentsEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response scheduleComponents(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The process group id.",
    +            required = true
    +        )
    +        @PathParam("id") String id,
    +        ScheduleComponentsEntity scheduleComponentsEntity) {
    +
    +        authorizeFlow();
    +
    +        // ensure the same id is being used
    +        if (!id.equals(scheduleComponentsEntity.getId())) {
    +            throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
    +                + "not equal the process group id of the requested resource (%s).", scheduleComponentsEntity.getId(), id));
    +        }
    +
    +        final ScheduledState state;
    +        if (scheduleComponentsEntity.getState() == null) {
    +            throw new IllegalArgumentException("The scheduled state must be specified.");
    +        } else {
    +            try {
    +                state = ScheduledState.valueOf(scheduleComponentsEntity.getState());
    +            } catch (final IllegalArgumentException iae) {
    +                throw new IllegalArgumentException(String.format("The scheduled must be one of [].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
    +            }
    +        }
    +
    +        // ensure its a supported scheduled state
    +        if (ScheduledState.DISABLED.equals(state) || ScheduledState.STARTING.equals(state) || ScheduledState.STOPPING.equals(state)) {
    +            throw new IllegalArgumentException(String.format("The scheduled must be one of [].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
    +        }
    +
    +        // if the components are not specified, gather all components and their current revision
    +        if (scheduleComponentsEntity.getComponents() == null) {
    +            // TODO - this will break while clustered until nodes are able to process/replicate requests
    +            // get the current revisions for the components being updated
    +            final Set<Revision> revisions = serviceFacade.getRevisionsFromGroup(id, group -> {
    +                final Set<String> componentIds = new HashSet<>();
    +
    +                // ensure authorized for each processor we will attempt to schedule
    +                group.findAllProcessors().stream().filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS).forEach(processor -> {
    +                    if (processor.isAuthorized(authorizer, RequestAction.WRITE)) {
    +                        componentIds.add(processor.getIdentifier());
    +                    }
    +                });
    +
    +                // ensure authorized for each input port we will attempt to schedule
    +                group.findAllInputPorts().stream().filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS).forEach(inputPort -> {
    +                    if (inputPort.isAuthorized(authorizer, RequestAction.WRITE)) {
    +                        componentIds.add(inputPort.getIdentifier());
    +                    }
    +                });
    +
    +                // ensure authorized for each output port we will attempt to schedule
    +                group.findAllOutputPorts().stream().filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS).forEach(outputPort -> {
    +                    if (outputPort.isAuthorized(authorizer, RequestAction.WRITE)) {
    +                        componentIds.add(outputPort.getIdentifier());
    +                    }
    +                });
    +
    +                return componentIds;
    +            });
    +
    +            // build the component mapping
    +            final Map<String, RevisionDTO> componentsToSchedule = new HashMap<>();
    +            revisions.forEach(revision -> {
    +                final RevisionDTO dto = new RevisionDTO();
    +                dto.setClientId(revision.getClientId());
    +                dto.setVersion(revision.getVersion());
    +                componentsToSchedule.put(revision.getComponentId(), dto);
    +            });
    +
    +            // set the components and their current revision
    +            scheduleComponentsEntity.setComponents(componentsToSchedule);
    +        }
    +
    +        if (properties.isClusterManager()) {
    +            return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), scheduleComponentsEntity, getHeaders()).getResponse();
    +        }
    +
    +        final NiFiUser user = NiFiUserUtils.getNiFiUser();
    +
    +        final Map<String, RevisionDTO> componentsToSchedule = scheduleComponentsEntity.getComponents();
    +        final Map<String, Revision> componentRevisions = componentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
    +        final Set<Revision> revisions = new HashSet<>(componentRevisions.values());
    +
    +        // handle expects request (usually from the cluster manager)
    +        final boolean validationPhase = isValidationPhase(httpServletRequest);
    +        if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
    +
    +            // ensure access to everything component being scheduled
    --- End diff --
    
    should this comment say "every component"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64250769
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java ---
    @@ -46,6 +48,26 @@
     public interface ProcessGroup extends Authorizable {
     
         /**
    +     * Predicate for filtering schedulable Processors.
    +     */
    +    Predicate<ProcessorNode> SCHEDULABLE_PROCESSORS = node -> !node.isRunning() && node.getScheduledState() != ScheduledState.DISABLED;
    +
    +    /**
    +     * Predicate for filtering unschedulable Processors.
    +     */
    +    Predicate<ProcessorNode> UNSCHEDULABLE_PROCESSORS = node -> node.isRunning();
    +
    +    /**
    +     * Predicate for filtering schedulable Ports
    +     */
    +    Predicate<Port> SCHEDULABLE_PORTS = port -> port.getScheduledState() != ScheduledState.DISABLED;
    --- End diff --
    
    I did not modify any logic here. Simply relocated to not repeat. If that logic is incorrect, I can certainly update it, but it hasn't changed here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64250889
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java ---
    @@ -568,164 +699,50 @@ public void verifyDeleteReportingTask(String reportingTaskId) {
     
     
         @Override
    -    public void verifyUpdateSnippet(SnippetDTO snippetDto) {
    +    public void verifyUpdateSnippet(SnippetDTO snippetDto, final Set<String> affectedComponentIds) {
             try {
                 // if snippet does not exist, then the update request is likely creating it
                 // so we don't verify since it will fail
                 if (snippetDAO.hasSnippet(snippetDto.getId())) {
                     snippetDAO.verifyUpdate(snippetDto);
                 }
             } catch (final Exception e) {
    -            revisionManager.cancelClaim(snippetDto.getId());
    +            affectedComponentIds.forEach(id -> revisionManager.cancelClaim(snippetDto.getId()));
                 throw e;
             }
         }
     
    -    private Set<Revision> getRevisionsForGroup(final String groupId) {
    -        final Set<Revision> revisions = new HashSet<>();
    -
    -        revisions.add(revisionManager.getRevision(groupId));
    -        final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
    -        if (processGroup == null) {
    -            throw new IllegalArgumentException("Snippet contains a reference to Process Group with ID " + groupId + " but no Process Group exists with that ID");
    -        }
    -
    -        processGroup.getConnections().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getFunnels().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getInputPorts().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getOutputPorts().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getLabels().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getProcessors().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getRemoteProcessGroups().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
    -        processGroup.getProcessGroups().stream().map(c -> c.getIdentifier()).forEach(id -> revisions.addAll(getRevisionsForGroup(id)));
    -
    -        return revisions;
    -    }
    -
    -    private Set<Revision> getRevisionsForSnippet(final SnippetDTO snippetDto) {
    -        final Set<Revision> requiredRevisions = new HashSet<>();
    -        requiredRevisions.add(revisionManager.getRevision(snippetDto.getId()));
    -        snippetDto.getConnections().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getFunnels().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getInputPorts().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getOutputPorts().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getLabels().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getProcessors().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        snippetDto.getRemoteProcessGroups().entrySet().stream()
    -            .map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
    -            .forEach(rev -> requiredRevisions.add(rev));
    -
    -        for (final String groupId : snippetDto.getProcessGroups().keySet()) {
    -            requiredRevisions.addAll(getRevisionsForGroup(groupId));
    -        }
    -
    -        return requiredRevisions;
    -    }
    -
    -    private ProcessGroup getGroup(final String groupId) {
    -        return revisionManager.get(groupId, rev -> processGroupDAO.getProcessGroup(groupId));
    -    }
    -
         @Override
    -    public UpdateResult<SnippetEntity> updateSnippet(final Revision revision, final SnippetDTO snippetDto) {
    -        // if label does not exist, then create new label
    -        if (snippetDAO.hasSnippet(snippetDto.getId()) == false) {
    -            return new UpdateResult<>(createSnippet(snippetDto), true);
    -        }
    -
    -        final Set<Revision> requiredRevisions = getRevisionsForSnippet(snippetDto);
    -
    -        // if the parent group is specified in the request, ensure write access to it as it could be moving the components in the snippet
    -        final String requestProcessGroupIdentifier = snippetDto.getParentGroupId();
    -        if (requestProcessGroupIdentifier != null) {
    -            final ProcessGroup requestProcessGroup = processGroupDAO.getProcessGroup(requestProcessGroupIdentifier);
    -            requestProcessGroup.authorize(authorizer, RequestAction.WRITE);
    -        }
    -
    +    public SnippetEntity updateSnippet(final Set<Revision> revisions, final SnippetDTO snippetDto) {
    --- End diff --
    
    Will do!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64251051
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java ---
    @@ -46,6 +48,26 @@
     public interface ProcessGroup extends Authorizable {
     
         /**
    +     * Predicate for filtering schedulable Processors.
    +     */
    +    Predicate<ProcessorNode> SCHEDULABLE_PROCESSORS = node -> !node.isRunning() && node.getScheduledState() != ScheduledState.DISABLED;
    +
    +    /**
    +     * Predicate for filtering unschedulable Processors.
    +     */
    +    Predicate<ProcessorNode> UNSCHEDULABLE_PROCESSORS = node -> node.isRunning();
    +
    +    /**
    +     * Predicate for filtering schedulable Ports
    +     */
    +    Predicate<Port> SCHEDULABLE_PORTS = port -> port.getScheduledState() != ScheduledState.DISABLED;
    +
    +    /**
    +     * Predicate for filtering schedulable Ports
    +     */
    +    Predicate<Port> UNSCHEDULABLE_PORTS = port -> port.getScheduledState() == ScheduledState.RUNNING;
    --- End diff --
    
    Happy to update the logic to whatever it needs to be. However, this is how it was previously.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/461


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on the pull request:

    https://github.com/apache/nifi/pull/461#issuecomment-221051044
  
    Thanks @markap14 
    
    All changes have been applied.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the pull request:

    https://github.com/apache/nifi/pull/461#issuecomment-221046258
  
    @mcgilman looks good for the most part. I left some comments in-line (most of which you've already addressed before I finished the review even!) Excellent. Once the last few things are merged in I'm a +1. Thanks for knocking this out.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64257999
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java ---
    @@ -2024,6 +1843,24 @@ public Response instantiateTemplate(
         // templates
         // ---------
     
    +    private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId) {
    +        // ensure write access to the target process group
    +        lookup.getProcessGroup(groupId).authorize(authorizer, RequestAction.WRITE);
    +
    +        // ensure read permission to every component in the snippet
    +        final Snippet snippet = lookup.getSnippet(snippetId);
    +        final Set<Authorizable> authorizables = new HashSet<>();
    +        authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    --- End diff --
    
    It probably makes sense here to change this to:
    `snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(auth -> authorizables.add(auth));
    `
    This way we don't have to keep creating all of these intermediate Set objects just so that we can add them to the 'authorizables' set.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64259242
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java ---
    @@ -0,0 +1,397 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.nifi.web.api;
    +
    +import com.wordnik.swagger.annotations.Api;
    +import com.wordnik.swagger.annotations.ApiOperation;
    +import com.wordnik.swagger.annotations.ApiParam;
    +import com.wordnik.swagger.annotations.ApiResponse;
    +import com.wordnik.swagger.annotations.ApiResponses;
    +import com.wordnik.swagger.annotations.Authorization;
    +import org.apache.nifi.authorization.Authorizer;
    +import org.apache.nifi.authorization.RequestAction;
    +import org.apache.nifi.authorization.resource.Authorizable;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.authorization.user.NiFiUserUtils;
    +import org.apache.nifi.cluster.manager.impl.WebClusterManager;
    +import org.apache.nifi.controller.Snippet;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.apache.nifi.web.NiFiServiceFacade;
    +import org.apache.nifi.web.Revision;
    +import org.apache.nifi.web.api.dto.SnippetDTO;
    +import org.apache.nifi.web.api.entity.SnippetEntity;
    +
    +import javax.servlet.http.HttpServletRequest;
    +import javax.ws.rs.Consumes;
    +import javax.ws.rs.DELETE;
    +import javax.ws.rs.HttpMethod;
    +import javax.ws.rs.POST;
    +import javax.ws.rs.PUT;
    +import javax.ws.rs.Path;
    +import javax.ws.rs.PathParam;
    +import javax.ws.rs.Produces;
    +import javax.ws.rs.core.Context;
    +import javax.ws.rs.core.MediaType;
    +import javax.ws.rs.core.Response;
    +import java.net.URI;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +/**
    + * RESTful endpoint for querying dataflow snippets.
    + */
    +@Path("/snippets")
    +@Api(
    +    value = "/snippets",
    +    description = "Endpoint for accessing dataflow snippets."
    +)
    +public class SnippetResource extends ApplicationResource {
    +
    +    private NiFiServiceFacade serviceFacade;
    +    private WebClusterManager clusterManager;
    +    private NiFiProperties properties;
    +    private Authorizer authorizer;
    +
    +    /**
    +     * Populate the uri's for the specified snippet.
    +     *
    +     * @param entity processors
    +     * @return dtos
    +     */
    +    private SnippetEntity populateRemainingSnippetEntityContent(SnippetEntity entity) {
    +        if (entity.getSnippet() != null) {
    +            populateRemainingSnippetContent(entity.getSnippet());
    +        }
    +        return entity;
    +    }
    +
    +    /**
    +     * Populates the uri for the specified snippet.
    +     */
    +    private SnippetDTO populateRemainingSnippetContent(SnippetDTO snippet) {
    +        String snippetGroupId = snippet.getParentGroupId();
    +
    +        // populate the snippet href
    +        snippet.setUri(generateResourceUri("process-groups", snippetGroupId, "snippets", snippet.getId()));
    +
    +        return snippet;
    +    }
    +
    +    // --------
    +    // snippets
    +    // --------
    +
    +    /**
    +     * Creates a snippet based off the specified configuration.
    +     *
    +     * @param httpServletRequest request
    +     * @param snippetEntity A snippetEntity
    +     * @return A snippetEntity
    +     */
    +    @POST
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Creates a snippet",
    +        response = SnippetEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Read Only", type = "ROLE_MONITOR"),
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
    +            @Authorization(value = "Administrator", type = "ROLE_ADMIN")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response createSnippet(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The snippet configuration details.",
    +            required = true
    +        )
    +        final SnippetEntity snippetEntity) {
    +
    +        if (snippetEntity == null || snippetEntity.getSnippet() == null) {
    +            throw new IllegalArgumentException("Snippet details must be specified.");
    +        }
    +
    +        if (snippetEntity.getSnippet().getId() != null) {
    +            throw new IllegalArgumentException("Snippet ID cannot be specified.");
    +        }
    +
    +        if (properties.isClusterManager()) {
    +            return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), snippetEntity, getHeaders()).getResponse();
    +        }
    +
    +        // handle expects request (usually from the cluster manager)
    +        final boolean validationPhase = isValidationPhase(httpServletRequest);
    +        if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
    +            // authorize access
    +            serviceFacade.authorizeAccess(lookup -> {
    +                final SnippetDTO snippet = snippetEntity.getSnippet();
    +
    +                // ensure read permission to every component in the snippet
    +                final Set<Authorizable> authorizables = new HashSet<>();
    +                authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).collect(Collectors.toSet()));
    +                authorizables.stream().forEach(authorizable -> authorizable.authorize(authorizer, RequestAction.READ));
    +            });
    +        }
    +        if (validationPhase) {
    +            return generateContinueResponse().build();
    +        }
    +
    +        // set the processor id as appropriate
    +        snippetEntity.getSnippet().setId(generateUuid());
    +
    +        // create the snippet
    +        final SnippetEntity entity = serviceFacade.createSnippet(snippetEntity.getSnippet());
    +        populateRemainingSnippetEntityContent(entity);
    +
    +        // build the response
    +        return clusterContext(generateCreatedResponse(URI.create(entity.getSnippet().getUri()), entity)).build();
    +    }
    +
    +    /**
    +     * Updates the specified snippet. The contents of the snippet (component
    +     * ids) cannot be updated once the snippet is created.
    +     *
    +     * @param httpServletRequest request
    +     * @param snippetId The id of the snippet.
    +     * @param snippetEntity A snippetEntity
    +     * @return A snippetEntity
    +     */
    +    @PUT
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @Path("{id}")
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Updates a snippet",
    +        response = SnippetEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response updateSnippet(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The snippet id.",
    +            required = true
    +        )
    +        @PathParam("id") String snippetId,
    +        @ApiParam(
    +            value = "The snippet configuration details.",
    +            required = true
    +        ) final SnippetEntity snippetEntity) {
    +
    +        if (snippetEntity == null || snippetEntity.getSnippet() == null) {
    +            throw new IllegalArgumentException("Snippet details must be specified.");
    +        }
    +
    +        // ensure the ids are the same
    +        final SnippetDTO requestSnippetDTO = snippetEntity.getSnippet();
    +        if (!snippetId.equals(requestSnippetDTO.getId())) {
    +            throw new IllegalArgumentException(String.format("The snippet id (%s) in the request body does not equal the "
    +                + "snippet id of the requested resource (%s).", requestSnippetDTO.getId(), snippetId));
    +        }
    +
    +        // replicate if cluster manager
    +        if (properties.isClusterManager()) {
    +            return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), snippetEntity, getHeaders()).getResponse();
    +        }
    +
    +        final NiFiUser user = NiFiUserUtils.getNiFiUser();
    +
    +        // get the revision from this snippet
    +        final Set<Revision> revisions = serviceFacade.getRevisionsFromSnippet(snippetId);
    +
    +        // 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 target process group
    +                if (requestSnippetDTO.getParentGroupId() != null) {
    +                    lookup.getProcessGroup(requestSnippetDTO.getParentGroupId()).authorize(authorizer, RequestAction.WRITE);
    +                }
    +
    +                // ensure read permission to every component in the snippet
    +                final Snippet snippet = lookup.getSnippet(snippetId);
    +                final Set<Authorizable> authorizables = new HashSet<>();
    +                authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).collect(Collectors.toSet()));
    +                authorizables.addAll(snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).collect(Collectors.toSet()));
    +                authorizables.stream().forEach(authorizable -> authorizable.authorize(authorizer, RequestAction.WRITE));
    +            });
    +
    +            serviceFacade.claimRevisions(revisions, user);
    +        }
    +
    +        try {
    +            if (validationPhase) {
    +                serviceFacade.verifyUpdateSnippet(requestSnippetDTO, revisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet()));
    +                return generateContinueResponse().build();
    +            }
    +        } catch (final Exception e) {
    +            serviceFacade.cancelRevisions(revisions);
    +            throw e;
    +        }
    +
    +        try {
    +            // update the snippet
    +            final SnippetEntity entity = serviceFacade.updateSnippet(revisions, snippetEntity.getSnippet());
    +            populateRemainingSnippetEntityContent(entity);
    +            return clusterContext(generateOkResponse(entity)).build();
    +        } finally {
    +            serviceFacade.cancelRevisions(revisions);
    +        }
    +    }
    +
    +    /**
    +     * Removes the specified snippet.
    +     *
    +     * @param httpServletRequest request
    +     * @param snippetId The id of the snippet to remove.
    +     * @return A entity containing the client id and an updated revision.
    +     */
    +    @DELETE
    +    @Consumes(MediaType.WILDCARD)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @Path("{id}")
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Deletes the components in a snippet and drops the snippet",
    +        response = SnippetEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response deleteSnippet(
    +        @Context final HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The snippet id.",
    +            required = true
    +        )
    +        @PathParam("id") final String snippetId) {
    +
    +        // replicate if cluster manager
    +        if (properties.isClusterManager()) {
    +            return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
    +        }
    +
    +        final NiFiUser user = NiFiUserUtils.getNiFiUser();
    +
    +        // get the revision from this snippet
    +        final Set<Revision> revisions = serviceFacade.getRevisionsFromSnippet(snippetId);
    +
    +        // handle expects request (usually from the cluster manager)
    +        final boolean validationPhase = isValidationPhase(httpServletRequest);
    +        if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
    +            // authorize access
    +            serviceFacade.authorizeAccess(lookup -> {
    +                // ensure read permission to every component in the snippet
    +                final Snippet snippet = lookup.getSnippet(snippetId);
    +                final Set<Authorizable> authorizables = new HashSet<>();
    +                authorizables.addAll(snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).collect(Collectors.toSet()));
    --- End diff --
    
    And again, same as above... we should probably actually pull this out into its own method, as it's a decent chunk of code being repeated a few times.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64247888
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java ---
    @@ -46,6 +48,26 @@
     public interface ProcessGroup extends Authorizable {
     
         /**
    +     * Predicate for filtering schedulable Processors.
    +     */
    +    Predicate<ProcessorNode> SCHEDULABLE_PROCESSORS = node -> !node.isRunning() && node.getScheduledState() != ScheduledState.DISABLED;
    +
    +    /**
    +     * Predicate for filtering unschedulable Processors.
    +     */
    +    Predicate<ProcessorNode> UNSCHEDULABLE_PROCESSORS = node -> node.isRunning();
    +
    +    /**
    +     * Predicate for filtering schedulable Ports
    +     */
    +    Predicate<Port> SCHEDULABLE_PORTS = port -> port.getScheduledState() != ScheduledState.DISABLED;
    --- End diff --
    
    It seems odd to me that we are checking "!node.isRunning()" for ProcessorsNode but not checking "!port.isRunning()" for ports.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64256311
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java ---
    @@ -321,6 +320,157 @@ public Response getControllerServices(
             return clusterContext(generateOkResponse(entity)).build();
         }
     
    +    /**
    +     * Updates the specified process group.
    +     *
    +     * @param httpServletRequest request
    +     * @param id The id of the process group.
    +     * @param scheduleComponentsEntity A scheduleComponentsEntity.
    +     * @return A processGroupEntity.
    +     */
    +    @PUT
    +    @Consumes(MediaType.APPLICATION_JSON)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @Path("process-groups/{id}")
    +    // TODO - @PreAuthorize("hasRole('ROLE_DFM')")
    +    @ApiOperation(
    +        value = "Updates a process group",
    +        response = ScheduleComponentsEntity.class,
    +        authorizations = {
    +            @Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
    +        }
    +    )
    +    @ApiResponses(
    +        value = {
    +            @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
    +            @ApiResponse(code = 401, message = "Client could not be authenticated."),
    +            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
    +            @ApiResponse(code = 404, message = "The specified resource could not be found."),
    +            @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
    +        }
    +    )
    +    public Response scheduleComponents(
    +        @Context HttpServletRequest httpServletRequest,
    +        @ApiParam(
    +            value = "The process group id.",
    +            required = true
    +        )
    +        @PathParam("id") String id,
    +        ScheduleComponentsEntity scheduleComponentsEntity) {
    +
    +        authorizeFlow();
    +
    +        // ensure the same id is being used
    +        if (!id.equals(scheduleComponentsEntity.getId())) {
    +            throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
    +                + "not equal the process group id of the requested resource (%s).", scheduleComponentsEntity.getId(), id));
    +        }
    +
    +        final ScheduledState state;
    +        if (scheduleComponentsEntity.getState() == null) {
    +            throw new IllegalArgumentException("The scheduled state must be specified.");
    +        } else {
    +            try {
    +                state = ScheduledState.valueOf(scheduleComponentsEntity.getState());
    +            } catch (final IllegalArgumentException iae) {
    +                throw new IllegalArgumentException(String.format("The scheduled must be one of [].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
    +            }
    +        }
    +
    +        // ensure its a supported scheduled state
    +        if (ScheduledState.DISABLED.equals(state) || ScheduledState.STARTING.equals(state) || ScheduledState.STOPPING.equals(state)) {
    +            throw new IllegalArgumentException(String.format("The scheduled must be one of [].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
    +        }
    +
    +        // if the components are not specified, gather all components and their current revision
    +        if (scheduleComponentsEntity.getComponents() == null) {
    +            // TODO - this will break while clustered until nodes are able to process/replicate requests
    +            // get the current revisions for the components being updated
    +            final Set<Revision> revisions = serviceFacade.getRevisionsFromGroup(id, group -> {
    +                final Set<String> componentIds = new HashSet<>();
    +
    +                // ensure authorized for each processor we will attempt to schedule
    +                group.findAllProcessors().stream().filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS).forEach(processor -> {
    +                    if (processor.isAuthorized(authorizer, RequestAction.WRITE)) {
    --- End diff --
    
    It seems cleaner to me to use a filter here to check if the processor is authorized, rather than mixing stream filters with a forEach and then performing additional if-checks. I.e., instead of "if (processor.isAuthorized...) {...}" we could just do ".filter(processor -> processor.isAuthorized(...).forEach(...)" for processors, input ports & output ports


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1781: UI authorization updates

Posted by mcgilman <gi...@git.apache.org>.
Github user mcgilman commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/461#discussion_r64250267
  
    --- Diff: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java ---
    @@ -283,38 +283,21 @@ public boolean isRootGroup() {
         public void startProcessing() {
             readLock.lock();
             try {
    -            for (final ProcessorNode node : processors.values()) {
    +            findAllProcessors().stream().filter(SCHEDULABLE_PROCESSORS).forEach(node -> {
    --- End diff --
    
    Totally agree. There are a number of improvements that we can make with Java 8 available. Let's create a separate JIRA to identify those.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---