This is an automated email from the ASF dual-hosted git repository.

dmvk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 84d000cf47b [FLINK-31383] Add support for documenting 
additionProperties of the REST API payloads.
84d000cf47b is described below

commit 84d000cf47b833625fbd5b5f72e48963b3156103
Author: David Moravek <[email protected]>
AuthorDate: Thu Mar 9 15:04:12 2023 +0100

    [FLINK-31383] Add support for documenting additionProperties of the REST 
API payloads.
---
 docs/static/generated/rest_v1_dispatcher.yml       |   2 +-
 docs/static/generated/rest_v1_sql_gateway.yml      |   2 +-
 .../flink/annotation/docs/FlinkJsonSchema.java     |  49 +++++++++
 .../flink/docs/rest/ApiSpecGeneratorUtils.java     |  62 ++++++++++++
 .../flink/docs/rest/OpenApiSpecGenerator.java      |  44 +++++++--
 .../flink/docs/rest/RestAPIDocGenerator.java       |  35 +++----
 .../flink/docs/rest/OpenApiSpecGeneratorTest.java  |  65 ++++++++----
 .../flink/docs/rest/RestAPIDocGeneratorTest.java   |  21 ++++
 .../data/TestAdditionalFieldsMessageHeaders.java   | 110 +++++++++++++++++++++
 9 files changed, 341 insertions(+), 49 deletions(-)

diff --git a/docs/static/generated/rest_v1_dispatcher.yml 
b/docs/static/generated/rest_v1_dispatcher.yml
index 787fbab50eb..eeb684f6268 100644
--- a/docs/static/generated/rest_v1_dispatcher.yml
+++ b/docs/static/generated/rest_v1_dispatcher.yml
@@ -6,7 +6,7 @@ info:
   license:
     name: Apache 2.0
     url: https://www.apache.org/licenses/LICENSE-2.0.html
-  version: v1/1.17-SNAPSHOT
+  version: v1/1.18-SNAPSHOT
 paths:
   /cluster:
     delete:
diff --git a/docs/static/generated/rest_v1_sql_gateway.yml 
b/docs/static/generated/rest_v1_sql_gateway.yml
index 31ae67595c7..d91e75cafff 100644
--- a/docs/static/generated/rest_v1_sql_gateway.yml
+++ b/docs/static/generated/rest_v1_sql_gateway.yml
@@ -6,7 +6,7 @@ info:
   license:
     name: Apache 2.0
     url: https://www.apache.org/licenses/LICENSE-2.0.html
-  version: v1/1.17-SNAPSHOT
+  version: v1/1.18-SNAPSHOT
 paths:
   /api_versions:
     get:
diff --git 
a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/FlinkJsonSchema.java
 
b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/FlinkJsonSchema.java
new file mode 100644
index 00000000000..92360e32341
--- /dev/null
+++ 
b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/FlinkJsonSchema.java
@@ -0,0 +1,49 @@
+/*
+ * 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.annotation.docs;
+
+import org.apache.flink.annotation.Internal;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/** Annotations for auto-generating a documentation of json payloads. */
+@Internal
+public class FlinkJsonSchema {
+
+    private FlinkJsonSchema() {}
+
+    /**
+     * This allows documenting a class that supports setting dynamic 
properties of a certain type.
+     */
+    @Target(ElementType.TYPE)
+    @Retention(RetentionPolicy.RUNTIME)
+    @Internal
+    public @interface AdditionalFields {
+
+        /**
+         * An actual type the additional fields need to match.
+         *
+         * @return type of the additional fields
+         */
+        Class<?> type();
+    }
+}
diff --git 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/ApiSpecGeneratorUtils.java
 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/ApiSpecGeneratorUtils.java
new file mode 100644
index 00000000000..705a9cfe801
--- /dev/null
+++ 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/ApiSpecGeneratorUtils.java
@@ -0,0 +1,62 @@
+/*
+ * 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.docs.rest;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.annotation.docs.FlinkJsonSchema;
+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.messages.ResponseBody;
+
+import java.util.Optional;
+
+/** Helper methods for generation API documentation. */
+public class ApiSpecGeneratorUtils {
+
+    private ApiSpecGeneratorUtils() {}
+
+    /**
+     * Checks whether the given endpoint should be documented.
+     *
+     * @param spec endpoint to check
+     * @return true if the endpoint should be documented
+     */
+    public static boolean shouldBeDocumented(
+            MessageHeaders<
+                            ? extends RequestBody,
+                            ? extends ResponseBody,
+                            ? extends MessageParameters>
+                    spec) {
+        return 
spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == 
null;
+    }
+
+    /**
+     * Find whether the class contains dynamic fields that need to be 
documented.
+     *
+     * @param clazz class to check
+     * @return optional that is non-empty if the class is annotated with {@link
+     *     FlinkJsonSchema.AdditionalFields}
+     */
+    public static Optional<Class<?>> findAdditionalFieldType(Class<?> clazz) {
+        final FlinkJsonSchema.AdditionalFields annotation =
+                clazz.getAnnotation(FlinkJsonSchema.AdditionalFields.class);
+        return 
Optional.ofNullable(annotation).map(FlinkJsonSchema.AdditionalFields::type);
+    }
+}
diff --git 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
index 38560db5f2a..915743651f9 100644
--- 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
+++ 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
@@ -19,7 +19,6 @@
 package org.apache.flink.docs.rest;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -41,6 +40,7 @@ import 
org.apache.flink.runtime.rest.util.DocumentingRestEndpoint;
 import org.apache.flink.runtime.rest.versioning.RestAPIVersion;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.webmonitor.handlers.JarUploadHeaders;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.SerializedThrowable;
 import org.apache.flink.util.jackson.JacksonMapperFactory;
 
@@ -113,6 +113,14 @@ public class OpenApiSpecGenerator {
             RestAPIVersion apiVersion,
             Path outputFile)
             throws IOException {
+        final OpenAPI openApi = createDocumentation(title, restEndpoint, 
apiVersion);
+        Files.deleteIfExists(outputFile);
+        Files.write(outputFile, 
Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
+    }
+
+    @VisibleForTesting
+    static OpenAPI createDocumentation(
+            String title, DocumentingRestEndpoint restEndpoint, RestAPIVersion 
apiVersion) {
         final OpenAPI openApi = new OpenAPI();
 
         // eagerly initialize some data-structures to simplify operations 
later on
@@ -124,7 +132,7 @@ public class OpenApiSpecGenerator {
         List<MessageHeaders> specs =
                 restEndpoint.getSpecs().stream()
                         .filter(spec -> 
spec.getSupportedAPIVersions().contains(apiVersion))
-                        .filter(OpenApiSpecGenerator::shouldBeDocumented)
+                        .filter(ApiSpecGeneratorUtils::shouldBeDocumented)
                         .collect(Collectors.toList());
         final Set<String> usedOperationIds = new HashSet<>();
         specs.forEach(spec -> add(spec, openApi, usedOperationIds));
@@ -143,8 +151,7 @@ public class OpenApiSpecGenerator {
         sortProperties(openApi);
         sortSchemas(openApi);
 
-        Files.deleteIfExists(outputFile);
-        Files.write(outputFile, 
Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
+        return openApi;
     }
 
     @SuppressWarnings("rawtypes")
@@ -172,10 +179,6 @@ public class OpenApiSpecGenerator {
         components.setSchemas(sortedSchemas);
     }
 
-    private static boolean shouldBeDocumented(MessageHeaders spec) {
-        return 
spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == 
null;
-    }
-
     private static void setInfo(
             final OpenAPI openApi, String title, final RestAPIVersion 
apiVersion) {
         openApi.info(
@@ -452,7 +455,30 @@ public class OpenApiSpecGenerator {
     }
 
     private static Schema<?> getSchema(Type type) {
-        return modelConverterContext.resolve(new 
AnnotatedType(type).resolveAsRef(true));
+        final AnnotatedType annotatedType = new 
AnnotatedType(type).resolveAsRef(true);
+        final Schema<?> schema = modelConverterContext.resolve(annotatedType);
+        if (type instanceof Class<?>) {
+            final Class<?> clazz = (Class<?>) type;
+            ApiSpecGeneratorUtils.findAdditionalFieldType(clazz)
+                    .map(OpenApiSpecGenerator::getSchema)
+                    .ifPresent(
+                            additionalPropertiesSchema -> {
+                                // We need to update the schema of the 
component, that is referenced
+                                // by the resolved schema (because we're 
setting resolveAsRef to
+                                // true).
+                                final String referencedComponentName = 
clazz.getSimpleName();
+                                final Schema<?> referencedComponentSchema =
+                                        Preconditions.checkNotNull(
+                                                modelConverterContext
+                                                        .getDefinedModels()
+                                                        
.get(referencedComponentName),
+                                                "Schema of the referenced 
component [%s] was not found.",
+                                                referencedComponentName);
+                                
referencedComponentSchema.setAdditionalProperties(
+                                        additionalPropertiesSchema);
+                            });
+        }
+        return schema;
     }
 
     private static PathItem.HttpMethod convert(HttpMethodWrapper wrapper) {
diff --git 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java
index 3202c4a75f6..f05581fcd81 100644
--- 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java
+++ 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java
@@ -19,7 +19,6 @@
 package org.apache.flink.docs.rest;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.runtime.rest.RestServerEndpoint;
 import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationResult;
 import 
org.apache.flink.runtime.rest.handler.async.AsynchronousOperationStatusMessageHeaders;
@@ -40,6 +39,7 @@ import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMap
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.module.jsonSchema.JsonSchema;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.module.jsonSchema.JsonSchemaGenerator;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.module.jsonSchema.types.ObjectSchema;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,9 +52,7 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 
 import static org.apache.flink.docs.util.Utils.escapeCharacters;
 
@@ -107,23 +105,16 @@ public class RestAPIDocGenerator {
     static void createHtmlFile(
             DocumentingRestEndpoint restEndpoint, RestAPIVersion apiVersion, 
Path outputFile)
             throws IOException {
-        StringBuilder html = new StringBuilder();
-
-        List<MessageHeaders> specs =
-                restEndpoint.getSpecs().stream()
-                        .filter(spec -> 
spec.getSupportedAPIVersions().contains(apiVersion))
-                        .filter(RestAPIDocGenerator::shouldBeDocumented)
-                        .collect(Collectors.toList());
-        specs.forEach(spec -> html.append(createHtmlEntry(spec)));
-
+        final StringBuilder html = new StringBuilder();
+        restEndpoint.getSpecs().stream()
+                .filter(spec -> 
spec.getSupportedAPIVersions().contains(apiVersion))
+                .filter(ApiSpecGeneratorUtils::shouldBeDocumented)
+                .map(RestAPIDocGenerator::createHtmlEntry)
+                .forEach(html::append);
         Files.deleteIfExists(outputFile);
         Files.write(outputFile, 
html.toString().getBytes(StandardCharsets.UTF_8));
     }
 
-    private static boolean shouldBeDocumented(MessageHeaders spec) {
-        return 
spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == 
null;
-    }
-
     private static String createHtmlEntry(MessageHeaders<?, ?, ?> spec) {
         Class<?> nestedAsyncOperationResultClass = null;
         if (spec instanceof AsynchronousOperationStatusMessageHeaders) {
@@ -253,7 +244,8 @@ public class RestAPIDocGenerator {
         return queryParameterList.toString();
     }
 
-    private static String createMessageHtmlEntry(
+    @VisibleForTesting
+    static String createMessageHtmlEntry(
             Class<?> messageClass,
             @Nullable Class<?> nestedAsyncOperationResultClass,
             Class<?> emptyMessageClass) {
@@ -266,6 +258,15 @@ public class RestAPIDocGenerator {
                     .put(AsynchronousOperationResult.FIELD_NAME_OPERATION, 
innerSchema);
         }
 
+        ApiSpecGeneratorUtils.findAdditionalFieldType(messageClass)
+                .map(RestAPIDocGenerator::generateSchema)
+                .ifPresent(
+                        additionalFieldSchema ->
+                                schema.asObjectSchema()
+                                        .setAdditionalProperties(
+                                                new 
ObjectSchema.SchemaAdditionalProperties(
+                                                        
additionalFieldSchema)));
+
         String json;
         if (messageClass == emptyMessageClass) {
             json = "{}";
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 6aa2b15f72d..884f584a93e 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
@@ -19,52 +19,52 @@
 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.util.FileUtils;
 
 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;
+import org.junit.jupiter.api.io.TempDir;
 
-import java.io.File;
+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.AssertionsForClassTypes.assertThat;
-import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Test class for {@link OpenApiSpecGenerator}. */
 class OpenApiSpecGeneratorTest {
 
     @Test
-    void testTitle() throws Exception {
+    void testTitle(@TempDir Path tmpDir) throws Exception {
         final String title = "Funky title";
 
-        File file = File.createTempFile("rest_v0_", ".html");
+        final Path file = tmpDir.resolve("openapi_spec.yaml");
         OpenApiSpecGenerator.createDocumentationFile(
-                title,
-                new TestExcludeDocumentingRestEndpoint(),
-                RuntimeRestAPIVersion.V0,
-                file.toPath());
-        String actual = FileUtils.readFile(file, "UTF-8");
+                title, new TestExcludeDocumentingRestEndpoint(), 
RuntimeRestAPIVersion.V0, file);
+        final String actual = new String(Files.readAllBytes(file), 
StandardCharsets.UTF_8);
 
         assertThat(actual).contains("title: " + title);
     }
 
     @Test
-    void testExcludeFromDocumentation() throws Exception {
-        File file = File.createTempFile("rest_v0_", ".html");
+    void testExcludeFromDocumentation(@TempDir Path tmpDir) throws Exception {
+        final Path file = tmpDir.resolve("openapi_spec.yaml");
         OpenApiSpecGenerator.createDocumentationFile(
-                "title",
-                new TestExcludeDocumentingRestEndpoint(),
-                RuntimeRestAPIVersion.V0,
-                file.toPath());
-        String actual = FileUtils.readFile(file, "UTF-8");
+                "title", new TestExcludeDocumentingRestEndpoint(), 
RuntimeRestAPIVersion.V0, file);
+        final String actual = new String(Files.readAllBytes(file), 
StandardCharsets.UTF_8);
 
         assertThat(actual).contains("/test/empty1");
         assertThat(actual).contains("This is a testing REST API.");
@@ -107,15 +107,15 @@ class OpenApiSpecGeneratorTest {
     }
 
     @Test
-    void testDuplicateOperationIdsAreRejected() throws Exception {
-        File file = File.createTempFile("rest_v0_", ".html");
+    void testDuplicateOperationIdsAreRejected(@TempDir Path tmpDir) {
+        final Path file = tmpDir.resolve("openapi_spec.yaml");
         assertThatThrownBy(
                         () ->
                                 OpenApiSpecGenerator.createDocumentationFile(
                                         "title",
                                         new 
TestDuplicateOperationIdDocumentingRestEndpoint(),
                                         RuntimeRestAPIVersion.V0,
-                                        file.toPath()))
+                                        file))
                 .isInstanceOf(IllegalStateException.class)
                 .hasMessageContaining("Duplicate OperationId");
     }
@@ -131,4 +131,27 @@ class OpenApiSpecGeneratorTest {
                     Tuple2.of(new TestEmptyMessageHeaders("operation1"), 
null));
         }
     }
+
+    @Test
+    void testAdditionalFields(@TempDir Path tmpDir) throws Exception {
+        final OpenAPI documentation =
+                OpenApiSpecGenerator.createDocumentation(
+                        "title", new TestAdditionalFieldsRestEndpoint(), 
RuntimeRestAPIVersion.V0);
+        assertThat(documentation.getComponents().getSchemas())
+                .extractingByKey("AdditionalFieldsRequestBody")
+                .satisfies(
+                        x ->
+                                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 d3fc475d23c..315b6af6019 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
@@ -18,10 +18,12 @@
 
 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;
@@ -60,6 +62,22 @@ class RestAPIDocGeneratorTest {
                         "This REST API should also not appear in the generated 
documentation.");
     }
 
+    @Test
+    void testAdditionalFields() {
+        final String messageHtmlEntry =
+                RestAPIDocGenerator.createMessageHtmlEntry(
+                        TestAdditionalFields.class, null, 
EmptyRequestBody.class);
+        assertThat(messageHtmlEntry)
+                .isEqualTo(
+                        "{\n"
+                                + "  \"type\" : \"object\",\n"
+                                + "  \"id\" : 
\"urn:jsonschema:org:apache:flink:docs:rest:RestAPIDocGeneratorTest:TestAdditionalFields\",\n"
+                                + "  \"additionalProperties\" : {\n"
+                                + "    \"type\" : \"string\"\n"
+                                + "  }\n"
+                                + "}");
+    }
+
     private static class TestExcludeDocumentingRestEndpoint implements 
DocumentingRestEndpoint {
 
         @Override
@@ -86,4 +104,7 @@ class RestAPIDocGeneratorTest {
                             null));
         }
     }
+
+    @FlinkJsonSchema.AdditionalFields(type = String.class)
+    private static class TestAdditionalFields {}
 }
diff --git 
a/flink-docs/src/test/java/org/apache/flink/docs/rest/data/TestAdditionalFieldsMessageHeaders.java
 
b/flink-docs/src/test/java/org/apache/flink/docs/rest/data/TestAdditionalFieldsMessageHeaders.java
new file mode 100644
index 00000000000..b4139687004
--- /dev/null
+++ 
b/flink-docs/src/test/java/org/apache/flink/docs/rest/data/TestAdditionalFieldsMessageHeaders.java
@@ -0,0 +1,110 @@
+/*
+ * 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.docs.rest.data;
+
+import org.apache.flink.annotation.docs.FlinkJsonSchema;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.RequestBody;
+import org.apache.flink.runtime.rest.messages.RuntimeMessageHeaders;
+import org.apache.flink.runtime.rest.versioning.RuntimeRestAPIVersion;
+
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * A {@link MessageHeaders} for testing purpose. Its request body contains 
additional fields,
+ * response body and message parameters are empty.
+ */
+public class TestAdditionalFieldsMessageHeaders
+        implements RuntimeMessageHeaders<
+                TestAdditionalFieldsMessageHeaders.AdditionalFieldsRequestBody,
+                EmptyResponseBody,
+                EmptyMessageParameters> {
+
+    /** Testing request body. */
+    @FlinkJsonSchema.AdditionalFields(type = String.class)
+    public static class AdditionalFieldsRequestBody implements RequestBody {}
+
+    private static final String URL = "/test/additional-fields";
+    private static final String DESCRIPTION = "This is an testing REST API 
with additional fields.";
+
+    private final String url;
+    private final String description;
+    private final String operationId;
+
+    public TestAdditionalFieldsMessageHeaders(String operationId) {
+        this(URL, DESCRIPTION, operationId);
+    }
+
+    private TestAdditionalFieldsMessageHeaders(String url, String description, 
String operationId) {
+        this.url = url;
+        this.description = description;
+        this.operationId = operationId;
+    }
+
+    @Override
+    public Class<AdditionalFieldsRequestBody> getRequestClass() {
+        return AdditionalFieldsRequestBody.class;
+    }
+
+    @Override
+    public Class<EmptyResponseBody> getResponseClass() {
+        return EmptyResponseBody.class;
+    }
+
+    @Override
+    public HttpMethodWrapper getHttpMethod() {
+        return HttpMethodWrapper.PUT;
+    }
+
+    @Override
+    public HttpResponseStatus getResponseStatusCode() {
+        return HttpResponseStatus.OK;
+    }
+
+    @Override
+    public String operationId() {
+        return operationId;
+    }
+
+    @Override
+    public String getDescription() {
+        return description;
+    }
+
+    @Override
+    public EmptyMessageParameters getUnresolvedMessageParameters() {
+        return EmptyMessageParameters.getInstance();
+    }
+
+    @Override
+    public String getTargetRestEndpointURL() {
+        return url;
+    }
+
+    @Override
+    public Collection<RuntimeRestAPIVersion> getSupportedAPIVersions() {
+        return Collections.singleton(RuntimeRestAPIVersion.V0);
+    }
+}

Reply via email to