Skip to content

Commit

Permalink
[hotfix][runtime] A customized filename can be specified through Cont…
Browse files Browse the repository at this point in the history
…ent-Disposition that also allows passing of path information which was not properly handled. This is fixed now.

We just use the filename instead of interpreting any path information that was passed through a custom filename. Two tests were added to verify the proper behavior:
1. a custom filename without path information was used
2. a custom filename with path information was used

The change required adapting the MultipartUploadResource in a way that it is used not as a @ClassRule but as a @rule instead. This enables us to initialize it differently on a per-test level. The change makes the verification of the uploaded files configurable.
  • Loading branch information
XComp authored and rmetzger committed Nov 18, 2020
1 parent b561010 commit a5264a6
Show file tree
Hide file tree
Showing 3 changed files with 257 additions and 121 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@

import javax.annotation.Nullable;

import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
Expand Down Expand Up @@ -138,9 +139,11 @@ protected void channelRead0(final ChannelHandlerContext ctx, final HttpObject ms
final DiskFileUpload fileUpload = (DiskFileUpload) data;
checkState(fileUpload.isCompleted());

final Path dest = currentUploadDir.resolve(fileUpload.getFilename());
// wrapping around another File instantiation is a simple way to remove any path information - we're
// solely interested in the filename
final Path dest = currentUploadDir.resolve(new File(fileUpload.getFilename()).getName());
fileUpload.renameTo(dest.toFile());
LOG.trace("Upload of file {} complete.", fileUpload.getFilename());
LOG.trace("Upload of file {} into destination {} complete.", fileUpload.getFilename(), dest.toString());
} 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,15 @@
package org.apache.flink.runtime.rest;

import org.apache.flink.runtime.io.network.netty.NettyLeakDetectionResource;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.runtime.rest.messages.MessageParameters;
import org.apache.flink.runtime.rest.messages.RequestBody;
import org.apache.flink.runtime.rest.util.RestMapperUtils;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.function.BiConsumerWithException;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
Expand All @@ -31,17 +37,27 @@
import okhttp3.OkHttpClient;
import okhttp3.Request;
import okhttp3.Response;
import org.junit.After;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;

import java.io.File;
import java.io.IOException;
import java.io.StringWriter;
import java.lang.reflect.Field;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
Expand All @@ -52,70 +68,74 @@
*/
public class FileUploadHandlerTest extends TestLogger {

@ClassRule
public static final MultipartUploadResource MULTIPART_UPLOAD_RESOURCE = new MultipartUploadResource();
@Rule
public final MultipartUploadResource multipartUpdateResource = new MultipartUploadResource();

private static final ObjectMapper OBJECT_MAPPER = RestMapperUtils.getStrictObjectMapper();

@ClassRule
public static final NettyLeakDetectionResource LEAK_DETECTION = new NettyLeakDetectionResource();

@After
public void reset() {
MULTIPART_UPLOAD_RESOURCE.resetState();
}

private static Request buildMalformedRequest(String headerUrl) {
private Request buildMalformedRequest(String headerUrl) {
MultipartBody.Builder builder = new MultipartBody.Builder();
builder = addFilePart(builder);
// this causes a failure in the FileUploadHandler since the request should only contain form-data
builder = builder.addPart(okhttp3.RequestBody.create(MediaType.parse("text/plain"), "crash"));
return finalizeRequest(builder, headerUrl);
}

private static Request buildMixedRequestWithUnknownAttribute(String headerUrl) throws IOException {
private Request buildMixedRequestWithUnknownAttribute(String headerUrl) throws IOException {
MultipartBody.Builder builder = new MultipartBody.Builder();
builder = addJsonPart(builder, new MultipartUploadResource.TestRequestBody(), "hello");
builder = addFilePart(builder);
return finalizeRequest(builder, headerUrl);
}

private static Request buildFileRequest(String headerUrl) {
private Request buildRequestWithCustomFilenames(String headerUrl, String filename1, String filename2) {
MultipartBody.Builder builder = new MultipartBody.Builder();
builder = addFilePart(builder, multipartUpdateResource.file1, filename1);
builder = addFilePart(builder, multipartUpdateResource.file2, filename2);
return finalizeRequest(builder, headerUrl);
}

private Request buildFileRequest(String headerUrl) {
MultipartBody.Builder builder = new MultipartBody.Builder();
builder = addFilePart(builder);
return finalizeRequest(builder, headerUrl);
}

private static Request buildJsonRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
private Request buildJsonRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
MultipartBody.Builder builder = new MultipartBody.Builder();
builder = addJsonPart(builder, json, FileUploadHandler.HTTP_ATTRIBUTE_REQUEST);
return finalizeRequest(builder, headerUrl);
}

private static Request buildMixedRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
private Request buildMixedRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
MultipartBody.Builder builder = new MultipartBody.Builder();
builder = addJsonPart(builder, json, FileUploadHandler.HTTP_ATTRIBUTE_REQUEST);
builder = addFilePart(builder);
return finalizeRequest(builder, headerUrl);
}

private static Request finalizeRequest(MultipartBody.Builder builder, String headerUrl) {
private Request finalizeRequest(MultipartBody.Builder builder, String headerUrl) {
MultipartBody multipartBody = builder
.setType(MultipartBody.FORM)
.build();

return new Request.Builder()
.url(MULTIPART_UPLOAD_RESOURCE.serverAddress + headerUrl)
.url(multipartUpdateResource.serverAddress + headerUrl)
.post(multipartBody)
.build();
}

private static MultipartBody.Builder addFilePart(MultipartBody.Builder builder) {
for (File file : MULTIPART_UPLOAD_RESOURCE.getFilesToUpload()) {
okhttp3.RequestBody filePayload = okhttp3.RequestBody.create(MediaType.parse("application/octet-stream"), file);
private MultipartBody.Builder addFilePart(final MultipartBody.Builder builder) {
multipartUpdateResource.getFilesToUpload().forEach(f -> addFilePart(builder, f, f.getName()));
return builder;
}

builder = builder.addFormDataPart(file.getName(), file.getName(), filePayload);
}
private static MultipartBody.Builder addFilePart(MultipartBody.Builder builder, File file, String filename) {
okhttp3.RequestBody filePayload = okhttp3.RequestBody.create(MediaType.parse("application/octet-stream"), file);
builder = builder.addFormDataPart(file.getName(), filename, filePayload);

return builder;
}
Expand All @@ -133,9 +153,9 @@ private static MultipartBody.Builder addJsonPart(MultipartBody.Builder builder,
public void testUploadDirectoryRegeneration() throws Exception {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

MultipartUploadResource.MultipartFileHandler fileHandler = MULTIPART_UPLOAD_RESOURCE.getFileHandler();
MultipartUploadResource.MultipartFileHandler fileHandler = multipartUpdateResource.getFileHandler();

FileUtils.deleteDirectory(MULTIPART_UPLOAD_RESOURCE.getUploadDirectory().toFile());
FileUtils.deleteDirectory(multipartUpdateResource.getUploadDirectory().toFile());

Request fileRequest = buildFileRequest(fileHandler.getMessageHeaders().getTargetRestEndpointURL());
try (Response response = client.newCall(fileRequest).execute()) {
Expand All @@ -149,7 +169,7 @@ public void testUploadDirectoryRegeneration() throws Exception {
public void testMixedMultipart() throws Exception {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

MultipartUploadResource.MultipartMixedHandler mixedHandler = MULTIPART_UPLOAD_RESOURCE.getMixedHandler();
MultipartUploadResource.MultipartMixedHandler mixedHandler = multipartUpdateResource.getMixedHandler();

Request jsonRequest = buildJsonRequest(mixedHandler.getMessageHeaders().getTargetRestEndpointURL(), new MultipartUploadResource.TestRequestBody());
try (Response response = client.newCall(jsonRequest).execute()) {
Expand Down Expand Up @@ -177,7 +197,7 @@ public void testMixedMultipart() throws Exception {
public void testJsonMultipart() throws Exception {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

MultipartUploadResource.MultipartJsonHandler jsonHandler = MULTIPART_UPLOAD_RESOURCE.getJsonHandler();
MultipartUploadResource.MultipartJsonHandler jsonHandler = multipartUpdateResource.getJsonHandler();

MultipartUploadResource.TestRequestBody json = new MultipartUploadResource.TestRequestBody();
Request jsonRequest = buildJsonRequest(jsonHandler.getMessageHeaders().getTargetRestEndpointURL(), json);
Expand Down Expand Up @@ -205,7 +225,7 @@ public void testJsonMultipart() throws Exception {
public void testFileMultipart() throws Exception {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

MultipartUploadResource.MultipartFileHandler fileHandler = MULTIPART_UPLOAD_RESOURCE.getFileHandler();
MultipartUploadResource.MultipartFileHandler fileHandler = multipartUpdateResource.getFileHandler();

Request jsonRequest = buildJsonRequest(fileHandler.getMessageHeaders().getTargetRestEndpointURL(), new MultipartUploadResource.TestRequestBody());
try (Response response = client.newCall(jsonRequest).execute()) {
Expand All @@ -231,11 +251,12 @@ public void testFileMultipart() throws Exception {
public void testUploadCleanupOnUnknownAttribute() throws IOException {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

Request request = buildMixedRequestWithUnknownAttribute(MULTIPART_UPLOAD_RESOURCE.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
Request request = buildMixedRequestWithUnknownAttribute(multipartUpdateResource
.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
try (Response response = client.newCall(request).execute()) {
assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
}
MULTIPART_UPLOAD_RESOURCE.assertUploadDirectoryIsEmpty();
multipartUpdateResource.assertUploadDirectoryIsEmpty();

verifyNoFileIsRegisteredToDeleteOnExitHook();
}
Expand All @@ -247,16 +268,117 @@ public void testUploadCleanupOnUnknownAttribute() throws IOException {
public void testUploadCleanupOnFailure() throws IOException {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

Request request = buildMalformedRequest(MULTIPART_UPLOAD_RESOURCE.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
Request request = buildMalformedRequest(multipartUpdateResource
.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
try (Response response = client.newCall(request).execute()) {
// decoding errors aren't handled separately by the FileUploadHandler
assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.code());
}
MULTIPART_UPLOAD_RESOURCE.assertUploadDirectoryIsEmpty();
multipartUpdateResource.assertUploadDirectoryIsEmpty();

verifyNoFileIsRegisteredToDeleteOnExitHook();
}

@Test
public void testFileUploadUsingCustomFilename() throws IOException {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

String customFilename1 = "different-name-1.jar";
String customFilename2 = "different-name-2.jar";

multipartUpdateResource.setFileUploadVerifier(new CustomFilenameVerifier(
customFilename1,
multipartUpdateResource.file1.toPath(),
customFilename2,
multipartUpdateResource.file2.toPath()));

MessageHeaders<?, ?, ?> messageHeaders = multipartUpdateResource.getFileHandler().getMessageHeaders();
Request request = buildRequestWithCustomFilenames(
messageHeaders.getTargetRestEndpointURL(),
customFilename1,
customFilename2);
try (Response response = client.newCall(request).execute()) {
assertEquals(messageHeaders.getResponseStatusCode().code(), response.code());
}

verifyNoFileIsRegisteredToDeleteOnExitHook();
}

@Test
public void testFileUploadUsingCustomFilenameWithParentFolderPath() throws IOException {
OkHttpClient client = createOkHttpClientWithNoTimeouts();

String customFilename1 = "different-name-1.jar";
String customFilename2 = "different-name-2.jar";

multipartUpdateResource.setFileUploadVerifier(new CustomFilenameVerifier(
customFilename1,
multipartUpdateResource.file1.toPath(),
customFilename2,
multipartUpdateResource.file2.toPath()));

// referring to the parent folder within the filename should be ignored
MessageHeaders<?, ?, ?> messageHeaders = multipartUpdateResource.getFileHandler().getMessageHeaders();
Request request = buildRequestWithCustomFilenames(
multipartUpdateResource.getFileHandler().getMessageHeaders().getTargetRestEndpointURL(),
String.format("../%s", customFilename1),
String.format("../%s", customFilename2));
try (Response response = client.newCall(request).execute()) {
assertEquals(messageHeaders.getResponseStatusCode().code(), response.code());
}

verifyNoFileIsRegisteredToDeleteOnExitHook();
}

private static class CustomFilenameVerifier implements BiConsumerWithException<HandlerRequest<? extends RequestBody, ? extends MessageParameters>, RestfulGateway, Exception> {

private final String customFilename1;
private final Path fileContent1;

private final String customFilename2;
private final Path fileContent2;

public CustomFilenameVerifier(String customFilename1, Path fileContent1, String customFilename2, Path fileContent2) {
this.customFilename1 = customFilename1;
this.fileContent1 = fileContent1;

this.customFilename2 = customFilename2;
this.fileContent2 = fileContent2;
}

@Override
public void accept(
HandlerRequest<? extends RequestBody, ? extends MessageParameters> request,
RestfulGateway restfulGateway) throws Exception {
List<Path> uploadedFiles = request.getUploadedFiles().stream().map(File::toPath).collect(
Collectors.toList());

List<Path> actualList = new ArrayList<>(uploadedFiles);
actualList.sort(Comparator.comparing(Path::toString));

SortedMap<String, Path> expectedFilenamesAndContent = new TreeMap<>();
expectedFilenamesAndContent.put(customFilename1, fileContent1);
expectedFilenamesAndContent.put(customFilename2, fileContent2);

assertEquals(expectedFilenamesAndContent.size(), uploadedFiles.size());

Iterator<Path> uploadedFileIterator = actualList.iterator();
for (Map.Entry<String, Path> expectedFilenameAndContent : expectedFilenamesAndContent.entrySet()) {
String expectedFilename = expectedFilenameAndContent.getKey();
Path expectedContent = expectedFilenameAndContent.getValue();

assertTrue(uploadedFileIterator.hasNext());
Path actual = uploadedFileIterator.next();

assertEquals(expectedFilename, actual.getFileName().toString());

byte[] originalContent = java.nio.file.Files.readAllBytes(expectedContent);
byte[] receivedContent = java.nio.file.Files.readAllBytes(actual);
assertArrayEquals(originalContent, receivedContent);
}
}
}

private OkHttpClient createOkHttpClientWithNoTimeouts() {
// don't fail if some OkHttpClient operations take longer. See FLINK-17725
return new OkHttpClient.Builder()
Expand Down
Loading

0 comments on commit a5264a6

Please sign in to comment.