You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/02/25 16:46:23 UTC

[GitHub] [nifi] mtien-apache opened a new pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow files.

mtien-apache opened a new pull request #4846:
URL: https://github.com/apache/nifi/pull/4846


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _This feature allows the user to upload a previously exported JSON flow file into the NiFi UI and it will create a new Process Group, bypassing NiFi Registry.
   
   NIFI-8260 - [WIP] Created Upload File capability for front end and server side logic.
   Created a new API endpoint to upload a JSON flow file and transform it to a Versioned Flow Snapshot.
   Fixed the client side file form to reset after submit.
   Fixed the canvas to instantly update and show the process group after submitting the file.
   Changed the Add Processor Group dialog UI based on design notes.
   Changed the Upload File link to an icon and moved to the name input field.
   Changed the 'Registry Import' link to say 'Import from Registry' and moved to the bottom of the dialog.
   Display the filename when a file is selected.
   Added a cancel file button to the Process Group dialog._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [x] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583767187



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
##########
@@ -3766,7 +3765,7 @@ public void updateFlow(final VersionedFlowSnapshot proposedSnapshot, final Strin
                            final boolean updateDescendantVersionedFlows) {
         writeLock.lock();
         try {
-            verifyCanUpdate(proposedSnapshot, true, verifyNotDirty);
+             verifyCanUpdate(proposedSnapshot, true, verifyNotDirty);

Review comment:
       Strange - I don't think I touched this file. I'll revert it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-789200993


   @sardell Thanks for reviewing!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583918059



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4187,205 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // get the contents of the InputStream as a String
+        String stringContent;
+        if (in != null) {
+            try {
+                stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+            } catch (IOException e) {
+                throw new IOException("Unable to read the InputStream", e);
+            }
+        } else {
+            logger.warn("The InputStream is null");
+            throw new NullPointerException("The InputStream is null");
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        if (stringContent.length() > 0) {
+            try {
+                deserializedSnapshot = MAPPER.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("Parsing uploaded JSON failed", jpe);
+                return Response.status(Response.Status.OK).entity(INVALID_JSON_RESPONSE).type("application/json").build();
+            } catch (IOException e) {
+                logger.warn("Deserialization of uploaded JSON failed", e);
+                throw new IOException("Deserialization of uploaded JSON failed", e);
+            }
+        } else {
+            logger.warn("The uploaded file was empty");
+            throw new IOException("The uploaded file was empty.");
+        }
+
+        // create a new ProcessGroupEntity
+        final ProcessGroupEntity newProcessGroupEntity = createProcessGroupEntity(groupId, groupName, positionX, positionY, deserializedSnapshot);
+
+        // replicate the request or call serviceFacade.updateProcessGroup
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST, newProcessGroupEntity);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(newProcessGroupEntity.isDisconnectedNodeAcknowledged());
+        }
+
+        return withWriteLock(
+                serviceFacade,
+                newProcessGroupEntity,
+                lookup -> {
+                    final NiFiUser user = NiFiUserUtils.getNiFiUser();
+                    final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+                    processGroup.authorize(authorizer, RequestAction.WRITE, user);
+
+                    // if request specifies a Parameter Context, need to authorize that user has READ policy for the Parameter Context.
+                    final ParameterContextReferenceEntity referencedParamContext = newProcessGroupEntity.getComponent().getParameterContext();
+                    if (referencedParamContext != null && referencedParamContext.getId() != null) {
+                        lookup.getParameterContext(referencedParamContext.getId()).authorize(authorizer, RequestAction.READ, user);
+                    }
+
+                    // if any of the components is a Restricted Component, then we must authorize the user
+                    // for write access to the RestrictedComponents resource
+                    final VersionedFlowSnapshot versionedFlowSnapshot = newProcessGroupEntity.getVersionedFlowSnapshot();
+                    if (versionedFlowSnapshot != null) {
+                        final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(versionedFlowSnapshot.getFlowContents(), serviceFacade);
+                        restrictedComponents.forEach(restrictedComponent -> {
+                            final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
+                            authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
+                        });
+
+                        final Map<String, VersionedParameterContext> parameterContexts = versionedFlowSnapshot.getParameterContexts();
+                        if (parameterContexts != null) {
+                            parameterContexts.values().forEach(context -> AuthorizeParameterReference.authorizeParameterContextAddition(context, serviceFacade, authorizer, lookup, user));
+                        }
+                    }
+                },
+                () -> {
+                    final VersionedFlowSnapshot versionedFlowSnapshot = newProcessGroupEntity.getVersionedFlowSnapshot();
+                    if (versionedFlowSnapshot != null) {
+                        serviceFacade.verifyComponentTypes(versionedFlowSnapshot.getFlowContents());
+                    }
+                },
+                processGroupEntity -> {
+                    final ProcessGroupDTO processGroup = processGroupEntity.getComponent();
+
+                    // set the processor id as appropriate
+                    processGroup.setId(generateUuid());
+
+                    // get the versioned flow
+                    final VersionedFlowSnapshot flowSnapshot = processGroupEntity.getVersionedFlowSnapshot();
+
+                    // create the process group contents
+                    final Revision revision = new Revision((long) 0, clientId, processGroup.getId());
+
+                    ProcessGroupEntity entity = serviceFacade.createProcessGroup(revision, groupId, processGroup);
+
+                    if (flowSnapshot != null) {
+                        final RevisionDTO revisionDto = entity.getRevision();
+                        final String newGroupId = entity.getComponent().getId();
+                        final Revision newGroupRevision = new Revision(revisionDto.getVersion(), revisionDto.getClientId(), newGroupId);
+
+                        // We don't want the Process Group's position to be updated because we want to keep the position where the user
+                        // placed the Process Group. We do not want to use the name of the Process Group that is in the Flow Contents.
+                        // To accomplish this, we call updateProcessGroupContents() passing 'false' for the updateSettings flag, set
+                        // the Process Group name, and null out the position.
+                        flowSnapshot.getFlowContents().setPosition(null);
+                        flowSnapshot.getFlowContents().setName(groupName);
+
+                        entity = serviceFacade.updateProcessGroupContents(newGroupRevision, newGroupId, null, flowSnapshot,
+                                getIdGenerationSeed().orElse(null), false, false, true);
+                    }
+
+                    populateRemainingProcessGroupEntityContent(entity);
+
+                    // generate a 201 created response
+                    String uri = entity.getUri();
+                    return generateCreatedResponse(URI.create(uri), entity).build();
+                }
+        );

Review comment:
       I believe I can partially refactor this block. Will make the change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786974249


   Just pushed a commit - 
   - Hide the Upload File button when grouping components @pvillard31 
   - Refactored `uploadProcessGroup` and exception handling @exceptionfactory 
   - Refactored `nf-ng-group-component.js` @sardell
   
   Thanks
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786894887


   @pvillard31 Agreed the upload file button should be removed when a user clicks on the "Group" button. Will make the change.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583767556



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");

Review comment:
       Revised the message.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583774943



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-group-component.js
##########
@@ -304,7 +461,7 @@
                     });
                 }).promise();
             }
-        }
+        };

Review comment:
       It's not necessary to have the semicolon, but I thought it would be more complete.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583773895



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";
+                return Response.status(Response.Status.OK).entity(responseJson).type("application/json").build();
+            } catch (IOException e) {
+                logger.warn("An error occurred while deserializing the flow version.", e);
+            }
+        } else {
+            logger.warn("The uploaded file was empty");
+            throw new IOException("The uploaded file was empty.");
+        }
+
+        sanitizeRegistryInfo(deserializedSnapshot.getFlowContents());
+
+        // resolve Bundle info
+        serviceFacade.discoverCompatibleBundles(deserializedSnapshot.getFlowContents());
+
+        // if there are any Controller Services referenced that are inherited from the parent group, resolve those to point to the appropriate Controller Service, if we are able to.
+        serviceFacade.resolveInheritedControllerServices(deserializedSnapshot, groupId, NiFiUserUtils.getNiFiUser());
+
+        // create a new ProcessGroupDTO
+        ProcessGroupDTO processGroupDTO = new ProcessGroupDTO();
+
+        processGroupDTO.setParentGroupId(groupId);
+        processGroupDTO.setName(groupName);
+
+        newProcessGroupEntity.setComponent(processGroupDTO);
+        newProcessGroupEntity.setVersionedFlowSnapshot(deserializedSnapshot);
+
+        // create a PositionDTO
+        PositionDTO positionDTO = new PositionDTO();
+        positionDTO.setX(positionX);
+        positionDTO.setY(positionY);
+        newProcessGroupEntity.getComponent().setPosition(positionDTO);

Review comment:
       Good idea - refactored.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786062176


   @joewitt Yes! Changed the term to flow definition.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583773261



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";

Review comment:
       Revised.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] pvillard31 commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786156302


   Built the PR and gave it a try. When trying to import a flow (json file) that I just exported from somewhere else on the canvas by right clicking on a process group / download flow definition, I always got the below stack trace:
   
   ````
   2021-02-25 23:42:36,617 ERROR [NiFi Web Server-234] o.a.nifi.web.api.config.ThrowableMapper An unexpected error has occurred: java.lang.NullPointerException. Returning Internal Server Error response.
   java.lang.NullPointerException: null
           at org.apache.nifi.web.api.ProcessGroupResource.lambda$uploadProcessGroup$106(ProcessGroupResource.java:4373)
           at org.apache.nifi.web.api.ApplicationResource.withWriteLock(ApplicationResource.java:719)
           at org.apache.nifi.web.api.ProcessGroupResource.uploadProcessGroup(ProcessGroupResource.java:4328)
           at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at org.glassfish.jersey.server.model.internal.ResourceMethodInvocationHandlerFactory.lambda$static$0(ResourceMethodInvocationHandlerFactory.java:76)
           at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher$1.run(AbstractJavaResourceMethodDispatcher.java:148)
           at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.invoke(AbstractJavaResourceMethodDispatcher.java:191)
           at org.glassfish.jersey.server.model.internal.JavaResourceMethodDispatcherProvider$ResponseOutInvoker.doDispatch(JavaResourceMethodDispatcherProvider.java:200)
           at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.dispatch(AbstractJavaResourceMethodDispatcher.java:103)
           at org.glassfish.jersey.server.model.ResourceMethodInvoker.invoke(ResourceMethodInvoker.java:493)
           at org.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:415)
           at org.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:104)
           at org.glassfish.jersey.server.ServerRuntime$1.run(ServerRuntime.java:277)
           at org.glassfish.jersey.internal.Errors$1.call(Errors.java:272)
           at org.glassfish.jersey.internal.Errors$1.call(Errors.java:268)
           at org.glassfish.jersey.internal.Errors.process(Errors.java:316)
           at org.glassfish.jersey.internal.Errors.process(Errors.java:298)
           at org.glassfish.jersey.internal.Errors.process(Errors.java:268)
           at org.glassfish.jersey.process.internal.RequestScope.runInScope(RequestScope.java:289)
           at org.glassfish.jersey.server.ServerRuntime.process(ServerRuntime.java:256)
           at org.glassfish.jersey.server.ApplicationHandler.handle(ApplicationHandler.java:703)
           at org.glassfish.jersey.servlet.WebComponent.serviceImpl(WebComponent.java:416)
           at org.glassfish.jersey.servlet.WebComponent.service(WebComponent.java:370)
           at org.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:389)
           at org.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:342)
           at org.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:229)
           at org.eclipse.jetty.servlet.ServletHolder$NotAsync.service(ServletHolder.java:1452)
           at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:791)
           at org.eclipse.jetty.servlet.ServletHandler$ChainEnd.doFilter(ServletHandler.java:1626)
           at org.apache.nifi.web.filter.RequestLogger.doFilter(RequestLogger.java:66)
           at org.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:193)
           at org.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1601)
   ...
   ````


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583775115



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css
##########
@@ -286,45 +290,125 @@ span.details-title {
     height: 28px;
 }
 
-#template-file-field {
+#select-file-button button {
+    position: absolute;
+    float: right;
+    right: 4px;
+    top: 22px;
+    font-size: 18px;
+    color: #004849;
+    border: none;
+    background-color: transparent;
+}
+
+/*#select-file-button button:hover {*/

Review comment:
       Removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] pvillard31 commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786855079


   Note sure if this has been addressed but I noticed that: if selecting a flow on the canvas ans clicking on the "Group" icon (to make a process group from the selection), then the icon to upload a JSON file appears when it should just ask for the new process group to create from the selection.
   
   <img width="1407" alt="Screenshot 2021-02-26 at 23 36 54" src="https://user-images.githubusercontent.com/11541012/109346830-ad681980-788b-11eb-9793-891cb90fe332.png">
   
   <img width="450" alt="Screenshot 2021-02-26 at 23 37 38" src="https://user-images.githubusercontent.com/11541012/109346855-b48f2780-788b-11eb-9b06-e8162451f9b7.png">
   
   I don't think this is expected. And if I do select a JSON file at this step (instead of just giving a name for my selection to be added in a process group), the process group for my JSON file will be added instead of putting my selection into a process group.
   
   So... I guess it's just about removing the option to upload the file when we click on the "Group" button. It should only be available when drag-and-dropping a process group on the canvas. Thoughts?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583786018



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4187,205 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // get the contents of the InputStream as a String
+        String stringContent;
+        if (in != null) {
+            try {
+                stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+            } catch (IOException e) {
+                throw new IOException("Unable to read the InputStream", e);
+            }
+        } else {
+            logger.warn("The InputStream is null");
+            throw new NullPointerException("The InputStream is null");
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        if (stringContent.length() > 0) {
+            try {
+                deserializedSnapshot = MAPPER.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("Parsing uploaded JSON failed", jpe);
+                return Response.status(Response.Status.OK).entity(INVALID_JSON_RESPONSE).type("application/json").build();
+            } catch (IOException e) {
+                logger.warn("Deserialization of uploaded JSON failed", e);
+                throw new IOException("Deserialization of uploaded JSON failed", e);
+            }
+        } else {
+            logger.warn("The uploaded file was empty");
+            throw new IOException("The uploaded file was empty.");
+        }
+
+        // create a new ProcessGroupEntity
+        final ProcessGroupEntity newProcessGroupEntity = createProcessGroupEntity(groupId, groupName, positionX, positionY, deserializedSnapshot);
+
+        // replicate the request or call serviceFacade.updateProcessGroup
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST, newProcessGroupEntity);
+        } else if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(newProcessGroupEntity.isDisconnectedNodeAcknowledged());
+        }
+
+        return withWriteLock(
+                serviceFacade,
+                newProcessGroupEntity,
+                lookup -> {
+                    final NiFiUser user = NiFiUserUtils.getNiFiUser();
+                    final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+                    processGroup.authorize(authorizer, RequestAction.WRITE, user);
+
+                    // if request specifies a Parameter Context, need to authorize that user has READ policy for the Parameter Context.
+                    final ParameterContextReferenceEntity referencedParamContext = newProcessGroupEntity.getComponent().getParameterContext();
+                    if (referencedParamContext != null && referencedParamContext.getId() != null) {
+                        lookup.getParameterContext(referencedParamContext.getId()).authorize(authorizer, RequestAction.READ, user);
+                    }
+
+                    // if any of the components is a Restricted Component, then we must authorize the user
+                    // for write access to the RestrictedComponents resource
+                    final VersionedFlowSnapshot versionedFlowSnapshot = newProcessGroupEntity.getVersionedFlowSnapshot();
+                    if (versionedFlowSnapshot != null) {
+                        final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(versionedFlowSnapshot.getFlowContents(), serviceFacade);
+                        restrictedComponents.forEach(restrictedComponent -> {
+                            final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
+                            authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
+                        });
+
+                        final Map<String, VersionedParameterContext> parameterContexts = versionedFlowSnapshot.getParameterContexts();
+                        if (parameterContexts != null) {
+                            parameterContexts.values().forEach(context -> AuthorizeParameterReference.authorizeParameterContextAddition(context, serviceFacade, authorizer, lookup, user));
+                        }
+                    }
+                },
+                () -> {
+                    final VersionedFlowSnapshot versionedFlowSnapshot = newProcessGroupEntity.getVersionedFlowSnapshot();
+                    if (versionedFlowSnapshot != null) {
+                        serviceFacade.verifyComponentTypes(versionedFlowSnapshot.getFlowContents());
+                    }
+                },
+                processGroupEntity -> {
+                    final ProcessGroupDTO processGroup = processGroupEntity.getComponent();
+
+                    // set the processor id as appropriate
+                    processGroup.setId(generateUuid());
+
+                    // get the versioned flow
+                    final VersionedFlowSnapshot flowSnapshot = processGroupEntity.getVersionedFlowSnapshot();
+
+                    // create the process group contents
+                    final Revision revision = new Revision((long) 0, clientId, processGroup.getId());
+
+                    ProcessGroupEntity entity = serviceFacade.createProcessGroup(revision, groupId, processGroup);
+
+                    if (flowSnapshot != null) {
+                        final RevisionDTO revisionDto = entity.getRevision();
+                        final String newGroupId = entity.getComponent().getId();
+                        final Revision newGroupRevision = new Revision(revisionDto.getVersion(), revisionDto.getClientId(), newGroupId);
+
+                        // We don't want the Process Group's position to be updated because we want to keep the position where the user
+                        // placed the Process Group. We do not want to use the name of the Process Group that is in the Flow Contents.
+                        // To accomplish this, we call updateProcessGroupContents() passing 'false' for the updateSettings flag, set
+                        // the Process Group name, and null out the position.
+                        flowSnapshot.getFlowContents().setPosition(null);
+                        flowSnapshot.getFlowContents().setName(groupName);
+
+                        entity = serviceFacade.updateProcessGroupContents(newGroupRevision, newGroupId, null, flowSnapshot,
+                                getIdGenerationSeed().orElse(null), false, false, true);
+                    }
+
+                    populateRemainingProcessGroupEntityContent(entity);
+
+                    // generate a 201 created response
+                    String uri = entity.getUri();
+                    return generateCreatedResponse(URI.create(uri), entity).build();
+                }
+        );

Review comment:
       This block of code looks very similar to the implementation in the `createProcessGroup` method.  Did you consider refactoring both methods to minimize duplication?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {

Review comment:
       Thanks for adding the `NullPointException`.  To the second point of the comment, it should be possible to remove the call to `IOUtils.toString()` and pass the `InputStream` directly to `ObjectMapper.readValue()`.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4187,205 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // get the contents of the InputStream as a String
+        String stringContent;
+        if (in != null) {
+            try {
+                stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+            } catch (IOException e) {
+                throw new IOException("Unable to read the InputStream", e);
+            }
+        } else {
+            logger.warn("The InputStream is null");
+            throw new NullPointerException("The InputStream is null");
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        if (stringContent.length() > 0) {
+            try {
+                deserializedSnapshot = MAPPER.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("Parsing uploaded JSON failed", jpe);
+                return Response.status(Response.Status.OK).entity(INVALID_JSON_RESPONSE).type("application/json").build();

Review comment:
       The call to `type()` indicates that the string provided to `entity()` should be formatted as JSON, which corresponds to the `@Produces(MediaType.APPLICATION_JSON)` annotation on this method.
   
   On closer review, it looks like the status should be `Response.Status.BAD_REQUEST` to return an HTTP 400 instead of `Response.Status.OK`, otherwise wouldn't the JavaScript handling call success in this case?
   
   Stepping back for a moment, is there a reason for returning a response at this point instead of throwing an exception and having the framework return an appropriately formatted response?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583189069



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
##########
@@ -3766,7 +3765,7 @@ public void updateFlow(final VersionedFlowSnapshot proposedSnapshot, final Strin
                            final boolean updateDescendantVersionedFlows) {
         writeLock.lock();
         try {
-            verifyCanUpdate(proposedSnapshot, true, verifyNotDirty);
+             verifyCanUpdate(proposedSnapshot, true, verifyNotDirty);

Review comment:
       It looks like the changes in this file are just formatting related, should the changes be reverted?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
##########
@@ -15,9 +15,35 @@
  * limitations under the License.
  */
 package org.apache.nifi.web;
-
 import com.google.common.collect.Sets;
 import io.prometheus.client.CollectorRegistry;
+import java.io.IOException;

Review comment:
       Recommend reverting the changes to this file since it is only reordering imports.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";
+                return Response.status(Response.Status.OK).entity(responseJson).type("application/json").build();
+            } catch (IOException e) {

Review comment:
       This `IOException` is logged, but is not wrapped an thrown again.  As a result, following method calls could fail, so this Exception should be wrapped and a new Exception should be thrown.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";
+                return Response.status(Response.Status.OK).entity(responseJson).type("application/json").build();
+            } catch (IOException e) {
+                logger.warn("An error occurred while deserializing the flow version.", e);
+            }
+        } else {
+            logger.warn("The uploaded file was empty");
+            throw new IOException("The uploaded file was empty.");
+        }
+
+        sanitizeRegistryInfo(deserializedSnapshot.getFlowContents());
+
+        // resolve Bundle info
+        serviceFacade.discoverCompatibleBundles(deserializedSnapshot.getFlowContents());
+
+        // if there are any Controller Services referenced that are inherited from the parent group, resolve those to point to the appropriate Controller Service, if we are able to.
+        serviceFacade.resolveInheritedControllerServices(deserializedSnapshot, groupId, NiFiUserUtils.getNiFiUser());
+
+        // create a new ProcessGroupDTO
+        ProcessGroupDTO processGroupDTO = new ProcessGroupDTO();
+
+        processGroupDTO.setParentGroupId(groupId);
+        processGroupDTO.setName(groupName);
+
+        newProcessGroupEntity.setComponent(processGroupDTO);
+        newProcessGroupEntity.setVersionedFlowSnapshot(deserializedSnapshot);
+
+        // create a PositionDTO
+        PositionDTO positionDTO = new PositionDTO();
+        positionDTO.setX(positionX);
+        positionDTO.setY(positionY);
+        newProcessGroupEntity.getComponent().setPosition(positionDTO);

Review comment:
       Is it possible to refactor the creation of the ProcessGroupEntity into a separate method?  That would encapsulate the initial verification and building of the object prior to it being processed.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css
##########
@@ -286,45 +290,125 @@ span.details-title {
     height: 28px;
 }
 
-#template-file-field {
+#select-file-button button {
+    position: absolute;
+    float: right;
+    right: 4px;
+    top: 22px;
+    font-size: 18px;
+    color: #004849;
+    border: none;
+    background-color: transparent;
+}
+
+/*#select-file-button button:hover {*/

Review comment:
       Should this commented style be retained, or can it be removed?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");

Review comment:
       The check evaluates whether `groupId` is blank, but the error message seems to indicate that some other check is happening, should the error message be changed, or is some other check necessary?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {

Review comment:
       Should a `null` InputStream throw an Exception?  That would allow passing the InputStream directly to `ObjectMapper.readValue()` as opposed to reading it into a String.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);

Review comment:
       The `ObjectMapper` instance is thread-safe, so it could be configured as a member variable of `ProcessGroupResource` and these configuration methods can be run once instead of on each upload.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-group-component.js
##########
@@ -129,17 +133,78 @@
                  * Initialize the modal.
                  */
                 init: function () {
+                    var self = this;
+
+                    self.fileForm = $('#file-upload-form').ajaxForm({
+                        url: '../nifi-api/process-groups/',
+                        dataType: 'json',
+                        beforeSubmit: function ($form, options) {
+                            // ensure uploading to the current process group
+                            options.url += (encodeURIComponent(nfCanvasUtils.getGroupId()) + '/process-groups/upload');
+                        }
+                    });
+
                     // configure the new process group dialog
                     this.getElement().modal({
                         scrollableContentStyle: 'scrollable',
                         headerText: 'Add Process Group',
                         handler: {
                             close: function () {
+                                self.fileToBeUploaded = null;
+                                $('#selected-file-name').text('');
+                                $('#upload-file-field').val('');
                                 $('#new-process-group-name').val('');
                                 $('#new-process-group-dialog').removeData('pt');
+
+                                // reset the form to ensure that the change fire will fire
+                                self.fileForm.resetForm();
                             }
                         }
                     });
+
+                    $('#upload-file-field-button').on('click', function (e) {
+                        $('#upload-file-field').click();
+                    });
+
+                    $('#upload-file-field').on('change', function (e) {
+                        $('#upload-file-field-button').hide();
+
+                        self.fileToBeUploaded = e.target;
+                        var filename = $(this).val();
+                        var filenameExtension;
+                        if (!nfCommon.isBlank(filename)) {
+                            filenameExtension = filename.replace(/^.*[\\\/]/, '');
+                            filename = filename.replace(/^.*[\\\/]/, '').replace(/\..*/, '');

Review comment:
       Should these regular expression replacements be moved into a separate function that describes what is happening?  Otherwise the purpose is not quite clear.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";

Review comment:
       This could be changed to a `private static final String` since it is always the same.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";
+                return Response.status(Response.Status.OK).entity(responseJson).type("application/json").build();
+            } catch (IOException e) {
+                logger.warn("An error occurred while deserializing the flow version.", e);

Review comment:
       This error message references a "flow version" but it should be more general.
   ```suggestion
                   logger.warn("Deserialization of uploaded JSON failed", e);
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-group-component.js
##########
@@ -304,7 +461,7 @@
                     });
                 }).promise();
             }
-        }
+        };

Review comment:
       Is there a reason for adding this semicolon?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);

Review comment:
       The log message seems to indicate that there should be more information, recommend adjusting the invocation:
   ```suggestion
                   logger.warn("Parsing uploaded JSON failed", jpe);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] pvillard31 commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-789616099


   Merged (and took care of reverting import ordering changes)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786770072


   My last commit includes the disabled buttons, which completes this PR and is ready for merge once it's approved. Thanks.
   @pvillard31 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r585908114



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4158,264 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("positionX") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("positionY") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws InterruptedException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // deserialize InputStream to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        try {
+            deserializedSnapshot = MAPPER.readValue(in, VersionedFlowSnapshot.class);
+        } catch (IOException e) {
+            logger.warn("Deserialization of uploaded JSON failed", e);
+            throw new IllegalArgumentException("Deserialization of uploaded JSON failed", e);
+        }
+
+        // clear Registry info
+        sanitizeRegistryInfo(deserializedSnapshot.getFlowContents());
+
+        // resolve Bundle info
+        serviceFacade.discoverCompatibleBundles(deserializedSnapshot.getFlowContents());
+
+        // if there are any Controller Services referenced that are inherited from the parent group,
+        // resolve those to point to the appropriate Controller Service, if we are able to.
+        serviceFacade.resolveInheritedControllerServices(deserializedSnapshot, groupId, NiFiUserUtils.getNiFiUser());
+
+        if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(disconnectedNodeAcknowledged);
+        }
+
+        // build the response entity for a replicate request
+        ProcessGroupUploadEntity pgUploadEntity = new ProcessGroupUploadEntity();
+        pgUploadEntity.setId(groupId);
+        pgUploadEntity.setGroupName(groupName);
+        pgUploadEntity.setPositionX(positionX);
+        pgUploadEntity.setPositionY(positionY);
+        pgUploadEntity.setClientId(clientId);
+        pgUploadEntity.setDisconnectedNodeAcknowledged(disconnectedNodeAcknowledged);
+        pgUploadEntity.setFlowSnapshot(deserializedSnapshot);
+
+        // replicate the request
+        if (isReplicateRequest()) {
+            // convert request accordingly
+            final UriBuilder uriBuilder = uriInfo.getBaseUriBuilder();
+            uriBuilder.segment("process-groups", groupId, "process-groups", "import");
+            final URI importUri = uriBuilder.build();
+
+            final Map<String, String> headersToOverride = new HashMap<>();
+            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
+
+            // Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly
+            // to the cluster nodes themselves.
+            if (getReplicationTarget() == ReplicationTarget.CLUSTER_NODES) {
+                return getRequestReplicator().replicate(HttpMethod.POST, importUri, pgUploadEntity, getHeaders(headersToOverride)).awaitMergedResponse().getResponse();
+            } else {
+                return getRequestReplicator().forwardToCoordinator(
+                        getClusterCoordinatorNode(), HttpMethod.POST, importUri, pgUploadEntity, getHeaders(headersToOverride)).awaitMergedResponse().getResponse();
+            }
+        }
+
+        // otherwise import the process group locally
+        return importProcessGroup(httpServletRequest, groupId, pgUploadEntity);
+    }
+
+    /**
+     * Imports the specified process group.
+     *
+     * @param httpServletRequest request
+     * @param processGroupUploadEntity     A ProcessGroupUploadEntity.
+     * @return A processGroupEntity.
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/import")
+    @ApiOperation(
+            value = "Imports a specified process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 = 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 importProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,

Review comment:
       Is this parameter necessary? It does not appear to be used.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r584764372



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4157,163 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // deserialize InputStream to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        try {
+            deserializedSnapshot = MAPPER.readValue(in, VersionedFlowSnapshot.class);
+        } catch (IOException e) {
+            logger.warn("Deserialization of uploaded JSON failed", e);
+            throw new IllegalArgumentException("Deserialization of uploaded JSON failed", e);
+        }
+
+        // create a PositionDTO
+        final PositionDTO positionDTO = new PositionDTO();
+        positionDTO.setX(positionX);
+        positionDTO.setY(positionY);
+
+        // create a new ProcessGroupEntity
+        final ProcessGroupEntity newProcessGroupEntity = createProcessGroupEntity(groupId, groupName, positionDTO, deserializedSnapshot);
+
+        // replicate the request or call serviceFacade.updateProcessGroup
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST, newProcessGroupEntity);

Review comment:
       Unfortunately, this doesn't work when clustered. Here, it is attempting to replicate a POST method by provided an entity. But the receiving method (this method) does not accept an entity - it expects form data and an input stream. Please take a look at the uploadTemplate() method to see how this is handled there. Rather than calling the `replicate` method, it gets the `requestReplicator` and replicates the request itself.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583776586



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);

Review comment:
       Moved `ObjectMapper` and its configurations to be a member variable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r586001406



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupUploadEntity.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ProcessGroupUploadDTO.
+ */
+@XmlRootElement(name = "processGroupUploadEntity")
+public class ProcessGroupUploadEntity extends Entity {
+    private String id;
+    private String groupName;
+    private Double positionX;
+    private Double positionY;
+    private String clientId;
+    private Boolean disconnectedNodeAcknowledged;
+    private VersionedFlowSnapshot versionedFlowSnapshot;
+
+    /**
+     * @return The group ID
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {

Review comment:
       If this is the ID of the process group, and not the ID of the Upload Entity, I would call this getGroupId/setGroupId.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupUploadEntity.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ProcessGroupUploadDTO.
+ */
+@XmlRootElement(name = "processGroupUploadEntity")
+public class ProcessGroupUploadEntity extends Entity {
+    private String id;
+    private String groupName;
+    private Double positionX;
+    private Double positionY;
+    private String clientId;
+    private Boolean disconnectedNodeAcknowledged;
+    private VersionedFlowSnapshot versionedFlowSnapshot;
+
+    /**
+     * @return The group ID
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return The process group name
+     */
+    public String getGroupName() {
+        return groupName;
+    }
+
+    public void setGroupName(String groupName) {
+        this.groupName = groupName;
+    }
+
+    /**
+     * @return The process group X-position
+     */
+    public Double getPositionX() {
+        return positionX;
+    }
+
+    public void setPositionX(Double positionX) {
+        this.positionX = positionX;
+    }
+
+    /**
+     * @return The process group Y-position
+     */
+    public Double getPositionY() {
+        return positionY;
+    }
+
+    public void setPositionY(Double positionY) {
+        this.positionY = positionY;
+    }
+
+    /**
+     * @return The client ID
+     */
+    public String getClientId() {

Review comment:
       The Client ID is never really used on its own like this throughout the data model. Client ID is always part of a RevisionDTO. Recommend changing this to be a RevisionDTO. You can see in the ProcessGroupResource.createProcessGroup (or most any other createXYZ method in the *Resource classes) the pattern that is followed:
   ```
           if (requestProcessGroupEntity.getRevision() == null || (requestProcessGroupEntity.getRevision().getVersion() == null || requestProcessGroupEntity.getRevision().getVersion() != 0)) {
               throw new IllegalArgumentException("A revision of 0 must be specified when creating a new Process group.");
           }
   ```
   
   So it enforces that the revision be specified, that it have a version, and that the version be 0, in order to create the object. I'm personally not a huge fan of requiring that the field exists and be set to 0, but it's the pattern that is followed everywhere else, so probably best to be consistent.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupUploadEntity.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ProcessGroupUploadDTO.
+ */
+@XmlRootElement(name = "processGroupUploadEntity")
+public class ProcessGroupUploadEntity extends Entity {
+    private String id;
+    private String groupName;
+    private Double positionX;
+    private Double positionY;
+    private String clientId;
+    private Boolean disconnectedNodeAcknowledged;
+    private VersionedFlowSnapshot versionedFlowSnapshot;
+
+    /**
+     * @return The group ID
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return The process group name
+     */
+    public String getGroupName() {
+        return groupName;
+    }
+
+    public void setGroupName(String groupName) {
+        this.groupName = groupName;
+    }
+
+    /**
+     * @return The process group X-position
+     */
+    public Double getPositionX() {

Review comment:
       There exists a PositionDTO that is preferable here, instead of get/set PositionX/PositionY.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r586062548



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupUploadEntity.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ProcessGroupUploadDTO.
+ */
+@XmlRootElement(name = "processGroupUploadEntity")
+public class ProcessGroupUploadEntity extends Entity {
+    private String id;
+    private String groupName;
+    private Double positionX;
+    private Double positionY;
+    private String clientId;
+    private Boolean disconnectedNodeAcknowledged;
+    private VersionedFlowSnapshot versionedFlowSnapshot;
+
+    /**
+     * @return The group ID
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return The process group name
+     */
+    public String getGroupName() {
+        return groupName;
+    }
+
+    public void setGroupName(String groupName) {
+        this.groupName = groupName;
+    }
+
+    /**
+     * @return The process group X-position
+     */
+    public Double getPositionX() {

Review comment:
       Created a PositionDTO with the x/y positions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r585936826



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4158,264 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("positionX") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("positionY") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws InterruptedException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // deserialize InputStream to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        try {
+            deserializedSnapshot = MAPPER.readValue(in, VersionedFlowSnapshot.class);
+        } catch (IOException e) {
+            logger.warn("Deserialization of uploaded JSON failed", e);
+            throw new IllegalArgumentException("Deserialization of uploaded JSON failed", e);
+        }
+
+        // clear Registry info
+        sanitizeRegistryInfo(deserializedSnapshot.getFlowContents());
+
+        // resolve Bundle info
+        serviceFacade.discoverCompatibleBundles(deserializedSnapshot.getFlowContents());
+
+        // if there are any Controller Services referenced that are inherited from the parent group,
+        // resolve those to point to the appropriate Controller Service, if we are able to.
+        serviceFacade.resolveInheritedControllerServices(deserializedSnapshot, groupId, NiFiUserUtils.getNiFiUser());
+
+        if (isDisconnectedFromCluster()) {
+            verifyDisconnectedNodeModification(disconnectedNodeAcknowledged);
+        }
+
+        // build the response entity for a replicate request
+        ProcessGroupUploadEntity pgUploadEntity = new ProcessGroupUploadEntity();
+        pgUploadEntity.setId(groupId);
+        pgUploadEntity.setGroupName(groupName);
+        pgUploadEntity.setPositionX(positionX);
+        pgUploadEntity.setPositionY(positionY);
+        pgUploadEntity.setClientId(clientId);
+        pgUploadEntity.setDisconnectedNodeAcknowledged(disconnectedNodeAcknowledged);
+        pgUploadEntity.setFlowSnapshot(deserializedSnapshot);
+
+        // replicate the request
+        if (isReplicateRequest()) {
+            // convert request accordingly
+            final UriBuilder uriBuilder = uriInfo.getBaseUriBuilder();
+            uriBuilder.segment("process-groups", groupId, "process-groups", "import");
+            final URI importUri = uriBuilder.build();
+
+            final Map<String, String> headersToOverride = new HashMap<>();
+            headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
+
+            // Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly
+            // to the cluster nodes themselves.
+            if (getReplicationTarget() == ReplicationTarget.CLUSTER_NODES) {
+                return getRequestReplicator().replicate(HttpMethod.POST, importUri, pgUploadEntity, getHeaders(headersToOverride)).awaitMergedResponse().getResponse();
+            } else {
+                return getRequestReplicator().forwardToCoordinator(
+                        getClusterCoordinatorNode(), HttpMethod.POST, importUri, pgUploadEntity, getHeaders(headersToOverride)).awaitMergedResponse().getResponse();
+            }
+        }
+
+        // otherwise import the process group locally
+        return importProcessGroup(httpServletRequest, groupId, pgUploadEntity);
+    }
+
+    /**
+     * Imports the specified process group.
+     *
+     * @param httpServletRequest request
+     * @param processGroupUploadEntity     A ProcessGroupUploadEntity.
+     * @return A processGroupEntity.
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_JSON)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/import")
+    @ApiOperation(
+            value = "Imports a specified process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 = 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 importProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,

Review comment:
       Removed - not necessary.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583773455



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";
+                return Response.status(Response.Status.OK).entity(responseJson).type("application/json").build();
+            } catch (IOException e) {

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786767902


   @exceptionfactory Thanks for reviewing! I've addressed your feedback and committed the changes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583772280



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);

Review comment:
       Revised the message.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow files.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786045792


   The PR is ready for initial review. The overall functionality works. I have one more UI enhancement that I'm working on now - it's for disabling the "Add Process Group" dialog "Add" button until the dialog is ready for submission.
   cc: @pvillard31 @mcgilman 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786770245


   > @exceptionfactory Thanks for reviewing! I've addressed your feedback and committed the changes.
   
   Thanks for making the changes @mtien-apache.  I will take another look following the updates. I noticed that `StandardProcessGroup`, `NiFiServiceFacade`, and `StandardNiFiServiceFacade` still show up with reordered imports, can you remove those changes?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4846:
URL: https://github.com/apache/nifi/pull/4846


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r586062768



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupUploadEntity.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ProcessGroupUploadDTO.
+ */
+@XmlRootElement(name = "processGroupUploadEntity")
+public class ProcessGroupUploadEntity extends Entity {
+    private String id;
+    private String groupName;
+    private Double positionX;
+    private Double positionY;
+    private String clientId;
+    private Boolean disconnectedNodeAcknowledged;
+    private VersionedFlowSnapshot versionedFlowSnapshot;
+
+    /**
+     * @return The group ID
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {

Review comment:
       Changed to getGroupId and setGroupId.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r586063225



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupUploadEntity.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ProcessGroupUploadDTO.
+ */
+@XmlRootElement(name = "processGroupUploadEntity")
+public class ProcessGroupUploadEntity extends Entity {
+    private String id;
+    private String groupName;
+    private Double positionX;
+    private Double positionY;
+    private String clientId;
+    private Boolean disconnectedNodeAcknowledged;
+    private VersionedFlowSnapshot versionedFlowSnapshot;
+
+    /**
+     * @return The group ID
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return The process group name
+     */
+    public String getGroupName() {
+        return groupName;
+    }
+
+    public void setGroupName(String groupName) {
+        this.groupName = groupName;
+    }
+
+    /**
+     * @return The process group X-position
+     */
+    public Double getPositionX() {
+        return positionX;
+    }
+
+    public void setPositionX(Double positionX) {
+        this.positionX = positionX;
+    }
+
+    /**
+     * @return The process group Y-position
+     */
+    public Double getPositionY() {
+        return positionY;
+    }
+
+    public void setPositionY(Double positionY) {
+        this.positionY = positionY;
+    }
+
+    /**
+     * @return The client ID
+     */
+    public String getClientId() {

Review comment:
       Changed to a RevisionDTO and kept the version to be 0 to follow conventions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786191291


   @pvillard31 Good catch - I just pushed a commit that should fix this. If uploading a flow that was downloaded from the canvas, the flow does not contain snapshot metadata. I was trying to access the metadata, but it's not necessary. Thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583774545



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-group-component.js
##########
@@ -129,17 +133,78 @@
                  * Initialize the modal.
                  */
                 init: function () {
+                    var self = this;
+
+                    self.fileForm = $('#file-upload-form').ajaxForm({
+                        url: '../nifi-api/process-groups/',
+                        dataType: 'json',
+                        beforeSubmit: function ($form, options) {
+                            // ensure uploading to the current process group
+                            options.url += (encodeURIComponent(nfCanvasUtils.getGroupId()) + '/process-groups/upload');
+                        }
+                    });
+
                     // configure the new process group dialog
                     this.getElement().modal({
                         scrollableContentStyle: 'scrollable',
                         headerText: 'Add Process Group',
                         handler: {
                             close: function () {
+                                self.fileToBeUploaded = null;
+                                $('#selected-file-name').text('');
+                                $('#upload-file-field').val('');
                                 $('#new-process-group-name').val('');
                                 $('#new-process-group-dialog').removeData('pt');
+
+                                // reset the form to ensure that the change fire will fire
+                                self.fileForm.resetForm();
                             }
                         }
                     });
+
+                    $('#upload-file-field-button').on('click', function (e) {
+                        $('#upload-file-field').click();
+                    });
+
+                    $('#upload-file-field').on('change', function (e) {
+                        $('#upload-file-field-button').hide();
+
+                        self.fileToBeUploaded = e.target;
+                        var filename = $(this).val();
+                        var filenameExtension;
+                        if (!nfCommon.isBlank(filename)) {
+                            filenameExtension = filename.replace(/^.*[\\\/]/, '');
+                            filename = filename.replace(/^.*[\\\/]/, '').replace(/\..*/, '');

Review comment:
       Revised - I moved to two separate methods describing its functions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] joewitt commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow files.

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-786059719


   When I read this PR title I was thinking this was a very different thing.  Reading the JIRA I think it is more specific to being able to upload a 'flow definition' which will create that flow on the NiFi it is uploaded to.  Can we please clarify the description?
   
   The term 'flow file' is a specific thing in NiFi and it means 'data in the flow' as represented by a set of attributes and content.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583775267



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {
+            stringContent = IOUtils.toString(in, StandardCharsets.UTF_8);
+        }
+
+        // deserialize content to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot = null;
+        final ObjectMapper mapper = new ObjectMapper();
+
+        if (stringContent.length() > 0) {
+            try {
+                mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+                mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+                mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+                mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+                deserializedSnapshot = mapper.readValue(stringContent, VersionedFlowSnapshot.class);
+            } catch (JsonParseException jpe) {
+                logger.warn("The file uploaded is not a valid JSON format: ", jpe);
+                String responseJson = "The specified file is not a valid JSON format.";
+                return Response.status(Response.Status.OK).entity(responseJson).type("application/json").build();
+            } catch (IOException e) {
+                logger.warn("An error occurred while deserializing the flow version.", e);

Review comment:
       Revised.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r583779054



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4179,227 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow file and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow file stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow file and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id must be the same as specified in the URI.");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // create a new process group
+        final ProcessGroupEntity newProcessGroupEntity = new ProcessGroupEntity();
+
+        // get the contents of the InputStream as a String
+        String stringContent = null;
+        if (in != null) {

Review comment:
       Fixed - a `NullPointerException` can now be thrown.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
##########
@@ -15,9 +15,35 @@
  * limitations under the License.
  */
 package org.apache.nifi.web;
-
 import com.google.common.collect.Sets;
 import io.prometheus.client.CollectorRegistry;
+import java.io.IOException;

Review comment:
       Reverted.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on a change in pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on a change in pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#discussion_r585892971



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
##########
@@ -4173,6 +4157,163 @@ private void sanitizeRegistryInfo(final VersionedProcessGroup versionedProcessGr
         }
     }
 
+    /**
+     * Uploads the specified versioned flow definition and adds it to a new process group.
+     *
+     * @param httpServletRequest request
+     * @param in The flow definition stream
+     * @return A processGroupEntity
+     * @throws IOException if there is an error during deserialization of the InputStream
+     */
+    @POST
+    @Consumes(MediaType.MULTIPART_FORM_DATA)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("{id}/process-groups/upload")
+    @ApiOperation(
+            value = "Uploads a versioned flow definition and creates a process group",
+            response = ProcessGroupEntity.class,
+            authorizations = {
+                    @Authorization(value = "Write - /process-groups/{uuid}")
+            }
+    )
+    @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 uploadProcessGroup(
+            @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The process group id.",
+                    required = true
+            )
+            @PathParam("id") final String groupId,
+            @ApiParam(
+                    value = "The process group name.",
+                    required = true
+            )
+            @FormDataParam("groupName") final String groupName,
+            @ApiParam(
+                    value = "The process group X position.",
+                    required = true
+            )
+            @FormDataParam("position-x") final Double positionX,
+            @ApiParam(
+                    value = "The process group Y position.",
+                    required = true
+            )
+            @FormDataParam("position-y") final Double positionY,
+            @ApiParam(
+                    value = "The client id.",
+                    required = true
+            )
+            @FormDataParam("clientId") final String clientId,
+            @ApiParam(
+                    value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.",
+                    required = false
+            )
+            @FormDataParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged,
+            @FormDataParam("file") final InputStream in) throws IOException {
+
+        // ensure the group name is specified
+        if (StringUtils.isBlank(groupName)) {
+            throw new IllegalArgumentException("The process group name is required.");
+        }
+
+        if (StringUtils.isBlank(groupId)) {
+            throw new IllegalArgumentException("The parent process group id is required");
+        }
+
+        if (positionX == null) {
+            throw new IllegalArgumentException("The x coordinate of the proposed position must be specified.");
+        }
+
+        if (positionY == null) {
+            throw new IllegalArgumentException("The y coordinate of the proposed position must be specified.");
+        }
+
+        if (StringUtils.isBlank(clientId)) {
+            throw new IllegalArgumentException("The client id must be specified");
+        }
+
+        // deserialize InputStream to a VersionedFlowSnapshot
+        VersionedFlowSnapshot deserializedSnapshot;
+
+        try {
+            deserializedSnapshot = MAPPER.readValue(in, VersionedFlowSnapshot.class);
+        } catch (IOException e) {
+            logger.warn("Deserialization of uploaded JSON failed", e);
+            throw new IllegalArgumentException("Deserialization of uploaded JSON failed", e);
+        }
+
+        // create a PositionDTO
+        final PositionDTO positionDTO = new PositionDTO();
+        positionDTO.setX(positionX);
+        positionDTO.setY(positionY);
+
+        // create a new ProcessGroupEntity
+        final ProcessGroupEntity newProcessGroupEntity = createProcessGroupEntity(groupId, groupName, positionDTO, deserializedSnapshot);
+
+        // replicate the request or call serviceFacade.updateProcessGroup
+        if (isReplicateRequest()) {
+            return replicate(HttpMethod.POST, newProcessGroupEntity);

Review comment:
       @markap14 Thanks for reviewing and catching this. I just pushed a commit that added a second endpoint to handle a cluster replicate request. Can you review again? Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mtien-apache commented on pull request #4846: NIFI-8260 Implement an Upload File capability in the NiFi UI for flow definitions.

Posted by GitBox <gi...@apache.org>.
mtien-apache commented on pull request #4846:
URL: https://github.com/apache/nifi/pull/4846#issuecomment-789382845


   @markap14 I made revisions from your feedback. Thanks for looking at this in detail!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org