From a5264a6f41524afe8ceadf1d8ddc8c80f323ebc4 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Sun, 15 Nov 2020 22:20:24 +0100 Subject: [PATCH] [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. --- .../flink/runtime/rest/FileUploadHandler.java | 7 +- .../runtime/rest/FileUploadHandlerTest.java | 180 ++++++++++++++--- .../runtime/rest/MultipartUploadResource.java | 191 +++++++++--------- 3 files changed, 257 insertions(+), 121 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java index 6f60830b94b09..7776b55244fef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java @@ -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; @@ -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 diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java index 74c0ee3021a98..79c21836030e7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java @@ -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; @@ -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; @@ -52,20 +68,15 @@ */ 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 @@ -73,49 +84,58 @@ private static Request buildMalformedRequest(String headerUrl) { 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; } @@ -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()) { @@ -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()) { @@ -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); @@ -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()) { @@ -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(); } @@ -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, 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 request, + RestfulGateway restfulGateway) throws Exception { + List uploadedFiles = request.getUploadedFiles().stream().map(File::toPath).collect( + Collectors.toList()); + + List actualList = new ArrayList<>(uploadedFiles); + actualList.sort(Comparator.comparing(Path::toString)); + + SortedMap expectedFilenamesAndContent = new TreeMap<>(); + expectedFilenamesAndContent.put(customFilename1, fileContent1); + expectedFilenamesAndContent.put(customFilename2, fileContent2); + + assertEquals(expectedFilenamesAndContent.size(), uploadedFiles.size()); + + Iterator uploadedFileIterator = actualList.iterator(); + for (Map.Entry 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() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/MultipartUploadResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/MultipartUploadResource.java index 7dc23e5becb09..4845edaf56e62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/MultipartUploadResource.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/MultipartUploadResource.java @@ -29,12 +29,14 @@ import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.EmptyResponseBody; 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.TestRestServerEndpoint; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.BiConsumerWithException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; @@ -90,6 +92,8 @@ public class MultipartUploadResource extends ExternalResource { private Path configuredUploadDir; + private BiConsumerWithException, RestfulGateway, RestHandlerException> fileUploadVerifier; + @Override public void before() throws Exception { temporaryFolder.create(); @@ -115,9 +119,9 @@ public void before() throws Exception { file2 = temporaryFolder.newFile(); Files.write(file2.toPath(), "world".getBytes(ConfigConstants.DEFAULT_CHARSET)); - mixedHandler = new MultipartMixedHandler(mockGatewayRetriever, Arrays.asList(file1.toPath(), file2.toPath())); + mixedHandler = new MultipartMixedHandler(mockGatewayRetriever); jsonHandler = new MultipartJsonHandler(mockGatewayRetriever); - fileHandler = new MultipartFileHandler(mockGatewayRetriever, Arrays.asList(file1.toPath(), file2.toPath())); + fileHandler = new MultipartFileHandler(mockGatewayRetriever); serverEndpoint = TestRestServerEndpoint.builder(serverConfig) .withHandler(mixedHandler) @@ -127,6 +131,41 @@ public void before() throws Exception { serverAddress = serverEndpoint.getRestBaseUrl(); serverSocketAddress = serverEndpoint.getServerAddress(); + + this.setFileUploadVerifier((request, restfulGateway) -> { + // the default verifier checks for identiy (i.e. same name and content) of all uploaded files + List expectedFiles = Arrays.asList(file1.toPath(), file2.toPath()); + List uploadedFiles = request.getUploadedFiles().stream().map(File::toPath).collect(Collectors.toList()); + + assertEquals(expectedFiles.size(), uploadedFiles.size()); + + List expectedList = new ArrayList<>(expectedFiles); + List actualList = new ArrayList<>(uploadedFiles); + expectedList.sort(Comparator.comparing(Path::toString)); + actualList.sort(Comparator.comparing(Path::toString)); + + for (int x = 0; x < expectedList.size(); x++) { + Path expected = expectedList.get(x); + Path actual = actualList.get(x); + + assertEquals(expected.getFileName().toString(), actual.getFileName().toString()); + + byte[] originalContent = Files.readAllBytes(expected); + byte[] receivedContent = Files.readAllBytes(actual); + assertArrayEquals(originalContent, receivedContent); + } + }); + } + + public void setFileUploadVerifier(BiConsumerWithException, RestfulGateway, Exception> verifier) { + this.fileUploadVerifier = (request, restfulGateway) -> { + try { + verifier.accept(request, restfulGateway); + } catch (Exception e) { + // return 505 to differentiate from common BAD_REQUEST responses in this test + throw new RestHandlerException("Test verification failed.", HttpResponseStatus.HTTP_VERSION_NOT_SUPPORTED, e); + } + }; } public Collection getFilesToUpload() { @@ -192,67 +231,67 @@ public void assertUploadDirectoryIsEmpty() throws IOException { /** * Handler that accepts a mixed request consisting of a {@link TestRequestBody} and {@link #file1} and {@link #file2}. */ - public static class MultipartMixedHandler extends AbstractRestHandler { - private final Collection expectedFiles; + public class MultipartMixedHandler extends AbstractRestHandler { + volatile TestRequestBody lastReceivedRequest = null; - MultipartMixedHandler(GatewayRetriever leaderRetriever, Collection expectedFiles) { + MultipartMixedHandler(GatewayRetriever leaderRetriever) { super(leaderRetriever, RpcUtils.INF_TIMEOUT, Collections.emptyMap(), MultipartMixedHeaders.INSTANCE); - this.expectedFiles = expectedFiles; } @Override protected CompletableFuture handleRequest(@Nonnull HandlerRequest request, @Nonnull RestfulGateway gateway) throws RestHandlerException { - MultipartFileHandler.verifyFileUpload(expectedFiles, request.getUploadedFiles().stream().map(File::toPath).collect(Collectors.toList())); + MultipartUploadResource.this.fileUploadVerifier.accept(request, gateway); this.lastReceivedRequest = request.getRequestBody(); return CompletableFuture.completedFuture(EmptyResponseBody.getInstance()); } - private static final class MultipartMixedHeaders implements MessageHeaders { - private static final MultipartMixedHeaders INSTANCE = new MultipartMixedHeaders(); + } - private MultipartMixedHeaders() { - } + private static final class MultipartMixedHeaders implements MessageHeaders { + private static final MultipartMixedHeaders INSTANCE = new MultipartMixedHeaders(); - @Override - public Class getRequestClass() { - return TestRequestBody.class; - } + private MultipartMixedHeaders() { + } - @Override - public Class getResponseClass() { - return EmptyResponseBody.class; - } + @Override + public Class getRequestClass() { + return TestRequestBody.class; + } - @Override - public HttpResponseStatus getResponseStatusCode() { - return HttpResponseStatus.OK; - } + @Override + public Class getResponseClass() { + return EmptyResponseBody.class; + } - @Override - public String getDescription() { - return ""; - } + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } - @Override - public EmptyMessageParameters getUnresolvedMessageParameters() { - return EmptyMessageParameters.getInstance(); - } + @Override + public String getDescription() { + return ""; + } - @Override - public HttpMethodWrapper getHttpMethod() { - return HttpMethodWrapper.POST; - } + @Override + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } - @Override - public String getTargetRestEndpointURL() { - return "/test/upload/mixed"; - } + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.POST; + } - @Override - public boolean acceptsFileUploads() { - return true; - } + @Override + public String getTargetRestEndpointURL() { + return "/test/upload/mixed"; + } + + @Override + public boolean acceptsFileUploads() { + return true; } } @@ -300,68 +339,40 @@ public boolean acceptsFileUploads() { } /** - * Handler that accepts a file request consisting of and {@link #file1} and {@link #file2}. + * Handler that accepts a file request and calls {@link MultipartUploadResource#fileUploadVerifier} to verify it. */ - public static class MultipartFileHandler extends AbstractRestHandler { - - private final Collection expectedFiles; + public class MultipartFileHandler extends AbstractRestHandler { - MultipartFileHandler(GatewayRetriever leaderRetriever, Collection expectedFiles) { + MultipartFileHandler(GatewayRetriever leaderRetriever) { super(leaderRetriever, RpcUtils.INF_TIMEOUT, Collections.emptyMap(), MultipartFileHeaders.INSTANCE); - this.expectedFiles = expectedFiles; } @Override protected CompletableFuture handleRequest(@Nonnull HandlerRequest request, @Nonnull RestfulGateway gateway) throws RestHandlerException { - verifyFileUpload(expectedFiles, request.getUploadedFiles().stream().map(File::toPath).collect(Collectors.toList())); + MultipartUploadResource.this.fileUploadVerifier.accept(request, gateway); return CompletableFuture.completedFuture(EmptyResponseBody.getInstance()); } + } - static void verifyFileUpload(Collection expectedFiles, Collection uploadedFiles) throws RestHandlerException { - try { - assertEquals(expectedFiles.size(), uploadedFiles.size()); - - List expectedList = new ArrayList<>(expectedFiles); - List actualList = new ArrayList<>(uploadedFiles); - expectedList.sort(Comparator.comparing(Path::toString)); - actualList.sort(Comparator.comparing(Path::toString)); - - for (int x = 0; x < expectedList.size(); x++) { - Path expected = expectedList.get(x); - Path actual = actualList.get(x); - - assertEquals(expected.getFileName().toString(), actual.getFileName().toString()); + private static class MultipartFileHeaders extends TestHeadersBase { + static final MultipartFileHeaders INSTANCE = new MultipartFileHeaders(); - byte[] originalContent = Files.readAllBytes(expected); - byte[] receivedContent = Files.readAllBytes(actual); - assertArrayEquals(originalContent, receivedContent); - } - } catch (Exception e) { - // return 505 to differentiate from common BAD_REQUEST responses in this test - throw new RestHandlerException("Test verification failed.", HttpResponseStatus.HTTP_VERSION_NOT_SUPPORTED, e); - } + private MultipartFileHeaders() { } - private static final class MultipartFileHeaders extends TestHeadersBase { - private static final MultipartFileHeaders INSTANCE = new MultipartFileHeaders(); - - private MultipartFileHeaders() { - } - - @Override - public Class getRequestClass() { - return EmptyRequestBody.class; - } + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } - @Override - public String getTargetRestEndpointURL() { - return "/test/upload/file"; - } + @Override + public String getTargetRestEndpointURL() { + return "/test/upload/file"; + } - @Override - public boolean acceptsFileUploads() { - return true; - } + @Override + public boolean acceptsFileUploads() { + return true; } }