This is an automated email from the ASF dual-hosted git repository. chesnay pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit dcca520cb47a22354301174d0e1d2bde7f65061e Author: Chesnay Schepler <[email protected]> AuthorDate: Wed Apr 5 14:47:43 2023 +0200 [FLINK-31733][tests] Simplify DocumentingRestEndpoint setup --- .../flink/docs/rest/OpenApiSpecGeneratorTest.java | 86 +++++++--------------- .../flink/docs/rest/RestAPIDocGeneratorTest.java | 47 +++--------- .../runtime/rest/util/DocumentingRestEndpoint.java | 8 ++ 3 files changed, 45 insertions(+), 96 deletions(-) diff --git a/flink-docs/src/test/java/org/apache/flink/docs/rest/OpenApiSpecGeneratorTest.java b/flink-docs/src/test/java/org/apache/flink/docs/rest/OpenApiSpecGeneratorTest.java index 884f584a93e..5b83d46c2af 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/rest/OpenApiSpecGeneratorTest.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/rest/OpenApiSpecGeneratorTest.java @@ -18,16 +18,12 @@ package org.apache.flink.docs.rest; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.docs.rest.data.TestAdditionalFieldsMessageHeaders; import org.apache.flink.docs.rest.data.TestEmptyMessageHeaders; import org.apache.flink.docs.rest.data.TestExcludeMessageHeaders; -import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.util.DocumentingRestEndpoint; import org.apache.flink.runtime.rest.versioning.RuntimeRestAPIVersion; -import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; - import io.swagger.v3.oas.models.OpenAPI; import io.swagger.v3.oas.models.media.StringSchema; import org.junit.jupiter.api.Test; @@ -36,10 +32,6 @@ import org.junit.jupiter.api.io.TempDir; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CompletableFuture; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -53,7 +45,11 @@ class OpenApiSpecGeneratorTest { final Path file = tmpDir.resolve("openapi_spec.yaml"); OpenApiSpecGenerator.createDocumentationFile( - title, new TestExcludeDocumentingRestEndpoint(), RuntimeRestAPIVersion.V0, file); + title, + DocumentingRestEndpoint.forRestHandlerSpecifications( + new TestEmptyMessageHeaders("/test/empty1", "This is a testing REST API.")), + RuntimeRestAPIVersion.V0, + file); final String actual = new String(Files.readAllBytes(file), StandardCharsets.UTF_8); assertThat(actual).contains("title: " + title); @@ -63,7 +59,19 @@ class OpenApiSpecGeneratorTest { void testExcludeFromDocumentation(@TempDir Path tmpDir) throws Exception { final Path file = tmpDir.resolve("openapi_spec.yaml"); OpenApiSpecGenerator.createDocumentationFile( - "title", new TestExcludeDocumentingRestEndpoint(), RuntimeRestAPIVersion.V0, file); + "title", + DocumentingRestEndpoint.forRestHandlerSpecifications( + new TestEmptyMessageHeaders("/test/empty1", "This is a testing REST API."), + new TestEmptyMessageHeaders( + "/test/empty2", "This is another testing REST API."), + new TestExcludeMessageHeaders( + "/test/exclude1", + "This REST API should not appear in the generated documentation."), + new TestExcludeMessageHeaders( + "/test/exclude2", + "This REST API should also not appear in the generated documentation.")), + RuntimeRestAPIVersion.V0, + file); final String actual = new String(Files.readAllBytes(file), StandardCharsets.UTF_8); assertThat(actual).contains("/test/empty1"); @@ -79,33 +87,6 @@ class OpenApiSpecGeneratorTest { "This REST API should also not appear in the generated documentation."); } - private static class TestExcludeDocumentingRestEndpoint implements DocumentingRestEndpoint { - - @Override - public List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers( - CompletableFuture<String> localAddressFuture) { - return Arrays.asList( - Tuple2.of( - new TestEmptyMessageHeaders( - "/test/empty1", "This is a testing REST API."), - null), - Tuple2.of( - new TestEmptyMessageHeaders( - "/test/empty2", "This is another testing REST API."), - null), - Tuple2.of( - new TestExcludeMessageHeaders( - "/test/exclude1", - "This REST API should not appear in the generated documentation."), - null), - Tuple2.of( - new TestExcludeMessageHeaders( - "/test/exclude2", - "This REST API should also not appear in the generated documentation."), - null)); - } - } - @Test void testDuplicateOperationIdsAreRejected(@TempDir Path tmpDir) { final Path file = tmpDir.resolve("openapi_spec.yaml"); @@ -113,30 +94,23 @@ class OpenApiSpecGeneratorTest { () -> OpenApiSpecGenerator.createDocumentationFile( "title", - new TestDuplicateOperationIdDocumentingRestEndpoint(), + DocumentingRestEndpoint.forRestHandlerSpecifications( + new TestEmptyMessageHeaders("operation1"), + new TestEmptyMessageHeaders("operation1")), RuntimeRestAPIVersion.V0, file)) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Duplicate OperationId"); } - private static class TestDuplicateOperationIdDocumentingRestEndpoint - implements DocumentingRestEndpoint { - - @Override - public List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers( - CompletableFuture<String> localAddressFuture) { - return Arrays.asList( - Tuple2.of(new TestEmptyMessageHeaders("operation1"), null), - Tuple2.of(new TestEmptyMessageHeaders("operation1"), null)); - } - } - @Test void testAdditionalFields(@TempDir Path tmpDir) throws Exception { final OpenAPI documentation = OpenApiSpecGenerator.createDocumentation( - "title", new TestAdditionalFieldsRestEndpoint(), RuntimeRestAPIVersion.V0); + "title", + DocumentingRestEndpoint.forRestHandlerSpecifications( + new TestAdditionalFieldsMessageHeaders("operation1")), + RuntimeRestAPIVersion.V0); assertThat(documentation.getComponents().getSchemas()) .extractingByKey("AdditionalFieldsRequestBody") .satisfies( @@ -144,14 +118,4 @@ class OpenApiSpecGeneratorTest { assertThat(x.getAdditionalProperties()) .isInstanceOf(StringSchema.class)); } - - private static class TestAdditionalFieldsRestEndpoint implements DocumentingRestEndpoint { - - @Override - public List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers( - CompletableFuture<String> localAddressFuture) { - return Collections.singletonList( - Tuple2.of(new TestAdditionalFieldsMessageHeaders("operation1"), null)); - } - } } diff --git a/flink-docs/src/test/java/org/apache/flink/docs/rest/RestAPIDocGeneratorTest.java b/flink-docs/src/test/java/org/apache/flink/docs/rest/RestAPIDocGeneratorTest.java index 315b6af6019..3389ffd3bc4 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/rest/RestAPIDocGeneratorTest.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/rest/RestAPIDocGeneratorTest.java @@ -19,23 +19,16 @@ package org.apache.flink.docs.rest; import org.apache.flink.annotation.docs.FlinkJsonSchema; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.docs.rest.data.TestEmptyMessageHeaders; import org.apache.flink.docs.rest.data.TestExcludeMessageHeaders; -import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.util.DocumentingRestEndpoint; import org.apache.flink.runtime.rest.versioning.RuntimeRestAPIVersion; import org.apache.flink.util.FileUtils; -import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; - import org.junit.jupiter.api.Test; import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CompletableFuture; import static org.assertj.core.api.Assertions.assertThat; @@ -46,7 +39,18 @@ class RestAPIDocGeneratorTest { void testExcludeFromDocumentation() throws Exception { File file = File.createTempFile("rest_v0_", ".html"); RestAPIDocGenerator.createHtmlFile( - new TestExcludeDocumentingRestEndpoint(), RuntimeRestAPIVersion.V0, file.toPath()); + DocumentingRestEndpoint.forRestHandlerSpecifications( + new TestEmptyMessageHeaders("/test/empty1", "This is a testing REST API."), + new TestEmptyMessageHeaders( + "/test/empty2", "This is another testing REST API."), + new TestExcludeMessageHeaders( + "/test/exclude1", + "This REST API should not appear in the generated documentation."), + new TestExcludeMessageHeaders( + "/test/exclude2", + "This REST API should also not appear in the generated documentation.")), + RuntimeRestAPIVersion.V0, + file.toPath()); String actual = FileUtils.readFile(file, "UTF-8"); assertThat(actual).containsSequence("/test/empty1"); @@ -78,33 +82,6 @@ class RestAPIDocGeneratorTest { + "}"); } - private static class TestExcludeDocumentingRestEndpoint implements DocumentingRestEndpoint { - - @Override - public List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers( - CompletableFuture<String> localAddressFuture) { - return Arrays.asList( - Tuple2.of( - new TestEmptyMessageHeaders( - "/test/empty1", "This is a testing REST API."), - null), - Tuple2.of( - new TestEmptyMessageHeaders( - "/test/empty2", "This is another testing REST API."), - null), - Tuple2.of( - new TestExcludeMessageHeaders( - "/test/exclude1", - "This REST API should not appear in the generated documentation."), - null), - Tuple2.of( - new TestExcludeMessageHeaders( - "/test/exclude2", - "This REST API should also not appear in the generated documentation."), - null)); - } - } - @FlinkJsonSchema.AdditionalFields(type = String.class) private static class TestAdditionalFields {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingRestEndpoint.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingRestEndpoint.java index 971ea3410b5..fb27515242b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingRestEndpoint.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingRestEndpoint.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -50,4 +51,11 @@ public interface DocumentingRestEndpoint { spec2.getTargetRestEndpointURL())) .collect(Collectors.toList()); } + + static DocumentingRestEndpoint forRestHandlerSpecifications(RestHandlerSpecification... specs) { + return localAddressFuture -> + Arrays.stream(specs) + .map(spec -> Tuple2.of(spec, (ChannelInboundHandler) null)) + .collect(Collectors.toList()); + } }
