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

ASF GitHub Bot commented on FLINK-9599:
---------------------------------------

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

    https://github.com/apache/flink/pull/6178#discussion_r196667205
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java
 ---
    @@ -95,14 +107,22 @@ protected void channelRead0(final 
ChannelHandlerContext ctx, final HttpObject ms
                                        final DiskFileUpload fileUpload = 
(DiskFileUpload) data;
                                        checkState(fileUpload.isCompleted());
     
    -                                   final Path dest = 
uploadDir.resolve(Paths.get(UUID.randomUUID() +
    -                                           "_" + 
fileUpload.getFilename()));
    +                                   final Path dest = 
currentUploadDir.resolve(fileUpload.getFilename());
                                        fileUpload.renameTo(dest.toFile());
    -                                   
ctx.channel().attr(UPLOADED_FILE).set(dest);
    +                           } else if (data.getHttpDataType() == 
InterfaceHttpData.HttpDataType.Attribute) {
    +                                   final Attribute request = (Attribute) 
data;
    +                                   // this could also be implemented by 
using the first found Attribute as the payload
    +                                   if 
(data.getName().equals(HTTP_ATTRIBUTE_REQUEST)) {
    +                                           currentJsonPayload = 
request.get();
    +                                   } else {
    +                                           LOG.warn("Received unknown 
attribute {}, will be ignored.", data.getName());
    +                                   }
                                }
                        }
     
                        if (httpContent instanceof LastHttpContent) {
    +                           ctx.channel().attr(UPLOADED_FILES).set(new 
FileUploads(Collections.singleton(currentUploadDir)));
    +                           
ctx.channel().attr(UPLOADED_JSON).set(currentJsonPayload);
    --- End diff --
    
    I can try this (it would be neat to remove the special case in 
`AbstractHandler`, but I'm wondering whether we can "simply" replace the 
payload of the multipart request (as identified by the headers that we also 
forward) with plain json.


> Implement generic mechanism to receive files via rest
> -----------------------------------------------------
>
>                 Key: FLINK-9599
>                 URL: https://issues.apache.org/jira/browse/FLINK-9599
>             Project: Flink
>          Issue Type: New Feature
>          Components: REST
>            Reporter: Chesnay Schepler
>            Assignee: Chesnay Schepler
>            Priority: Major
>             Fix For: 1.6.0
>
>
> As a prerequisite for a cleaner implementation of FLINK-9280 we should
>  * extend the RestClient to allow the upload of Files
>  * extend FileUploadHandler to accept mixed multi-part requests (json + files)
>  * generalize mechanism for accessing uploaded files in {{AbstractHandler}}
> Uploaded files can be forwarded to subsequent handlers as an attribute, 
> similar to the existing special case for the {{JarUploadHandler}}. The JSON 
> body can be forwarded by replacing the incoming http requests with a simple 
> {{DefaultFullHttpRequest}}.
> Uploaded files will be retrievable through the {{HandlerRequest}}.
> I'm not certain if/how we can document that a handler accepts files.



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

Reply via email to