Skip to content

Commit a5264a6

Browse files
XComprmetzger
authored andcommitted
[hotfix][runtime] A customized filename can be specified through Content-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.
1 parent b561010 commit a5264a6

File tree

3 files changed

+257
-121
lines changed

3 files changed

+257
-121
lines changed

‎flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java‎

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@
5151

5252
import javax.annotation.Nullable;
5353

54+
import java.io.File;
5455
import java.io.IOException;
5556
import java.nio.file.Files;
5657
import java.nio.file.Path;
@@ -138,9 +139,11 @@ protected void channelRead0(final ChannelHandlerContext ctx, final HttpObject ms
138139
final DiskFileUpload fileUpload = (DiskFileUpload) data;
139140
checkState(fileUpload.isCompleted());
140141

141-
final Path dest = currentUploadDir.resolve(fileUpload.getFilename());
142+
// wrapping around another File instantiation is a simple way to remove any path information - we're
143+
// solely interested in the filename
144+
final Path dest = currentUploadDir.resolve(new File(fileUpload.getFilename()).getName());
142145
fileUpload.renameTo(dest.toFile());
143-
LOG.trace("Upload of file {} complete.", fileUpload.getFilename());
146+
LOG.trace("Upload of file {} into destination {} complete.", fileUpload.getFilename(), dest.toString());
144147
} else if (data.getHttpDataType() == InterfaceHttpData.HttpDataType.Attribute) {
145148
final Attribute request = (Attribute) data;
146149
// this could also be implemented by using the first found Attribute as the payload

‎flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java‎

Lines changed: 151 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,15 @@
1919
package org.apache.flink.runtime.rest;
2020

2121
import org.apache.flink.runtime.io.network.netty.NettyLeakDetectionResource;
22+
import org.apache.flink.runtime.rest.handler.HandlerRequest;
23+
import org.apache.flink.runtime.rest.messages.MessageHeaders;
24+
import org.apache.flink.runtime.rest.messages.MessageParameters;
25+
import org.apache.flink.runtime.rest.messages.RequestBody;
2226
import org.apache.flink.runtime.rest.util.RestMapperUtils;
27+
import org.apache.flink.runtime.webmonitor.RestfulGateway;
2328
import org.apache.flink.util.FileUtils;
2429
import org.apache.flink.util.TestLogger;
30+
import org.apache.flink.util.function.BiConsumerWithException;
2531

2632
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
2733
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
@@ -31,17 +37,27 @@
3137
import okhttp3.OkHttpClient;
3238
import okhttp3.Request;
3339
import okhttp3.Response;
34-
import org.junit.After;
3540
import org.junit.ClassRule;
41+
import org.junit.Rule;
3642
import org.junit.Test;
3743

3844
import java.io.File;
3945
import java.io.IOException;
4046
import java.io.StringWriter;
4147
import java.lang.reflect.Field;
48+
import java.nio.file.Path;
49+
import java.util.ArrayList;
50+
import java.util.Comparator;
51+
import java.util.Iterator;
4252
import java.util.LinkedHashSet;
53+
import java.util.List;
54+
import java.util.Map;
55+
import java.util.SortedMap;
56+
import java.util.TreeMap;
4357
import java.util.concurrent.TimeUnit;
58+
import java.util.stream.Collectors;
4459

60+
import static org.junit.Assert.assertArrayEquals;
4561
import static org.junit.Assert.assertEquals;
4662
import static org.junit.Assert.assertTrue;
4763
import static org.junit.Assert.fail;
@@ -52,70 +68,74 @@
5268
*/
5369
public class FileUploadHandlerTest extends TestLogger {
5470

55-
@ClassRule
56-
public static final MultipartUploadResource MULTIPART_UPLOAD_RESOURCE = new MultipartUploadResource();
71+
@Rule
72+
public final MultipartUploadResource multipartUpdateResource = new MultipartUploadResource();
5773

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

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

63-
@After
64-
public void reset() {
65-
MULTIPART_UPLOAD_RESOURCE.resetState();
66-
}
67-
68-
private static Request buildMalformedRequest(String headerUrl) {
79+
private Request buildMalformedRequest(String headerUrl) {
6980
MultipartBody.Builder builder = new MultipartBody.Builder();
7081
builder = addFilePart(builder);
7182
// this causes a failure in the FileUploadHandler since the request should only contain form-data
7283
builder = builder.addPart(okhttp3.RequestBody.create(MediaType.parse("text/plain"), "crash"));
7384
return finalizeRequest(builder, headerUrl);
7485
}
7586

76-
private static Request buildMixedRequestWithUnknownAttribute(String headerUrl) throws IOException {
87+
private Request buildMixedRequestWithUnknownAttribute(String headerUrl) throws IOException {
7788
MultipartBody.Builder builder = new MultipartBody.Builder();
7889
builder = addJsonPart(builder, new MultipartUploadResource.TestRequestBody(), "hello");
7990
builder = addFilePart(builder);
8091
return finalizeRequest(builder, headerUrl);
8192
}
8293

83-
private static Request buildFileRequest(String headerUrl) {
94+
private Request buildRequestWithCustomFilenames(String headerUrl, String filename1, String filename2) {
95+
MultipartBody.Builder builder = new MultipartBody.Builder();
96+
builder = addFilePart(builder, multipartUpdateResource.file1, filename1);
97+
builder = addFilePart(builder, multipartUpdateResource.file2, filename2);
98+
return finalizeRequest(builder, headerUrl);
99+
}
100+
101+
private Request buildFileRequest(String headerUrl) {
84102
MultipartBody.Builder builder = new MultipartBody.Builder();
85103
builder = addFilePart(builder);
86104
return finalizeRequest(builder, headerUrl);
87105
}
88106

89-
private static Request buildJsonRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
107+
private Request buildJsonRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
90108
MultipartBody.Builder builder = new MultipartBody.Builder();
91109
builder = addJsonPart(builder, json, FileUploadHandler.HTTP_ATTRIBUTE_REQUEST);
92110
return finalizeRequest(builder, headerUrl);
93111
}
94112

95-
private static Request buildMixedRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
113+
private Request buildMixedRequest(String headerUrl, MultipartUploadResource.TestRequestBody json) throws IOException {
96114
MultipartBody.Builder builder = new MultipartBody.Builder();
97115
builder = addJsonPart(builder, json, FileUploadHandler.HTTP_ATTRIBUTE_REQUEST);
98116
builder = addFilePart(builder);
99117
return finalizeRequest(builder, headerUrl);
100118
}
101119

102-
private static Request finalizeRequest(MultipartBody.Builder builder, String headerUrl) {
120+
private Request finalizeRequest(MultipartBody.Builder builder, String headerUrl) {
103121
MultipartBody multipartBody = builder
104122
.setType(MultipartBody.FORM)
105123
.build();
106124

107125
return new Request.Builder()
108-
.url(MULTIPART_UPLOAD_RESOURCE.serverAddress + headerUrl)
126+
.url(multipartUpdateResource.serverAddress + headerUrl)
109127
.post(multipartBody)
110128
.build();
111129
}
112130

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

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

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

136-
MultipartUploadResource.MultipartFileHandler fileHandler = MULTIPART_UPLOAD_RESOURCE.getFileHandler();
156+
MultipartUploadResource.MultipartFileHandler fileHandler = multipartUpdateResource.getFileHandler();
137157

138-
FileUtils.deleteDirectory(MULTIPART_UPLOAD_RESOURCE.getUploadDirectory().toFile());
158+
FileUtils.deleteDirectory(multipartUpdateResource.getUploadDirectory().toFile());
139159

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

152-
MultipartUploadResource.MultipartMixedHandler mixedHandler = MULTIPART_UPLOAD_RESOURCE.getMixedHandler();
172+
MultipartUploadResource.MultipartMixedHandler mixedHandler = multipartUpdateResource.getMixedHandler();
153173

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

180-
MultipartUploadResource.MultipartJsonHandler jsonHandler = MULTIPART_UPLOAD_RESOURCE.getJsonHandler();
200+
MultipartUploadResource.MultipartJsonHandler jsonHandler = multipartUpdateResource.getJsonHandler();
181201

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

208-
MultipartUploadResource.MultipartFileHandler fileHandler = MULTIPART_UPLOAD_RESOURCE.getFileHandler();
228+
MultipartUploadResource.MultipartFileHandler fileHandler = multipartUpdateResource.getFileHandler();
209229

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

234-
Request request = buildMixedRequestWithUnknownAttribute(MULTIPART_UPLOAD_RESOURCE.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
254+
Request request = buildMixedRequestWithUnknownAttribute(multipartUpdateResource
255+
.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
235256
try (Response response = client.newCall(request).execute()) {
236257
assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
237258
}
238-
MULTIPART_UPLOAD_RESOURCE.assertUploadDirectoryIsEmpty();
259+
multipartUpdateResource.assertUploadDirectoryIsEmpty();
239260

240261
verifyNoFileIsRegisteredToDeleteOnExitHook();
241262
}
@@ -247,16 +268,117 @@ public void testUploadCleanupOnUnknownAttribute() throws IOException {
247268
public void testUploadCleanupOnFailure() throws IOException {
248269
OkHttpClient client = createOkHttpClientWithNoTimeouts();
249270

250-
Request request = buildMalformedRequest(MULTIPART_UPLOAD_RESOURCE.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
271+
Request request = buildMalformedRequest(multipartUpdateResource
272+
.getMixedHandler().getMessageHeaders().getTargetRestEndpointURL());
251273
try (Response response = client.newCall(request).execute()) {
252274
// decoding errors aren't handled separately by the FileUploadHandler
253275
assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.code());
254276
}
255-
MULTIPART_UPLOAD_RESOURCE.assertUploadDirectoryIsEmpty();
277+
multipartUpdateResource.assertUploadDirectoryIsEmpty();
278+
279+
verifyNoFileIsRegisteredToDeleteOnExitHook();
280+
}
281+
282+
@Test
283+
public void testFileUploadUsingCustomFilename() throws IOException {
284+
OkHttpClient client = createOkHttpClientWithNoTimeouts();
285+
286+
String customFilename1 = "different-name-1.jar";
287+
String customFilename2 = "different-name-2.jar";
288+
289+
multipartUpdateResource.setFileUploadVerifier(new CustomFilenameVerifier(
290+
customFilename1,
291+
multipartUpdateResource.file1.toPath(),
292+
customFilename2,
293+
multipartUpdateResource.file2.toPath()));
294+
295+
MessageHeaders<?, ?, ?> messageHeaders = multipartUpdateResource.getFileHandler().getMessageHeaders();
296+
Request request = buildRequestWithCustomFilenames(
297+
messageHeaders.getTargetRestEndpointURL(),
298+
customFilename1,
299+
customFilename2);
300+
try (Response response = client.newCall(request).execute()) {
301+
assertEquals(messageHeaders.getResponseStatusCode().code(), response.code());
302+
}
303+
304+
verifyNoFileIsRegisteredToDeleteOnExitHook();
305+
}
306+
307+
@Test
308+
public void testFileUploadUsingCustomFilenameWithParentFolderPath() throws IOException {
309+
OkHttpClient client = createOkHttpClientWithNoTimeouts();
310+
311+
String customFilename1 = "different-name-1.jar";
312+
String customFilename2 = "different-name-2.jar";
313+
314+
multipartUpdateResource.setFileUploadVerifier(new CustomFilenameVerifier(
315+
customFilename1,
316+
multipartUpdateResource.file1.toPath(),
317+
customFilename2,
318+
multipartUpdateResource.file2.toPath()));
319+
320+
// referring to the parent folder within the filename should be ignored
321+
MessageHeaders<?, ?, ?> messageHeaders = multipartUpdateResource.getFileHandler().getMessageHeaders();
322+
Request request = buildRequestWithCustomFilenames(
323+
multipartUpdateResource.getFileHandler().getMessageHeaders().getTargetRestEndpointURL(),
324+
String.format("../%s", customFilename1),
325+
String.format("../%s", customFilename2));
326+
try (Response response = client.newCall(request).execute()) {
327+
assertEquals(messageHeaders.getResponseStatusCode().code(), response.code());
328+
}
256329

257330
verifyNoFileIsRegisteredToDeleteOnExitHook();
258331
}
259332

333+
private static class CustomFilenameVerifier implements BiConsumerWithException<HandlerRequest<? extends RequestBody, ? extends MessageParameters>, RestfulGateway, Exception> {
334+
335+
private final String customFilename1;
336+
private final Path fileContent1;
337+
338+
private final String customFilename2;
339+
private final Path fileContent2;
340+
341+
public CustomFilenameVerifier(String customFilename1, Path fileContent1, String customFilename2, Path fileContent2) {
342+
this.customFilename1 = customFilename1;
343+
this.fileContent1 = fileContent1;
344+
345+
this.customFilename2 = customFilename2;
346+
this.fileContent2 = fileContent2;
347+
}
348+
349+
@Override
350+
public void accept(
351+
HandlerRequest<? extends RequestBody, ? extends MessageParameters> request,
352+
RestfulGateway restfulGateway) throws Exception {
353+
List<Path> uploadedFiles = request.getUploadedFiles().stream().map(File::toPath).collect(
354+
Collectors.toList());
355+
356+
List<Path> actualList = new ArrayList<>(uploadedFiles);
357+
actualList.sort(Comparator.comparing(Path::toString));
358+
359+
SortedMap<String, Path> expectedFilenamesAndContent = new TreeMap<>();
360+
expectedFilenamesAndContent.put(customFilename1, fileContent1);
361+
expectedFilenamesAndContent.put(customFilename2, fileContent2);
362+
363+
assertEquals(expectedFilenamesAndContent.size(), uploadedFiles.size());
364+
365+
Iterator<Path> uploadedFileIterator = actualList.iterator();
366+
for (Map.Entry<String, Path> expectedFilenameAndContent : expectedFilenamesAndContent.entrySet()) {
367+
String expectedFilename = expectedFilenameAndContent.getKey();
368+
Path expectedContent = expectedFilenameAndContent.getValue();
369+
370+
assertTrue(uploadedFileIterator.hasNext());
371+
Path actual = uploadedFileIterator.next();
372+
373+
assertEquals(expectedFilename, actual.getFileName().toString());
374+
375+
byte[] originalContent = java.nio.file.Files.readAllBytes(expectedContent);
376+
byte[] receivedContent = java.nio.file.Files.readAllBytes(actual);
377+
assertArrayEquals(originalContent, receivedContent);
378+
}
379+
}
380+
}
381+
260382
private OkHttpClient createOkHttpClientWithNoTimeouts() {
261383
// don't fail if some OkHttpClient operations take longer. See FLINK-17725
262384
return new OkHttpClient.Builder()

0 commit comments

Comments
 (0)