[ 
https://issues.apache.org/jira/browse/FLINK-8133?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16281658#comment-16281658
 ] 

ASF GitHub Bot commented on FLINK-8133:
---------------------------------------

Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5052#discussion_r155483740
  
    --- Diff: 
flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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>This class can be either invoked directly
    + *
    + * <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 | 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 hte {@link 
DispatcherRestEndpoint} supports.
    --- End diff --
    
    "hte" -> "the"


> Generate documentation for new REST API
> ---------------------------------------
>
>                 Key: FLINK-8133
>                 URL: https://issues.apache.org/jira/browse/FLINK-8133
>             Project: Flink
>          Issue Type: Improvement
>          Components: Documentation, REST
>            Reporter: Chesnay Schepler
>            Assignee: Chesnay Schepler
>            Priority: Blocker
>             Fix For: 1.5.0
>
>




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to