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

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

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

    https://github.com/apache/flink/pull/6178#discussion_r196455211
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java 
---
    @@ -103,77 +104,74 @@ protected void respondAsLeader(ChannelHandlerContext 
ctx, RoutedRequest routedRe
                                return;
                        }
     
    -                   ByteBuf msgContent = ((FullHttpRequest) 
httpRequest).content();
    +                   final ByteBuf msgContent;
    +                   Optional<byte[]> multipartJsonPayload = 
FileUploadHandler.getMultipartJsonPayload(ctx);
    +                   if (multipartJsonPayload.isPresent()) {
    +                           msgContent = 
Unpooled.wrappedBuffer(multipartJsonPayload.get());
    +                   } else {
    +                           msgContent = ((FullHttpRequest) 
httpRequest).content();
    +                   }
     
    -                   R request;
    -                   if (isFileUpload()) {
    -                           final Path path = 
ctx.channel().attr(FileUploadHandler.UPLOADED_FILE).get();
    -                           if (path == null) {
    -                                   HandlerUtils.sendErrorResponse(
    -                                           ctx,
    -                                           httpRequest,
    -                                           new ErrorResponseBody("Client 
did not upload a file."),
    -                                           HttpResponseStatus.BAD_REQUEST,
    -                                           responseHeaders);
    -                                   return;
    -                           }
    -                           //noinspection unchecked
    -                           request = (R) new FileUpload(path);
    -                   } else if (msgContent.capacity() == 0) {
    -                           try {
    -                                   request = MAPPER.readValue("{}", 
untypedResponseMessageHeaders.getRequestClass());
    -                           } catch (JsonParseException | 
JsonMappingException je) {
    -                                   log.error("Request did not conform to 
expected format.", je);
    -                                   HandlerUtils.sendErrorResponse(
    -                                           ctx,
    -                                           httpRequest,
    -                                           new ErrorResponseBody("Bad 
request received."),
    -                                           HttpResponseStatus.BAD_REQUEST,
    -                                           responseHeaders);
    -                                   return;
    +                   try (FileUploads uploadedFiles = 
FileUploadHandler.getMultipartFileUploads(ctx)) {
    --- End diff --
    
    I would obtain the upload directory from `FileUploadHandler` and simply 
delete this directory after the call has been processed. We could, then also 
create `FileUploads` outside of the `FileUploadHandler` to instantiate a 
`HandlerRequest` with it. This would also simplify the `FileUploads` class 
significantly, because it is no longer responsible for deleting the files.


> 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