[ 
https://issues.apache.org/jira/browse/NIFI-3469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16620797#comment-16620797
 ] 

ASF GitHub Bot commented on NIFI-3469:
--------------------------------------

Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2991#discussion_r218868215
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
 ---
    @@ -521,161 +553,221 @@ public void onTrigger(final ProcessContext context, 
final ProcessSession session
     
             final long start = System.nanoTime();
             final HttpServletRequest request = container.getRequest();
    -        FlowFile flowFile = session.create();
    -        try (OutputStream flowFileOut = session.write(flowFile)) {
    -            StreamUtils.copy(request.getInputStream(), flowFileOut);
    -        } catch (final IOException e) {
    -            // There may be many reasons which can produce an IOException 
on the HTTP stream and in some of them, eg.
    -            // bad requests, the connection to the client is not closed. 
In order to address also these cases, we try
    -            // and answer with a BAD_REQUEST, which lets the client know 
that the request has not been correctly
    -            // processed and makes it aware that the connection can be 
closed.
    -            getLogger().error("Failed to receive content from HTTP Request 
from {} due to {}",
    -                    new Object[]{request.getRemoteAddr(), e});
    -            session.remove(flowFile);
     
    -            try {
    -                HttpServletResponse response = container.getResponse();
    -                response.sendError(Status.BAD_REQUEST.getStatusCode());
    -                response.flushBuffer();
    -                container.getContext().complete();
    -            } catch (final IOException ioe) {
    -                getLogger().warn("Failed to send HTTP response to {} due 
to {}",
    -                        new Object[]{request.getRemoteAddr(), ioe});
    +        if (!Strings.isNullOrEmpty(request.getContentType()) && 
request.getContentType().contains(MIME_TYPE__MULTIPART_FORM_DATA)) {
    +          final long maxRequestSize = 
context.getProperty(MAX_REQUEST_SIZE).asLong();
    +          request.setAttribute(Request.__MULTIPART_CONFIG_ELEMENT, new 
MultipartConfigElement("/tmp", maxRequestSize, maxRequestSize, 0));
    --- End diff --
    
    This also opens up a lot of security concerns. We need to be very careful 
about how we handle, sanitize, trust, store, and display this data. 
    
    Some good starting places for reading:
    * https://www.owasp.org/index.php/Deserialization_of_untrusted_data
    * https://www.owasp.org/index.php/Unrestricted_File_Upload
    * https://www.owasp.org/index.php/Insecure_Temporary_File
    * https://www.owasp.org/index.php/Protect_FileUpload_Against_Malicious_File


> Add multipart request support to HandleHttpRequest Processor
> ------------------------------------------------------------
>
>                 Key: NIFI-3469
>                 URL: https://issues.apache.org/jira/browse/NIFI-3469
>             Project: Apache NiFi
>          Issue Type: Improvement
>          Components: Extensions
>            Reporter: Koji Kawamura
>            Assignee: Endre Kovacs
>            Priority: Major
>
> Currently, HandleHttpRequest outputs a single FlowFile containing all 
> multipart values as following:
> {code}
> --------------------------ef07e8bf36c274d3
> Content-Disposition: form-data; name="p1"
> v1
> --------------------------ef07e8bf36c274d3
> Content-Disposition: form-data; name="p2"
> v2
> --------------------------ef07e8bf36c274d3--
> {code}
> Many users requested adding upload files support to NiFi.
> In order for HandleHttpRequest to support multipart data we need to add 
> followings (this is based on a brief researching and can be more complex or 
> simple):
> We need to use HttpServletRequest#getParts() as written in this stackoverflow 
> thread:
> http://stackoverflow.com/questions/3337056/convenient-way-to-parse-incoming-multipart-form-data-parameters-in-a-servlet
> Also, we probably need a custom MultiPartInputStreamParser implementation. 
> Because Jetty's default implementation writes input data to temporary 
> directory on file system, instead, we'd like NiFi to write those into output 
> FlowFiles content in streaming fashion.
> And we need request size validation checks, threshold for those validation 
> should be passed via javax.servlet.MultipartConfigElement.
> Finally, we have to do something with HandleHttpResponse processor.
> Once HandleHttpRequest processor start splitting incoming request into 
> multiple output FlowFiles, we need to wait for every fragment to be 
> processed, then execute HandleHttpRequest.
> I think Wait/Notify processors (available from next version) will be helpful 
> here.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to