You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/18 11:54:15 UTC

[GitHub] [flink] dmvk commented on a change in pull request #17498: [FLINK-14954][rest] Add OpenAPI spec generator

dmvk commented on a change in pull request #17498:
URL: https://github.com/apache/flink/pull/17498#discussion_r730825023



##########
File path: flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.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;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.FileUploadHandler;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+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;
+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.MessagePathParameter;
+import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
+import org.apache.flink.runtime.rest.messages.TriggerId;
+import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders;
+import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
+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.ConfigurationException;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
+
+import io.swagger.v3.core.converter.AnnotatedType;
+import io.swagger.v3.core.converter.ModelConverterContext;
+import io.swagger.v3.core.converter.ModelConverterContextImpl;
+import io.swagger.v3.core.jackson.ModelResolver;
+import io.swagger.v3.core.util.Yaml;
+import io.swagger.v3.oas.models.Components;
+import io.swagger.v3.oas.models.OpenAPI;
+import io.swagger.v3.oas.models.Operation;
+import io.swagger.v3.oas.models.PathItem;
+import io.swagger.v3.oas.models.info.Contact;
+import io.swagger.v3.oas.models.info.Info;
+import io.swagger.v3.oas.models.info.License;
+import io.swagger.v3.oas.models.media.ArraySchema;
+import io.swagger.v3.oas.models.media.ComposedSchema;
+import io.swagger.v3.oas.models.media.Content;
+import io.swagger.v3.oas.models.media.MediaType;
+import io.swagger.v3.oas.models.media.Schema;
+import io.swagger.v3.oas.models.parameters.Parameter;
+import io.swagger.v3.oas.models.parameters.RequestBody;
+import io.swagger.v3.oas.models.responses.ApiResponse;
+import io.swagger.v3.oas.models.responses.ApiResponses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * OpenAPI spec generator for the Rest API.
+ *
+ * <p>One OpenAPI yml file is generated for each {@link RestServerEndpoint} implementation that can
+ * be embedded into .md files using {@code {% include ${generated.docs.dir}/file.yml %}}.
+ */
+public class OpenApiSpecGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OpenApiSpecGenerator.class);
+
+    private static final ModelConverterContext modelConverterContext;
+
+    static {
+        final ObjectMapper mapper =
+                new ObjectMapper().configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        modelConverterContext =
+                new ModelConverterContextImpl(Collections.singletonList(new ModelResolver(mapper)));
+    }
+
+    /**
+     * Generates the REST API OpenAPI spec.
+     *
+     * @param args args[0] contains the directory into which the generated files are placed
+     * @throws IOException if any file operation failed
+     */
+    public static void main(String[] args) throws IOException, ConfigurationException {
+        String outputDirectory = args[0];
+
+        for (final RestAPIVersion apiVersion : RestAPIVersion.values()) {
+            if (apiVersion == RestAPIVersion.V0) {
+                // this version exists only for testing purposes
+                continue;
+            }
+            createDocumentationFile(
+                    new DocumentingDispatcherRestEndpoint(),
+                    apiVersion,
+                    Paths.get(
+                            outputDirectory,
+                            "rest_" + apiVersion.getURLVersionPrefix() + "_dispatcher.yml"));
+        }
+    }
+
+    @VisibleForTesting
+    static void createDocumentationFile(
+            DocumentingRestEndpoint restEndpoint, RestAPIVersion apiVersion, Path outputFile)
+            throws IOException {
+        final OpenAPI openApi = new OpenAPI();
+
+        // eagerly initialize some data-structures to simplify operations later on
+        openApi.setPaths(new io.swagger.v3.oas.models.Paths());
+        openApi.setComponents(new Components());
+
+        setInfo(openApi, apiVersion);
+
+        List<MessageHeaders> specs =
+                restEndpoint.getSpecs().stream()
+                        .filter(spec -> spec.getSupportedAPIVersions().contains(apiVersion))
+                        .filter(OpenApiSpecGenerator::shouldBeDocumented)
+                        .collect(Collectors.toList());
+        specs.forEach(spec -> add(spec, openApi));
+
+        final List<Schema> asyncOperationSchemas = collectAsyncOperationResultVariants(specs);
+
+        // this adds the schema for every JSON object
+        openApi.components(
+                new Components().schemas(new HashMap<>(modelConverterContext.getDefinedModels())));
+
+        injectAsyncOperationResultSchema(openApi, asyncOperationSchemas);
+
+        overrideIdSchemas(openApi);
+
+        Files.deleteIfExists(outputFile);
+        Files.write(outputFile, Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
+    }
+
+    private static boolean shouldBeDocumented(MessageHeaders spec) {
+        return spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == null;
+    }
+
+    private static void setInfo(final OpenAPI openApi, final RestAPIVersion apiVersion) {
+        openApi.info(
+                new Info()
+                        .title("Flink JobManager REST API")
+                        .version(
+                                String.format(
+                                        "%s/%s",
+                                        apiVersion.getURLVersionPrefix(),
+                                        EnvironmentInformation.getVersion()))
+                        .contact(new Contact().email("users@flink.apache.org"))
+                        .license(
+                                new License()
+                                        .name("Apache 2.0")
+                                        .url("https://www.apache.org/licenses/LICENSE-2.0.html")));
+    }
+
+    private static List<Schema> collectAsyncOperationResultVariants(
+            final Collection<MessageHeaders> specs) {
+        return specs.stream()
+                .filter(spec -> spec instanceof AsynchronousOperationStatusMessageHeaders)
+                .map(
+                        spec ->
+                                ((AsynchronousOperationStatusMessageHeaders<?, ?>) spec)
+                                        .getValueClass())
+                .collect(Collectors.toSet())
+                .stream()
+                .sorted(
+                        new Comparator<Class<?>>() {
+                            @Override
+                            public int compare(Class<?> o1, Class<?> o2) {
+                                return o1.getSimpleName().compareTo(o2.getSimpleName());
+                            }
+                        })
+                .map(clazz -> getSchema(clazz))

Review comment:
       ```suggestion
                   .sorted(Comparator.comparing(Class::getSimpleName))
                   .map(OpenApiSpecGenerator::getSchema)
   ```

##########
File path: flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.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;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.FileUploadHandler;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+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;
+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.MessagePathParameter;
+import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
+import org.apache.flink.runtime.rest.messages.TriggerId;
+import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders;
+import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
+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.ConfigurationException;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
+
+import io.swagger.v3.core.converter.AnnotatedType;
+import io.swagger.v3.core.converter.ModelConverterContext;
+import io.swagger.v3.core.converter.ModelConverterContextImpl;
+import io.swagger.v3.core.jackson.ModelResolver;
+import io.swagger.v3.core.util.Yaml;
+import io.swagger.v3.oas.models.Components;
+import io.swagger.v3.oas.models.OpenAPI;
+import io.swagger.v3.oas.models.Operation;
+import io.swagger.v3.oas.models.PathItem;
+import io.swagger.v3.oas.models.info.Contact;
+import io.swagger.v3.oas.models.info.Info;
+import io.swagger.v3.oas.models.info.License;
+import io.swagger.v3.oas.models.media.ArraySchema;
+import io.swagger.v3.oas.models.media.ComposedSchema;
+import io.swagger.v3.oas.models.media.Content;
+import io.swagger.v3.oas.models.media.MediaType;
+import io.swagger.v3.oas.models.media.Schema;
+import io.swagger.v3.oas.models.parameters.Parameter;
+import io.swagger.v3.oas.models.parameters.RequestBody;
+import io.swagger.v3.oas.models.responses.ApiResponse;
+import io.swagger.v3.oas.models.responses.ApiResponses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * OpenAPI spec generator for the Rest API.
+ *
+ * <p>One OpenAPI yml file is generated for each {@link RestServerEndpoint} implementation that can
+ * be embedded into .md files using {@code {% include ${generated.docs.dir}/file.yml %}}.
+ */
+public class OpenApiSpecGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OpenApiSpecGenerator.class);
+
+    private static final ModelConverterContext modelConverterContext;
+
+    static {
+        final ObjectMapper mapper =
+                new ObjectMapper().configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        modelConverterContext =
+                new ModelConverterContextImpl(Collections.singletonList(new ModelResolver(mapper)));
+    }
+
+    /**
+     * Generates the REST API OpenAPI spec.
+     *
+     * @param args args[0] contains the directory into which the generated files are placed
+     * @throws IOException if any file operation failed
+     */
+    public static void main(String[] args) throws IOException, ConfigurationException {
+        String outputDirectory = args[0];
+
+        for (final RestAPIVersion apiVersion : RestAPIVersion.values()) {
+            if (apiVersion == RestAPIVersion.V0) {
+                // this version exists only for testing purposes
+                continue;
+            }
+            createDocumentationFile(
+                    new DocumentingDispatcherRestEndpoint(),
+                    apiVersion,
+                    Paths.get(
+                            outputDirectory,
+                            "rest_" + apiVersion.getURLVersionPrefix() + "_dispatcher.yml"));
+        }
+    }
+
+    @VisibleForTesting
+    static void createDocumentationFile(
+            DocumentingRestEndpoint restEndpoint, RestAPIVersion apiVersion, Path outputFile)
+            throws IOException {
+        final OpenAPI openApi = new OpenAPI();
+
+        // eagerly initialize some data-structures to simplify operations later on
+        openApi.setPaths(new io.swagger.v3.oas.models.Paths());
+        openApi.setComponents(new Components());
+
+        setInfo(openApi, apiVersion);
+
+        List<MessageHeaders> specs =
+                restEndpoint.getSpecs().stream()
+                        .filter(spec -> spec.getSupportedAPIVersions().contains(apiVersion))
+                        .filter(OpenApiSpecGenerator::shouldBeDocumented)
+                        .collect(Collectors.toList());
+        specs.forEach(spec -> add(spec, openApi));
+
+        final List<Schema> asyncOperationSchemas = collectAsyncOperationResultVariants(specs);
+
+        // this adds the schema for every JSON object
+        openApi.components(
+                new Components().schemas(new HashMap<>(modelConverterContext.getDefinedModels())));
+
+        injectAsyncOperationResultSchema(openApi, asyncOperationSchemas);
+
+        overrideIdSchemas(openApi);
+
+        Files.deleteIfExists(outputFile);
+        Files.write(outputFile, Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
+    }
+
+    private static boolean shouldBeDocumented(MessageHeaders spec) {
+        return spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == null;
+    }
+
+    private static void setInfo(final OpenAPI openApi, final RestAPIVersion apiVersion) {
+        openApi.info(
+                new Info()
+                        .title("Flink JobManager REST API")
+                        .version(
+                                String.format(
+                                        "%s/%s",
+                                        apiVersion.getURLVersionPrefix(),
+                                        EnvironmentInformation.getVersion()))
+                        .contact(new Contact().email("users@flink.apache.org"))

Review comment:
       ```suggestion
                           .contact(new Contact().email("user@flink.apache.org"))
   ```

##########
File path: flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.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;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.FileUploadHandler;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+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;
+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.MessagePathParameter;
+import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
+import org.apache.flink.runtime.rest.messages.TriggerId;
+import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders;
+import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
+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.ConfigurationException;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
+
+import io.swagger.v3.core.converter.AnnotatedType;
+import io.swagger.v3.core.converter.ModelConverterContext;
+import io.swagger.v3.core.converter.ModelConverterContextImpl;
+import io.swagger.v3.core.jackson.ModelResolver;
+import io.swagger.v3.core.util.Yaml;
+import io.swagger.v3.oas.models.Components;
+import io.swagger.v3.oas.models.OpenAPI;
+import io.swagger.v3.oas.models.Operation;
+import io.swagger.v3.oas.models.PathItem;
+import io.swagger.v3.oas.models.info.Contact;
+import io.swagger.v3.oas.models.info.Info;
+import io.swagger.v3.oas.models.info.License;
+import io.swagger.v3.oas.models.media.ArraySchema;
+import io.swagger.v3.oas.models.media.ComposedSchema;
+import io.swagger.v3.oas.models.media.Content;
+import io.swagger.v3.oas.models.media.MediaType;
+import io.swagger.v3.oas.models.media.Schema;
+import io.swagger.v3.oas.models.parameters.Parameter;
+import io.swagger.v3.oas.models.parameters.RequestBody;
+import io.swagger.v3.oas.models.responses.ApiResponse;
+import io.swagger.v3.oas.models.responses.ApiResponses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * OpenAPI spec generator for the Rest API.
+ *
+ * <p>One OpenAPI yml file is generated for each {@link RestServerEndpoint} implementation that can
+ * be embedded into .md files using {@code {% include ${generated.docs.dir}/file.yml %}}.
+ */
+public class OpenApiSpecGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OpenApiSpecGenerator.class);
+
+    private static final ModelConverterContext modelConverterContext;
+
+    static {
+        final ObjectMapper mapper =
+                new ObjectMapper().configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        modelConverterContext =
+                new ModelConverterContextImpl(Collections.singletonList(new ModelResolver(mapper)));
+    }
+
+    /**
+     * Generates the REST API OpenAPI spec.
+     *
+     * @param args args[0] contains the directory into which the generated files are placed
+     * @throws IOException if any file operation failed
+     */
+    public static void main(String[] args) throws IOException, ConfigurationException {
+        String outputDirectory = args[0];
+
+        for (final RestAPIVersion apiVersion : RestAPIVersion.values()) {
+            if (apiVersion == RestAPIVersion.V0) {
+                // this version exists only for testing purposes
+                continue;
+            }
+            createDocumentationFile(
+                    new DocumentingDispatcherRestEndpoint(),
+                    apiVersion,
+                    Paths.get(
+                            outputDirectory,
+                            "rest_" + apiVersion.getURLVersionPrefix() + "_dispatcher.yml"));
+        }
+    }
+
+    @VisibleForTesting
+    static void createDocumentationFile(
+            DocumentingRestEndpoint restEndpoint, RestAPIVersion apiVersion, Path outputFile)
+            throws IOException {
+        final OpenAPI openApi = new OpenAPI();
+
+        // eagerly initialize some data-structures to simplify operations later on
+        openApi.setPaths(new io.swagger.v3.oas.models.Paths());
+        openApi.setComponents(new Components());
+
+        setInfo(openApi, apiVersion);
+
+        List<MessageHeaders> specs =
+                restEndpoint.getSpecs().stream()
+                        .filter(spec -> spec.getSupportedAPIVersions().contains(apiVersion))
+                        .filter(OpenApiSpecGenerator::shouldBeDocumented)
+                        .collect(Collectors.toList());
+        specs.forEach(spec -> add(spec, openApi));
+
+        final List<Schema> asyncOperationSchemas = collectAsyncOperationResultVariants(specs);
+
+        // this adds the schema for every JSON object
+        openApi.components(
+                new Components().schemas(new HashMap<>(modelConverterContext.getDefinedModels())));
+
+        injectAsyncOperationResultSchema(openApi, asyncOperationSchemas);
+
+        overrideIdSchemas(openApi);
+
+        Files.deleteIfExists(outputFile);
+        Files.write(outputFile, Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
+    }
+
+    private static boolean shouldBeDocumented(MessageHeaders spec) {
+        return spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == null;
+    }
+
+    private static void setInfo(final OpenAPI openApi, final RestAPIVersion apiVersion) {
+        openApi.info(
+                new Info()
+                        .title("Flink JobManager REST API")
+                        .version(
+                                String.format(
+                                        "%s/%s",
+                                        apiVersion.getURLVersionPrefix(),
+                                        EnvironmentInformation.getVersion()))
+                        .contact(new Contact().email("users@flink.apache.org"))
+                        .license(
+                                new License()
+                                        .name("Apache 2.0")
+                                        .url("https://www.apache.org/licenses/LICENSE-2.0.html")));
+    }
+
+    private static List<Schema> collectAsyncOperationResultVariants(
+            final Collection<MessageHeaders> specs) {
+        return specs.stream()
+                .filter(spec -> spec instanceof AsynchronousOperationStatusMessageHeaders)
+                .map(
+                        spec ->
+                                ((AsynchronousOperationStatusMessageHeaders<?, ?>) spec)
+                                        .getValueClass())
+                .collect(Collectors.toSet())
+                .stream()
+                .sorted(
+                        new Comparator<Class<?>>() {
+                            @Override
+                            public int compare(Class<?> o1, Class<?> o2) {
+                                return o1.getSimpleName().compareTo(o2.getSimpleName());
+                            }
+                        })
+                .map(clazz -> getSchema(clazz))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * The {@link AsynchronousOperationResult} contains a generic 'operation' field that can't be
+     * properly extracted from swagger. This method injects these manually.
+     *
+     * <p>Resulting spec diff:
+     *
+     * <pre>
+     * AsynchronousOperationResult:
+     *   type: object
+     *   properties:
+     *     operation:
+     * -     type: object
+     * +       oneOf:
+     * +       - $ref: '#/components/schemas/AsynchronousOperationInfo'
+     * +       - $ref: '#/components/schemas/SavepointInfo'
+     * </pre>
+     */
+    private static void injectAsyncOperationResultSchema(
+            final OpenAPI openApi, List<Schema> asyncOperationSchemas) {
+        openApi.getComponents()
+                .getSchemas()
+                .get(AsynchronousOperationResult.class.getSimpleName())
+                .getProperties()
+                .put(
+                        AsynchronousOperationResult.FIELD_NAME_OPERATION,
+                        new ComposedSchema().oneOf(asyncOperationSchemas));
+    }
+
+    /**
+     * Various ID classes are effectively internal classes that aren't sufficiently annotated to
+     * work with automatic schema extraction. This method overrides the schema of these to a string
+     * regex pattern.
+     *
+     * <p>Resulting spec diff:
+     *
+     * <pre>
+     * JobID:
+     * - type: object
+     * - properties:
+     * -  upperPart:
+     * -     type: integer
+     * -     format: int64
+     * -   lowerPart:
+     * -     type: integer
+     * -     format: int64
+     * -   bytes:
+     * -     type: array
+     * -     items:
+     * -       type: string
+     * -       format: byte
+     * + pattern: "[0-9a-f]{32}"
+     * + type: string
+     * </pre>
+     */
+    private static void overrideIdSchemas(final OpenAPI openApi) {
+        final Schema idSchema = new Schema().type("string").pattern("[0-9a-f]{32}");
+
+        openApi.getComponents()
+                .addSchemas(JobID.class.getSimpleName(), idSchema)
+                .addSchemas(JobVertexID.class.getSimpleName(), idSchema)
+                .addSchemas(IntermediateDataSetID.class.getSimpleName(), idSchema)
+                .addSchemas(TriggerId.class.getSimpleName(), idSchema)
+                .addSchemas(ResourceID.class.getSimpleName(), idSchema);
+    }
+
+    private static void add(MessageHeaders<?, ?, ?> spec, OpenAPI openApi) {
+        final PathItem pathItem =
+                openApi.getPaths()
+                        .computeIfAbsent(
+                                // convert netty to openapi syntax
+                                // ":parameter" -> "{parameter}"
+                                spec.getTargetRestEndpointURL().replaceAll(":([\\w]+)", "{$1}"),
+                                ignored -> new PathItem());
+
+        final Operation operation = new Operation();
+
+        operation.description(spec.getDescription());
+
+        setParameters(operation, spec);
+        setRequest(operation, spec);
+        setResponse(operation, spec);
+
+        pathItem.operation(convert(spec.getHttpMethod()), operation);
+    }
+
+    private static void setParameters(
+            final Operation operation, final MessageHeaders<?, ?, ?> spec) {
+        List<Parameter> parameters = new ArrayList<>();
+        for (MessagePathParameter<?> pathParameter :
+                spec.getUnresolvedMessageParameters().getPathParameters()) {
+            parameters.add(
+                    new Parameter()
+                            .name(pathParameter.getKey())
+                            .in("path")
+                            .required(pathParameter.isMandatory())
+                            .description(pathParameter.getDescription())
+                            .schema(getSchema(getParameterType(pathParameter))));
+        }
+        for (MessageQueryParameter<?> queryParameter :
+                spec.getUnresolvedMessageParameters().getQueryParameters()) {
+            parameters.add(
+                    new Parameter()
+                            .name(queryParameter.getKey())
+                            .in("query")
+                            .required(queryParameter.isMandatory())
+                            .description(queryParameter.getDescription())
+                            .schema(getSchema(getParameterType(queryParameter)))
+                            .style(Parameter.StyleEnum.FORM));
+        }
+        if (!parameters.isEmpty()) {
+            operation.parameters(parameters);
+        }
+    }
+
+    private static <T extends MessagePathParameter<?>> Type getParameterType(T o) {
+        Class<?> clazz = o.getClass();
+        while (clazz.getSuperclass() != MessagePathParameter.class) {
+            clazz = clazz.getSuperclass();
+        }
+        return ((ParameterizedType) o.getClass().getGenericSuperclass())
+                .getActualTypeArguments()[0];
+    }
+
+    private static <T extends MessageQueryParameter<?>> Type getParameterType(T o) {
+        Class<?> clazz = o.getClass();
+        while (clazz.getSuperclass() != MessageQueryParameter.class) {
+            clazz = clazz.getSuperclass();
+        }
+        return ((ParameterizedType) clazz.getGenericSuperclass()).getActualTypeArguments()[0];
+    }
+
+    private static void setRequest(final Operation operation, final MessageHeaders<?, ?, ?> spec) {
+        // empty request bodies should not be documented at all
+        // additionally, hide legacy APIs that accepted parameters via request body
+        if (spec.getRequestClass() != EmptyRequestBody.class
+                && spec.getHttpMethod() != HttpMethodWrapper.GET) {
+            operation.requestBody(
+                    new RequestBody()
+                            .content(
+                                    createContentWithMediaType(
+                                            "application/json",
+                                            getSchema(spec.getRequestClass()))));
+        }
+
+        // files upload response schemas cannot be generated automatically; do it manually
+        if (spec.acceptsFileUploads()) {
+            injectFileUploadRequest(spec, operation);
+        }
+    }
+
+    private static void injectFileUploadRequest(
+            final MessageHeaders<?, ?, ?> spec, final Operation operation) {
+        // TODO: unhack

Review comment:
       what can we do to address these?

##########
File path: flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.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;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.FileUploadHandler;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+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;
+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.MessagePathParameter;
+import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
+import org.apache.flink.runtime.rest.messages.TriggerId;
+import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders;
+import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
+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.ConfigurationException;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
+
+import io.swagger.v3.core.converter.AnnotatedType;
+import io.swagger.v3.core.converter.ModelConverterContext;
+import io.swagger.v3.core.converter.ModelConverterContextImpl;
+import io.swagger.v3.core.jackson.ModelResolver;
+import io.swagger.v3.core.util.Yaml;
+import io.swagger.v3.oas.models.Components;
+import io.swagger.v3.oas.models.OpenAPI;
+import io.swagger.v3.oas.models.Operation;
+import io.swagger.v3.oas.models.PathItem;
+import io.swagger.v3.oas.models.info.Contact;
+import io.swagger.v3.oas.models.info.Info;
+import io.swagger.v3.oas.models.info.License;
+import io.swagger.v3.oas.models.media.ArraySchema;
+import io.swagger.v3.oas.models.media.ComposedSchema;
+import io.swagger.v3.oas.models.media.Content;
+import io.swagger.v3.oas.models.media.MediaType;
+import io.swagger.v3.oas.models.media.Schema;
+import io.swagger.v3.oas.models.parameters.Parameter;
+import io.swagger.v3.oas.models.parameters.RequestBody;
+import io.swagger.v3.oas.models.responses.ApiResponse;
+import io.swagger.v3.oas.models.responses.ApiResponses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * OpenAPI spec generator for the Rest API.
+ *
+ * <p>One OpenAPI yml file is generated for each {@link RestServerEndpoint} implementation that can
+ * be embedded into .md files using {@code {% include ${generated.docs.dir}/file.yml %}}.
+ */
+public class OpenApiSpecGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OpenApiSpecGenerator.class);
+
+    private static final ModelConverterContext modelConverterContext;
+
+    static {
+        final ObjectMapper mapper =
+                new ObjectMapper().configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        modelConverterContext =
+                new ModelConverterContextImpl(Collections.singletonList(new ModelResolver(mapper)));
+    }
+
+    /**
+     * Generates the REST API OpenAPI spec.
+     *
+     * @param args args[0] contains the directory into which the generated files are placed
+     * @throws IOException if any file operation failed
+     */
+    public static void main(String[] args) throws IOException, ConfigurationException {
+        String outputDirectory = args[0];
+
+        for (final RestAPIVersion apiVersion : RestAPIVersion.values()) {
+            if (apiVersion == RestAPIVersion.V0) {
+                // this version exists only for testing purposes
+                continue;
+            }
+            createDocumentationFile(
+                    new DocumentingDispatcherRestEndpoint(),
+                    apiVersion,
+                    Paths.get(
+                            outputDirectory,
+                            "rest_" + apiVersion.getURLVersionPrefix() + "_dispatcher.yml"));
+        }
+    }
+
+    @VisibleForTesting
+    static void createDocumentationFile(
+            DocumentingRestEndpoint restEndpoint, RestAPIVersion apiVersion, Path outputFile)
+            throws IOException {
+        final OpenAPI openApi = new OpenAPI();
+
+        // eagerly initialize some data-structures to simplify operations later on
+        openApi.setPaths(new io.swagger.v3.oas.models.Paths());
+        openApi.setComponents(new Components());
+
+        setInfo(openApi, apiVersion);
+
+        List<MessageHeaders> specs =
+                restEndpoint.getSpecs().stream()
+                        .filter(spec -> spec.getSupportedAPIVersions().contains(apiVersion))
+                        .filter(OpenApiSpecGenerator::shouldBeDocumented)
+                        .collect(Collectors.toList());
+        specs.forEach(spec -> add(spec, openApi));
+
+        final List<Schema> asyncOperationSchemas = collectAsyncOperationResultVariants(specs);

Review comment:
       nit: It would be nice if we can get rid of the raw types, but no hard feelings if we don't

##########
File path: flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.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;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.FileUploadHandler;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+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;
+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.MessagePathParameter;
+import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
+import org.apache.flink.runtime.rest.messages.TriggerId;
+import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders;
+import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
+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.ConfigurationException;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
+
+import io.swagger.v3.core.converter.AnnotatedType;
+import io.swagger.v3.core.converter.ModelConverterContext;
+import io.swagger.v3.core.converter.ModelConverterContextImpl;
+import io.swagger.v3.core.jackson.ModelResolver;
+import io.swagger.v3.core.util.Yaml;
+import io.swagger.v3.oas.models.Components;
+import io.swagger.v3.oas.models.OpenAPI;
+import io.swagger.v3.oas.models.Operation;
+import io.swagger.v3.oas.models.PathItem;
+import io.swagger.v3.oas.models.info.Contact;
+import io.swagger.v3.oas.models.info.Info;
+import io.swagger.v3.oas.models.info.License;
+import io.swagger.v3.oas.models.media.ArraySchema;
+import io.swagger.v3.oas.models.media.ComposedSchema;
+import io.swagger.v3.oas.models.media.Content;
+import io.swagger.v3.oas.models.media.MediaType;
+import io.swagger.v3.oas.models.media.Schema;
+import io.swagger.v3.oas.models.parameters.Parameter;
+import io.swagger.v3.oas.models.parameters.RequestBody;
+import io.swagger.v3.oas.models.responses.ApiResponse;
+import io.swagger.v3.oas.models.responses.ApiResponses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * OpenAPI spec generator for the Rest API.
+ *
+ * <p>One OpenAPI yml file is generated for each {@link RestServerEndpoint} implementation that can
+ * be embedded into .md files using {@code {% include ${generated.docs.dir}/file.yml %}}.
+ */
+public class OpenApiSpecGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OpenApiSpecGenerator.class);
+
+    private static final ModelConverterContext modelConverterContext;
+
+    static {
+        final ObjectMapper mapper =
+                new ObjectMapper().configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        modelConverterContext =
+                new ModelConverterContextImpl(Collections.singletonList(new ModelResolver(mapper)));
+    }
+
+    /**
+     * Generates the REST API OpenAPI spec.
+     *
+     * @param args args[0] contains the directory into which the generated files are placed
+     * @throws IOException if any file operation failed
+     */
+    public static void main(String[] args) throws IOException, ConfigurationException {
+        String outputDirectory = args[0];
+
+        for (final RestAPIVersion apiVersion : RestAPIVersion.values()) {
+            if (apiVersion == RestAPIVersion.V0) {
+                // this version exists only for testing purposes
+                continue;
+            }
+            createDocumentationFile(
+                    new DocumentingDispatcherRestEndpoint(),
+                    apiVersion,
+                    Paths.get(
+                            outputDirectory,
+                            "rest_" + apiVersion.getURLVersionPrefix() + "_dispatcher.yml"));
+        }
+    }
+
+    @VisibleForTesting
+    static void createDocumentationFile(
+            DocumentingRestEndpoint restEndpoint, RestAPIVersion apiVersion, Path outputFile)
+            throws IOException {
+        final OpenAPI openApi = new OpenAPI();
+
+        // eagerly initialize some data-structures to simplify operations later on
+        openApi.setPaths(new io.swagger.v3.oas.models.Paths());
+        openApi.setComponents(new Components());
+
+        setInfo(openApi, apiVersion);
+
+        List<MessageHeaders> specs =
+                restEndpoint.getSpecs().stream()
+                        .filter(spec -> spec.getSupportedAPIVersions().contains(apiVersion))
+                        .filter(OpenApiSpecGenerator::shouldBeDocumented)
+                        .collect(Collectors.toList());
+        specs.forEach(spec -> add(spec, openApi));
+
+        final List<Schema> asyncOperationSchemas = collectAsyncOperationResultVariants(specs);
+
+        // this adds the schema for every JSON object
+        openApi.components(
+                new Components().schemas(new HashMap<>(modelConverterContext.getDefinedModels())));
+
+        injectAsyncOperationResultSchema(openApi, asyncOperationSchemas);
+
+        overrideIdSchemas(openApi);
+
+        Files.deleteIfExists(outputFile);
+        Files.write(outputFile, Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
+    }
+
+    private static boolean shouldBeDocumented(MessageHeaders spec) {
+        return spec.getClass().getAnnotation(Documentation.ExcludeFromDocumentation.class) == null;
+    }
+
+    private static void setInfo(final OpenAPI openApi, final RestAPIVersion apiVersion) {
+        openApi.info(
+                new Info()
+                        .title("Flink JobManager REST API")
+                        .version(
+                                String.format(
+                                        "%s/%s",
+                                        apiVersion.getURLVersionPrefix(),
+                                        EnvironmentInformation.getVersion()))
+                        .contact(new Contact().email("users@flink.apache.org"))
+                        .license(
+                                new License()
+                                        .name("Apache 2.0")
+                                        .url("https://www.apache.org/licenses/LICENSE-2.0.html")));
+    }
+
+    private static List<Schema> collectAsyncOperationResultVariants(
+            final Collection<MessageHeaders> specs) {
+        return specs.stream()
+                .filter(spec -> spec instanceof AsynchronousOperationStatusMessageHeaders)
+                .map(
+                        spec ->
+                                ((AsynchronousOperationStatusMessageHeaders<?, ?>) spec)
+                                        .getValueClass())
+                .collect(Collectors.toSet())
+                .stream()

Review comment:
       ```suggestion
                   .distinct()
   ```

##########
File path: docs/content/docs/ops/rest_api.md
##########
@@ -66,9 +65,18 @@ Querying unsupported/non-existing versions will return a 404 error.
 
 There exist several async operations among these APIs, e.g. `trigger savepoint`, `rescale a job`. They would return a `triggerid` to identify the operation you just POST and then you need to use that `triggerid` to query for the status of the operation.
 
+### JobManager
+
+[OpenAPI specification](/generated/rest_v1_dispatcher.yml)
+
+{{< hint warning >}}
+The OpenAPI specification is still experimental.
+{{< /hint >}} XXX

Review comment:
       XXX?

##########
File path: flink-docs/src/main/java/org/apache/flink/docs/rest/OpenApiSpecGenerator.java
##########
@@ -0,0 +1,427 @@
+/*
+ * 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.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;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.FileUploadHandler;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+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;
+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.MessagePathParameter;
+import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
+import org.apache.flink.runtime.rest.messages.TriggerId;
+import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders;
+import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
+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.ConfigurationException;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature;
+
+import io.swagger.v3.core.converter.AnnotatedType;
+import io.swagger.v3.core.converter.ModelConverterContext;
+import io.swagger.v3.core.converter.ModelConverterContextImpl;
+import io.swagger.v3.core.jackson.ModelResolver;
+import io.swagger.v3.core.util.Yaml;
+import io.swagger.v3.oas.models.Components;
+import io.swagger.v3.oas.models.OpenAPI;
+import io.swagger.v3.oas.models.Operation;
+import io.swagger.v3.oas.models.PathItem;
+import io.swagger.v3.oas.models.info.Contact;
+import io.swagger.v3.oas.models.info.Info;
+import io.swagger.v3.oas.models.info.License;
+import io.swagger.v3.oas.models.media.ArraySchema;
+import io.swagger.v3.oas.models.media.ComposedSchema;
+import io.swagger.v3.oas.models.media.Content;
+import io.swagger.v3.oas.models.media.MediaType;
+import io.swagger.v3.oas.models.media.Schema;
+import io.swagger.v3.oas.models.parameters.Parameter;
+import io.swagger.v3.oas.models.parameters.RequestBody;
+import io.swagger.v3.oas.models.responses.ApiResponse;
+import io.swagger.v3.oas.models.responses.ApiResponses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * OpenAPI spec generator for the Rest API.
+ *
+ * <p>One OpenAPI yml file is generated for each {@link RestServerEndpoint} implementation that can
+ * be embedded into .md files using {@code {% include ${generated.docs.dir}/file.yml %}}.
+ */
+public class OpenApiSpecGenerator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OpenApiSpecGenerator.class);

Review comment:
       unused




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org