提交 610fde72 编写于 作者: Z zentol

[FLINK-8133][REST][docs] Generate REST API documentation

This closes #5052.
上级 a3fd548e
此差异已折叠。
......@@ -674,4 +674,12 @@ Response:
{"jobid": "869a9868d49c679e7355700e0857af85"}
~~~
## FLIP-6
The following is the REST API documentation for FLIP-6.
### Dispatcher
{% include generated/rest_dispatcher.html %}
{% top %}
<!--
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.
-->
# Documentation generators
This module contains generators that create HTML files directly from Flink's source code.
## REST API documentation
The `RestAPIDocGenerator` can be used to generate a full reference of the REST API of a `RestServerEndpoint`. A separate file is generated for each endpoint.
To integrate a new endpoint into the generator
1. Add a new `DocumentingRestEndpoint` class to `RestAPIDocGenerator` that extends the new endpoint class
2. Add another call to `createHtmlFile` in `RestAPIDocGenerator#main`
3. Regenerate the documentation by running `mvn package -Dgenerate-rest-docs -nsu`
4. Integrate the generated file into the REST API documentation by adding `{% include generated/<file-name>.html %}` to the corresponding markdown file.
The documentation must be regenerated whenever
* a handler is added to/removed from a `RestServerEndpoint`
* any used `MessageHeaders` class or any referenced `RequestBody`, `ResponseBody`, `MessageParameters` or `MessageParameter` class is modified.
\ No newline at end of file
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-parent</artifactId>
<version>1.5-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
<artifactId>flink-docs</artifactId>
<name>flink-docs</name>
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-java</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-netty</artifactId>
</dependency>
<dependency>
<!-- We use standard jackson since jackson-module-jsonSchema isn't part of flink-shaded-jackson -->
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<version>${jackson.version}</version>
</dependency>
<dependency>
<!-- We use standard jackson since jackson-module-jsonSchema isn't part of flink-shaded-jackson -->
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>${jackson.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.module</groupId>
<artifactId>jackson-module-jsonSchema</artifactId>
<version>${jackson.version}</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</dependency>
</dependencies>
<profiles>
<profile>
<id>generate-rest-docs</id>
<activation>
<property>
<name>generate-rest-docs</name>
</property>
</activation>
<build>
<plugins>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<version>1.7</version>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
<configuration>
<target>
<mkdir dir="${rootDir}/${generated.docs.dir}"/>
<java classname="org.apache.flink.docs.rest.RestAPIDocGenerator" fork="true">
<classpath refid="maven.compile.classpath"/>
<arg value="${rootDir}/${generated.docs.dir}/"/>
</java>
</target>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>
/*
* 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.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.concurrent.Executors;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
import org.apache.flink.runtime.rest.RestServerEndpoint;
import org.apache.flink.runtime.rest.RestServerEndpointConfiguration;
import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
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.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
import org.apache.flink.util.ConfigurationException;
import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.module.jsonSchema.JsonSchema;
import com.fasterxml.jackson.module.jsonSchema.JsonSchemaGenerator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.stream.Collectors;
/**
* Generator for the Rest API documentation.
*
* <p>One HTML file is generated for each {@link RestServerEndpoint} implementation
* that can be embedded into .md files using {@code {% include ${generated.docs.dir}/file.html %}}.
* Each file contains a series of HTML tables, one for each REST call.
*
* <p>The generated table for each REST call looks like this:
* ----------------------------------------------------------
* | URL |
* ----------------------------------------------------------
* | Verb: verb (GET|POST|...) | Response code: responseCode|
* ----------------------------------------------------------
* | Path parameters (if any are defined) |
* ----------------------------------------------------------
* | - parameterName: description |
* | ... |
* ----------------------------------------------------------
* | Query parameters (if any are defined) |
* ----------------------------------------------------------
* | - parameterName (requisiteness): description |
* | ... |
* ----------------------------------------------------------
* | Request json schema (a collapsible "Request" button) |
* ----------------------------------------------------------
* | Response json schema (a collapsible "Response" button) |
* ----------------------------------------------------------
*/
public class RestAPIDocGenerator {
private static final Logger LOG = LoggerFactory.getLogger(RestAPIDocGenerator.class);
private static final ObjectMapper mapper;
private static final JsonSchemaGenerator schemaGen;
static {
mapper = new ObjectMapper();
schemaGen = new JsonSchemaGenerator(mapper);
}
/**
* Generates the REST API documentation.
*
* @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 {
String outputDirectory = args[0];
createHtmlFile(new DocumentingDispatcherRestEndpoint(), Paths.get(outputDirectory, "rest_dispatcher.html"));
}
private static void createHtmlFile(DocumentingRestEndpoint restEndpoint, Path outputFile) throws IOException {
StringBuilder html = new StringBuilder();
List<MessageHeaders> specs = restEndpoint.getSpecs();
specs.forEach(spec -> html.append(createHtmlEntry(spec)));
if (Files.exists(outputFile)) {
Files.delete(outputFile);
}
Files.write(outputFile, html.toString().getBytes(StandardCharsets.UTF_8));
}
private static String createHtmlEntry(MessageHeaders<?, ?, ?> spec) {
String requestEntry = createMessageHtmlEntry(
spec.getRequestClass(),
EmptyRequestBody.class);
String responseEntry = createMessageHtmlEntry(
spec.getResponseClass(),
EmptyResponseBody.class);
String pathParameterList = createPathParameterHtmlList(spec.getUnresolvedMessageParameters().getPathParameters());
String queryParameterList = createQueryParameterHtmlList(spec.getUnresolvedMessageParameters().getQueryParameters());
StringBuilder sb = new StringBuilder();
{
sb.append("<table class=\"table table-bordered\">\n");
sb.append(" <tbody>\n");
sb.append(" <tr>\n");
sb.append(" <td class=\"text-left\" colspan=\"2\"><strong>" + spec.getTargetRestEndpointURL() + "</strong></td>\n");
sb.append(" </tr>\n");
sb.append(" <tr>\n");
sb.append(" <td class=\"text-left\" style=\"width: 20%\">Verb: <code>" + spec.getHttpMethod() + "</code></td>\n");
sb.append(" <td class=\"text-left\">Response code: <code>" + spec.getResponseStatusCode() + "</code></td>\n");
sb.append(" </tr>\n");
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">" + "description" + "</td>\n");
sb.append(" </tr>\n");
}
if (!pathParameterList.isEmpty()) {
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">Path parameters</td>\n");
sb.append(" </tr>\n");
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">\n");
sb.append(" <ul>\n");
sb.append(pathParameterList);
sb.append(" </ul>\n");
sb.append(" </td>\n");
sb.append(" </tr>\n");
}
if (!queryParameterList.isEmpty()) {
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">Query parameters</td>\n");
sb.append(" </tr>\n");
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">\n");
sb.append(" <ul>\n");
sb.append(queryParameterList);
sb.append(" </ul>\n");
sb.append(" </td>\n");
sb.append(" </tr>\n");
}
int reqHash = spec.getTargetRestEndpointURL().hashCode() + spec.getHttpMethod().hashCode() + spec.getRequestClass().getCanonicalName().hashCode();
int resHash = spec.getTargetRestEndpointURL().hashCode() + spec.getHttpMethod().hashCode() + spec.getResponseClass().getCanonicalName().hashCode();
{
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">\n");
sb.append(" <button data-toggle=\"collapse\" data-target=\"#" + reqHash + "\">Request</button>\n");
sb.append(" <div id=\"" + reqHash + "\" class=\"collapse\">\n");
sb.append(" <pre>\n");
sb.append(" <code>\n");
sb.append(requestEntry);
sb.append(" </code>\n");
sb.append(" </pre>\n");
sb.append(" </div>\n");
sb.append(" </td>\n");
sb.append(" </tr>\n");
sb.append(" <tr>\n");
sb.append(" <td colspan=\"2\">\n");
sb.append(" <button data-toggle=\"collapse\" data-target=\"#" + resHash + "\">Response</button>\n");
sb.append(" <div id=\"" + resHash + "\" class=\"collapse\">\n");
sb.append(" <pre>\n");
sb.append(" <code>\n");
sb.append(responseEntry);
sb.append(" </code>\n");
sb.append(" </pre>\n");
sb.append(" </div>\n");
sb.append(" </td>\n");
sb.append(" </tr>\n");
sb.append(" </tbody>\n");
sb.append("</table>\n");
}
return sb.toString();
}
private static String createPathParameterHtmlList(Collection<MessagePathParameter<?>> pathParameters) {
StringBuilder pathParameterList = new StringBuilder();
pathParameters.forEach(messagePathParameter ->
pathParameterList.append(
String.format("<li><code>%s</code> - %s</li>\n",
messagePathParameter.getKey(),
"description")
));
return pathParameterList.toString();
}
private static String createQueryParameterHtmlList(Collection<MessageQueryParameter<?>> queryParameters) {
StringBuilder queryParameterList = new StringBuilder();
queryParameters.stream()
.sorted((param1, param2) -> Boolean.compare(param1.isMandatory(), param2.isMandatory()))
.forEach(parameter ->
queryParameterList.append(
String.format("<li><code>%s</code> (%s): %s</li>\n",
parameter.getKey(),
parameter.isMandatory() ? "mandatory" : "optional",
"description")
));
return queryParameterList.toString();
}
private static String createMessageHtmlEntry(Class<?> messageClass, Class<?> emptyMessageClass) {
JsonSchema schema;
try {
schema = schemaGen.generateSchema(messageClass);
} catch (JsonProcessingException e) {
LOG.error("Failed to generate message schema for class {}.", messageClass, e);
throw new RuntimeException("Failed to generate message schema for class " + messageClass.getCanonicalName() + ".", e);
}
String json;
if (messageClass == emptyMessageClass) {
json = "{}";
} else {
try {
json = mapper.writerWithDefaultPrettyPrinter()
.writeValueAsString(schema);
} catch (JsonProcessingException e) {
LOG.error("Failed to write message schema for class {}.", messageClass.getCanonicalName(), e);
throw new RuntimeException("Failed to write message schema for class " + messageClass.getCanonicalName() + ".", e);
}
}
return json;
}
/**
* Utility class to extract the {@link MessageHeaders} that the {@link DispatcherRestEndpoint} supports.
*/
private static class DocumentingDispatcherRestEndpoint extends DispatcherRestEndpoint implements DocumentingRestEndpoint {
private static final Configuration config;
private static final RestServerEndpointConfiguration restConfig;
private static final RestHandlerConfiguration handlerConfig;
private static final Executor executor;
private static final GatewayRetriever<DispatcherGateway> dispatcherGatewayRetriever;
private static final GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever;
private static final MetricQueryServiceRetriever metricQueryServiceRetriever;
static {
config = new Configuration();
try {
restConfig = RestServerEndpointConfiguration.fromConfiguration(config);
} catch (ConfigurationException e) {
throw new RuntimeException("Implementation error. RestServerEndpointConfiguration#fromConfiguration failed for default configuration.");
}
handlerConfig = RestHandlerConfiguration.fromConfiguration(config);
executor = Executors.directExecutor();
dispatcherGatewayRetriever = () -> null;
resourceManagerGatewayRetriever = () -> null;
metricQueryServiceRetriever = path -> null;
}
private DocumentingDispatcherRestEndpoint() {
super(restConfig, dispatcherGatewayRetriever, config, handlerConfig, resourceManagerGatewayRetriever, executor, metricQueryServiceRetriever);
}
@Override
public List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
return super.initializeHandlers(restAddressFuture);
}
}
/**
* Interface to expose the supported {@link MessageHeaders} of a {@link RestServerEndpoint}.
*/
private interface DocumentingRestEndpoint {
List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture);
default List<MessageHeaders> getSpecs() {
Comparator<String> comparator = new RestServerEndpoint.RestHandlerUrlComparator.CaseInsensitiveOrderComparator();
return initializeHandlers(CompletableFuture.completedFuture(null)).stream()
.map(tuple -> tuple.f0)
.filter(spec -> spec instanceof MessageHeaders)
.map(spec -> (MessageHeaders) spec)
.sorted((spec1, spec2) -> comparator.compare(spec1.getTargetRestEndpointURL(), spec2.getTargetRestEndpointURL()))
.collect(Collectors.toList());
}
}
}
......@@ -299,10 +299,10 @@ public abstract class RestServerEndpoint {
* /jobs/:jobid/config
* /:*
*
* <p>IMPORTANT: This comparator is highly specific to how Netty path parameter are encoded. Namely
* via a preceding ':' character.
* <p>IMPORTANT: This comparator is highly specific to how Netty path parameters are encoded. Namely
* with a preceding ':' character.
*/
static final class RestHandlerUrlComparator implements Comparator<Tuple2<RestHandlerSpecification, ChannelInboundHandler>>, Serializable {
public static final class RestHandlerUrlComparator implements Comparator<Tuple2<RestHandlerSpecification, ChannelInboundHandler>>, Serializable {
private static final long serialVersionUID = 2388466767835547926L;
......@@ -317,7 +317,21 @@ public abstract class RestServerEndpoint {
return CASE_INSENSITIVE_ORDER.compare(o1.f0.getTargetRestEndpointURL(), o2.f0.getTargetRestEndpointURL());
}
static final class CaseInsensitiveOrderComparator implements Comparator<String>, Serializable {
/**
* Comparator for Rest URLs.
*
* <p>The comparator orders the Rest URLs such that URLs with path parameters are ordered behind
* those without parameters. E.g.:
* /jobs
* /jobs/overview
* /jobs/:jobid
* /jobs/:jobid/config
* /:*
*
* <p>IMPORTANT: This comparator is highly specific to how Netty path parameters are encoded. Namely
* with a preceding ':' character.
*/
public static final class CaseInsensitiveOrderComparator implements Comparator<String>, Serializable {
private static final long serialVersionUID = 8550835445193437027L;
@Override
......
......@@ -83,6 +83,7 @@ under the License.
<module>flink-yarn</module>
<module>flink-yarn-tests</module>
<module>flink-fs-tests</module>
<module>flink-docs</module>
</modules>
<properties>
......@@ -1277,16 +1278,6 @@ under the License.
</execution>
</executions>
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
<configuration>
<filesets>
<fileset>
<directory>${generated.docs.dir}</directory>
</fileset>
</filesets>
</configuration>
</plugin>
</plugins>
<!-- Plugin configurations for plugins activated in sub-projects -->
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册