Skip to content

Commit

Permalink
[FLINK-31383] Add support for documenting additionProperties of the R…
Browse files Browse the repository at this point in the history
…EST API payloads.
  • Loading branch information
dmvk committed Mar 10, 2023
1 parent 3577edf commit 84d000c
Show file tree
Hide file tree
Showing 9 changed files with 341 additions and 49 deletions.
2 changes: 1 addition & 1 deletion docs/static/generated/rest_v1_dispatcher.yml
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
2 changes: 1 addition & 1 deletion docs/static/generated/rest_v1_sql_gateway.yml
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
Original file line number Diff line number Diff line change
@@ -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();
}
}
Original file line number Diff line number Diff line change
@@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -41,6 +40,7 @@
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;

Expand Down Expand Up @@ -113,6 +113,14 @@ static void createDocumentationFile(
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
Expand All @@ -124,7 +132,7 @@ static void createDocumentationFile(
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));
Expand All @@ -143,8 +151,7 @@ static void createDocumentationFile(
sortProperties(openApi);
sortSchemas(openApi);

Files.deleteIfExists(outputFile);
Files.write(outputFile, Yaml.pretty(openApi).getBytes(StandardCharsets.UTF_8));
return openApi;
}

@SuppressWarnings("rawtypes")
Expand Down Expand Up @@ -172,10 +179,6 @@ private static void sortSchemas(OpenAPI openApi) {
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(
Expand Down Expand Up @@ -452,7 +455,30 @@ private static Content createContentWithMediaType(String mediaType, Schema schem
}

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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -40,6 +39,7 @@
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;
Expand All @@ -52,9 +52,7 @@
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;

Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -253,7 +244,8 @@ private static String createQueryParameterHtmlList(
return queryParameterList.toString();
}

private static String createMessageHtmlEntry(
@VisibleForTesting
static String createMessageHtmlEntry(
Class<?> messageClass,
@Nullable Class<?> nestedAsyncOperationResultClass,
Class<?> emptyMessageClass) {
Expand All @@ -266,6 +258,15 @@ private static String createMessageHtmlEntry(
.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 = "{}";
Expand Down
Loading

0 comments on commit 84d000c

Please sign in to comment.