[
https://issues.apache.org/jira/browse/FLINK-7713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16360615#comment-16360615
]
ASF GitHub Bot commented on FLINK-7713:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/5442#discussion_r167523546
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java
---
@@ -0,0 +1,147 @@
+/*
+ * 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.flink.runtime.rest;
+
+import
org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import
org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPipeline;
+import
org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpContent;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObject;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DiskFileUpload;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpDataFactory;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.InterfaceHttpData;
+import org.apache.flink.shaded.netty4.io.netty.util.AttributeKey;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.UUID;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Writes multipart/form-data to disk. Delegates all other requests to the
next
+ * {@link ChannelInboundHandler} in the {@link ChannelPipeline}.
+ */
+public class FileUploadHandler extends
SimpleChannelInboundHandler<HttpObject> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(FileUploadHandler.class);
+
+ static final AttributeKey<Path> UPLOADED_FILE =
AttributeKey.valueOf("UPLOADED_FILE");
+
+ private static final HttpDataFactory DATA_FACTORY = new
DefaultHttpDataFactory(true);
+
+ private final Path uploadDir;
+
+ private HttpPostRequestDecoder currentHttpPostRequestDecoder;
+
+ private HttpRequest currentHttpRequest;
+
+ public FileUploadHandler(final Path uploadDir) {
+ super(false);
+
+ this.uploadDir = requireNonNull(uploadDir);
+ DiskFileUpload.baseDirectory =
uploadDir.toAbsolutePath().toString();
+ }
+
+ @Override
+ protected void channelRead0(final ChannelHandlerContext ctx, final
HttpObject msg) throws Exception {
+ if (msg instanceof HttpRequest) {
+ final HttpRequest httpRequest = (HttpRequest) msg;
+ if (httpRequest.getMethod().equals(HttpMethod.POST)) {
+ final HttpPostRequestDecoder
httpPostRequestDecoder = new HttpPostRequestDecoder(DATA_FACTORY, httpRequest);
+ if (httpPostRequestDecoder.isMultipart()) {
+ currentHttpPostRequestDecoder =
httpPostRequestDecoder;
+ currentHttpRequest = httpRequest;
+ } else {
+ ctx.fireChannelRead(msg);
--- End diff --
How do we handle non multipart uploads? Do they exist?
> Port JarUploadHandler to new REST endpoint
> ------------------------------------------
>
> Key: FLINK-7713
> URL: https://issues.apache.org/jira/browse/FLINK-7713
> Project: Flink
> Issue Type: Sub-task
> Components: Distributed Coordination, REST, Webfrontend
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Gary Yao
> Priority: Major
> Labels: flip-6
> Fix For: 1.5.0
>
>
> Port {{JarUploadHandler}} to new REST endpoint.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)