You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by dw...@apache.org on 2018/07/18 11:58:57 UTC

[2/2] flink git commit: [FLINK-9792] Added custom Description class for ConfigOptions to enable rich formatting.

[FLINK-9792] Added custom Description class for ConfigOptions to enable rich formatting.

This closes #6312


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f576381e
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f576381e
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f576381e

Branch: refs/heads/release-1.6
Commit: f576381e5fb45bf3cb4aaa54ba13ae11b4df3ca1
Parents: 969ab84
Author: Dawid Wysakowicz <dw...@apache.org>
Authored: Wed Jul 11 09:52:48 2018 +0200
Committer: Dawid Wysakowicz <dw...@apache.org>
Committed: Wed Jul 18 13:57:24 2018 +0200

----------------------------------------------------------------------
 .../_includes/generated/akka_configuration.html |   6 +-
 .../generated/metric_configuration.html         |  24 ++--
 .../generated/security_configuration.html       |   2 +-
 .../flink/configuration/ConfigOption.java       |  40 ++++++-
 .../configuration/description/BlockElement.java |  26 +++++
 .../configuration/description/Description.java  | 109 ++++++++++++++++++
 .../description/DescriptionElement.java         |  31 +++++
 .../configuration/description/Formatter.java    |  95 +++++++++++++++
 .../description/HtmlFormatter.java              |  62 ++++++++++
 .../description/InlineElement.java              |  26 +++++
 .../description/LineBreakElement.java           |  40 +++++++
 .../configuration/description/LinkElement.java  |  66 +++++++++++
 .../configuration/description/ListElement.java  |  59 ++++++++++
 .../configuration/description/TextElement.java  |  73 ++++++++++++
 .../description/DescriptionHtmlTest.java        | 115 +++++++++++++++++++
 .../ConfigOptionsDocGenerator.java              |   5 +-
 .../java/org/apache/flink/docs/util/Utils.java  |   4 +-
 .../ConfigOptionsDocGeneratorTest.java          |   7 +-
 .../ConfigOptionsDocsCompletenessITCase.java    |  12 +-
 19 files changed, 774 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/docs/_includes/generated/akka_configuration.html
----------------------------------------------------------------------
diff --git a/docs/_includes/generated/akka_configuration.html b/docs/_includes/generated/akka_configuration.html
index afaba6f..a2b49c7 100644
--- a/docs/_includes/generated/akka_configuration.html
+++ b/docs/_includes/generated/akka_configuration.html
@@ -80,17 +80,17 @@
         <tr>
             <td><h5>akka.watch.heartbeat.interval</h5></td>
             <td style="word-wrap: break-word;">"10 s"</td>
-            <td>Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch can be found &#60;a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector"&#62;here&#60;/a&#62;.</td>
+            <td>Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch can be found &lt;a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector"&gt;here&lt;/a&gt;.</td>
         </tr>
         <tr>
             <td><h5>akka.watch.heartbeat.pause</h5></td>
             <td style="word-wrap: break-word;">"60 s"</td>
-            <td>Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found &#60;a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector"&#62;here&#60;/a&#62;.</td>
+            <td>Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found &lt;a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector"&gt;here&lt;/a&gt;.</td>
         </tr>
         <tr>
             <td><h5>akka.watch.threshold</h5></td>
             <td style="word-wrap: break-word;">12</td>
-            <td>Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found &#60;a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector"&#62;here&#60;/a&#62;.</td>
+            <td>Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found &lt;a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector"&gt;here&lt;/a&gt;.</td>
         </tr>
     </tbody>
 </table>

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/docs/_includes/generated/metric_configuration.html
----------------------------------------------------------------------
diff --git a/docs/_includes/generated/metric_configuration.html b/docs/_includes/generated/metric_configuration.html
index 868a9bd..aef8fbb 100644
--- a/docs/_includes/generated/metric_configuration.html
+++ b/docs/_includes/generated/metric_configuration.html
@@ -13,19 +13,19 @@
             <td>Defines the number of measured latencies to maintain at each operator.</td>
         </tr>
         <tr>
-            <td><h5>metrics.reporter.&#60;name&#62;.&#60;parameter&#62;</h5></td>
+            <td><h5>metrics.reporter.&lt;name&gt;.&lt;parameter&gt;</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
-            <td>Configures the parameter &#60;parameter&#62; for the reporter named &#60;name&#62;.</td>
+            <td>Configures the parameter &lt;parameter&gt; for the reporter named &lt;name&gt;.</td>
         </tr>
         <tr>
-            <td><h5>metrics.reporter.&#60;name&#62;.class</h5></td>
+            <td><h5>metrics.reporter.&lt;name&gt;.class</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
-            <td>The reporter class to use for the reporter named &#60;name&#62;.</td>
+            <td>The reporter class to use for the reporter named &lt;name&gt;.</td>
         </tr>
         <tr>
-            <td><h5>metrics.reporter.&#60;name&#62;.interval</h5></td>
+            <td><h5>metrics.reporter.&lt;name&gt;.interval</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
-            <td>The reporter interval to use for the reporter named &#60;name&#62;.</td>
+            <td>The reporter interval to use for the reporter named &lt;name&gt;.</td>
         </tr>
         <tr>
             <td><h5>metrics.reporters</h5></td>
@@ -39,32 +39,32 @@
         </tr>
         <tr>
             <td><h5>metrics.scope.jm</h5></td>
-            <td style="word-wrap: break-word;">"&#60;host&#62;.jobmanager"</td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.jobmanager"</td>
             <td>Defines the scope format string that is applied to all metrics scoped to a JobManager.</td>
         </tr>
         <tr>
             <td><h5>metrics.scope.jm.job</h5></td>
-            <td style="word-wrap: break-word;">"&#60;host&#62;.jobmanager.&#60;job_name&#62;"</td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.jobmanager.&lt;job_name&gt;"</td>
             <td>Defines the scope format string that is applied to all metrics scoped to a job on a JobManager.</td>
         </tr>
         <tr>
             <td><h5>metrics.scope.operator</h5></td>
-            <td style="word-wrap: break-word;">"&#60;host&#62;.taskmanager.&#60;tm_id&#62;.&#60;job_name&#62;.&#60;operator_name&#62;.&#60;subtask_index&#62;"</td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;operator_name&gt;.&lt;subtask_index&gt;"</td>
             <td>Defines the scope format string that is applied to all metrics scoped to an operator.</td>
         </tr>
         <tr>
             <td><h5>metrics.scope.task</h5></td>
-            <td style="word-wrap: break-word;">"&#60;host&#62;.taskmanager.&#60;tm_id&#62;.&#60;job_name&#62;.&#60;task_name&#62;.&#60;subtask_index&#62;"</td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;.&lt;task_name&gt;.&lt;subtask_index&gt;"</td>
             <td>Defines the scope format string that is applied to all metrics scoped to a task.</td>
         </tr>
         <tr>
             <td><h5>metrics.scope.tm</h5></td>
-            <td style="word-wrap: break-word;">"&#60;host&#62;.taskmanager.&#60;tm_id&#62;"</td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.taskmanager.&lt;tm_id&gt;"</td>
             <td>Defines the scope format string that is applied to all metrics scoped to a TaskManager.</td>
         </tr>
         <tr>
             <td><h5>metrics.scope.tm.job</h5></td>
-            <td style="word-wrap: break-word;">"&#60;host&#62;.taskmanager.&#60;tm_id&#62;.&#60;job_name&#62;"</td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.taskmanager.&lt;tm_id&gt;.&lt;job_name&gt;"</td>
             <td>Defines the scope format string that is applied to all metrics scoped to a job on a TaskManager.</td>
         </tr>
     </tbody>

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/docs/_includes/generated/security_configuration.html
----------------------------------------------------------------------
diff --git a/docs/_includes/generated/security_configuration.html b/docs/_includes/generated/security_configuration.html
index 5042cf3..47846ae 100644
--- a/docs/_includes/generated/security_configuration.html
+++ b/docs/_includes/generated/security_configuration.html
@@ -10,7 +10,7 @@
         <tr>
             <td><h5>security.ssl.algorithms</h5></td>
             <td style="word-wrap: break-word;">"TLS_RSA_WITH_AES_128_CBC_SHA"</td>
-            <td>The comma separated list of standard SSL algorithms to be supported. Read more &#60;a href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites"&#62;here&#60;/a&#62;.</td>
+            <td>The comma separated list of standard SSL algorithms to be supported. Read more &lt;a href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites"&gt;here&lt;/a&gt;.</td>
         </tr>
         <tr>
             <td><h5>security.ssl.internal.enabled</h5></td>

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
index 8e7d79b..be242f5 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
@@ -19,6 +19,7 @@
 package org.apache.flink.configuration;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.description.Description;
 
 import java.util.Arrays;
 import java.util.Collections;
@@ -52,7 +53,7 @@ public class ConfigOption<T> {
 	private final T defaultValue;
 
 	/** The description for this option. */
-	private final String description;
+	private final Description description;
 
 	// ------------------------------------------------------------------------
 
@@ -64,7 +65,7 @@ public class ConfigOption<T> {
 	 */
 	ConfigOption(String key, T defaultValue) {
 		this.key = checkNotNull(key);
-		this.description = "";
+		this.description = Description.builder().text("").build();
 		this.defaultValue = defaultValue;
 		this.deprecatedKeys = EMPTY;
 	}
@@ -73,11 +74,29 @@ public class ConfigOption<T> {
 	 * Creates a new config option with deprecated keys.
 	 *
 	 * @param key             The current key for that config option
+	 * @param description     Description for that option
 	 * @param defaultValue    The default value for this option
 	 * @param deprecatedKeys  The list of deprecated keys, in the order to be checked
+	 * @deprecated use version with {@link Description} instead
 	 */
+	@Deprecated
 	ConfigOption(String key, String description, T defaultValue, String... deprecatedKeys) {
 		this.key = checkNotNull(key);
+		this.description = Description.builder().text(description).build();
+		this.defaultValue = defaultValue;
+		this.deprecatedKeys = deprecatedKeys == null || deprecatedKeys.length == 0 ? EMPTY : deprecatedKeys;
+	}
+
+	/**
+	 * Creates a new config option with deprecated keys.
+	 *
+	 * @param key             The current key for that config option
+	 * @param description     Description for that option
+	 * @param defaultValue    The default value for this option
+	 * @param deprecatedKeys  The list of deprecated keys, in the order to be checked
+	 */
+	ConfigOption(String key, Description description, T defaultValue, String... deprecatedKeys) {
+		this.key = checkNotNull(key);
 		this.description = description;
 		this.defaultValue = defaultValue;
 		this.deprecatedKeys = deprecatedKeys == null || deprecatedKeys.length == 0 ? EMPTY : deprecatedKeys;
@@ -104,15 +123,26 @@ public class ConfigOption<T> {
 	 * Creates a new config option, using this option's key and default value, and
 	 * adding the given description. The given description is used when generation the configuration documention.
 	 *
-	 * <p><b>NOTE:</b> You can use html to format the output of the generated cell.
-	 *
 	 * @param description The description for this option.
 	 * @return A new config option, with given description.
+	 * @deprecated use version with {@link Description}
 	 */
+	@Deprecated
 	public ConfigOption<T> withDescription(final String description) {
 		return new ConfigOption<>(key, description, defaultValue, deprecatedKeys);
 	}
 
+	/**
+	 * Creates a new config option, using this option's key and default value, and
+	 * adding the given description. The given description is used when generation the configuration documention.
+	 *
+	 * @param description The description for this option.
+	 * @return A new config option, with given description.
+	 */
+	public ConfigOption<T> withDescription(final Description description) {
+		return new ConfigOption<>(key, description, defaultValue, deprecatedKeys);
+	}
+
 	// ------------------------------------------------------------------------
 
 	/**
@@ -159,7 +189,7 @@ public class ConfigOption<T> {
 	 * Returns the description of this option.
 	 * @return The option's description.
 	 */
-	public String description() {
+	public Description description() {
 		return description;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java
new file mode 100644
index 0000000..c55b496
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java
@@ -0,0 +1,26 @@
+/*
+ * 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.configuration.description;
+
+/**
+ * Part of description that represents a block e.g. some text, linebreak or a list.
+ */
+public interface BlockElement extends DescriptionElement {
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java b/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java
new file mode 100644
index 0000000..25a6a64
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java
@@ -0,0 +1,109 @@
+/*
+ * 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.configuration.description;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Description for {@link org.apache.flink.configuration.ConfigOption}. Allows providing multiple rich formats.
+ */
+public class Description {
+
+	private final List<BlockElement> blocks;
+
+	public static DescriptionBuilder builder() {
+		return new DescriptionBuilder();
+	}
+
+	public List<BlockElement> getBlocks() {
+		return blocks;
+	}
+
+	/**
+	 * Builder for {@link Description}. Allows adding a rich formatting like lists, links, linebreaks etc.
+	 * For example:
+	 * <pre>{@code
+	 * Description description = Description.builder()
+	 * 	.text("This is some list: ")
+	 * 	.list(
+	 * 		text("this is first element of list"),
+	 * 		text("this is second element of list with a %s", link("https://link")))
+	 * 	.build();
+	 * }</pre>
+	 */
+	public static class DescriptionBuilder {
+
+		private final List<BlockElement> blocks = new ArrayList<>();
+
+		/**
+		 * Adds a block of text with placeholders ("%s") that will be replaced with proper string representation of
+		 * given {@link InlineElement}. For example:
+		 *
+		 * <p>{@code text("This is a text with a link %s", link("https://somepage", "to here"))}
+		 *
+		 * @param format   text with placeholders for elements
+		 * @param elements elements to be put in the text
+		 * @return description with added block of text
+		 */
+		public DescriptionBuilder text(String format, InlineElement... elements) {
+			blocks.add(TextElement.text(format, elements));
+			return this;
+		}
+
+		/**
+		 * Creates a simple block of text.
+		 *
+		 * @param text a simple block of text
+		 * @return block of text
+		 */
+		public DescriptionBuilder text(String text) {
+			blocks.add(TextElement.text(text));
+			return this;
+		}
+
+		/**
+		 * Creates a line break in the description.
+		 */
+		public DescriptionBuilder linebreak() {
+			blocks.add(LineBreakElement.linebreak());
+			return this;
+		}
+
+		/**
+		 * Adds a bulleted list to the description.
+		 */
+		public DescriptionBuilder list(InlineElement... elements) {
+			blocks.add(ListElement.list(elements));
+			return this;
+		}
+
+		/**
+		 * Creates description representation.
+		 */
+		public Description build() {
+			return new Description(blocks);
+		}
+
+	}
+
+	private Description(List<BlockElement> blocks) {
+		this.blocks = blocks;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java
new file mode 100644
index 0000000..7a889f4
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java
@@ -0,0 +1,31 @@
+/*
+ * 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.configuration.description;
+
+/**
+ * Part of a {@link Description} that can be converted into String representation.
+ */
+interface DescriptionElement {
+	/**
+	 * Transforms itself into String representation using given format.
+	 *
+	 * @param formatter formatter to use.
+	 */
+	void format(Formatter formatter);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java b/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java
new file mode 100644
index 0000000..d3fcf40
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java
@@ -0,0 +1,95 @@
+/*
+ * 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.configuration.description;
+
+/**
+ * Allows providing multiple formatters for the description. E.g. Html formatter, Markdown formatter etc.
+ */
+public abstract class Formatter {
+
+	private final StringBuilder state = new StringBuilder();
+
+	/**
+	 * Formats the description into a String using format specific tags.
+	 *
+	 * @param description description to be formatted
+	 * @return string representation of the description
+	 */
+	public String format(Description description) {
+		for (BlockElement blockElement : description.getBlocks()) {
+			blockElement.format(this);
+		}
+		return finalizeFormatting();
+	}
+
+	public void format(LinkElement element) {
+		formatLink(state, element.getLink(), element.getText());
+	}
+
+	public void format(TextElement element) {
+		String[] inlineElements = element.getElements().stream().map(el -> {
+				Formatter formatter = newInstance();
+				el.format(formatter);
+				return formatter.finalizeFormatting();
+			}
+		).toArray(String[]::new);
+		formatText(state, escapeFormatPlaceholder(element.getFormat()), inlineElements);
+	}
+
+	public void format(LineBreakElement element) {
+		formatLineBreak(state);
+	}
+
+	public void format(ListElement element) {
+		String[] inlineElements = element.getEntries().stream().map(el -> {
+				Formatter formatter = newInstance();
+				el.format(formatter);
+				return formatter.finalizeFormatting();
+			}
+		).toArray(String[]::new);
+		formatList(state, inlineElements);
+	}
+
+	private String finalizeFormatting() {
+		String result = state.toString();
+		state.setLength(0);
+		return result.replaceAll("%%", "%");
+	}
+
+	protected abstract void formatLink(StringBuilder state, String link, String description);
+
+	protected abstract void formatLineBreak(StringBuilder state);
+
+	protected abstract void formatText(StringBuilder state, String format, String[] elements);
+
+	protected abstract void formatList(StringBuilder state, String[] entries);
+
+	protected abstract Formatter newInstance();
+
+	private static final String TEMPORARY_PLACEHOLDER = "randomPlaceholderForStringFormat";
+
+	private static String escapeFormatPlaceholder(String value) {
+		return value
+			.replaceAll("%s", TEMPORARY_PLACEHOLDER)
+			.replaceAll("%", "%%")
+			.replaceAll(TEMPORARY_PLACEHOLDER, "%s");
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java b/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java
new file mode 100644
index 0000000..72531ab
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration.description;
+
+/**
+ * Formatter that transforms {@link Description} into Html representation.
+ */
+public class HtmlFormatter extends Formatter {
+
+	@Override
+	protected void formatLink(StringBuilder state, String link, String description) {
+		state.append(String.format("<a href=\"%s\">%s</a>", link, description));
+	}
+
+	@Override
+	protected void formatLineBreak(StringBuilder state) {
+		state.append("<br/>");
+	}
+
+	@Override
+	protected void formatText(StringBuilder state, String format, String[] elements) {
+		String escapedFormat = escapeCharacters(format);
+		state.append(String.format(escapedFormat, elements));
+	}
+
+	@Override
+	protected void formatList(StringBuilder state, String[] entries) {
+		state.append("<ul>");
+		for (String entry : entries) {
+			state.append(String.format("<li>%s</li>", entry));
+		}
+		state.append("</ul>");
+	}
+
+	@Override
+	protected Formatter newInstance() {
+		return new HtmlFormatter();
+	}
+
+	private static String escapeCharacters(String value) {
+		return value
+			.replaceAll("<", "&lt;")
+			.replaceAll(">", "&gt;");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java
new file mode 100644
index 0000000..2a21875
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java
@@ -0,0 +1,26 @@
+/*
+ * 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.configuration.description;
+
+/**
+ * Part of description that represents an element inside a block e.g. a link.
+ */
+public interface InlineElement extends DescriptionElement {
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java
new file mode 100644
index 0000000..4a3de9c
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java
@@ -0,0 +1,40 @@
+/*
+ * 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.configuration.description;
+
+/**
+ * Represents a line break in the {@link Description}.
+ */
+public class LineBreakElement implements BlockElement {
+
+	/**
+	 * Creates a line break in the description.
+	 */
+	public static LineBreakElement linebreak() {
+		return new LineBreakElement();
+	}
+
+	private LineBreakElement() {
+	}
+
+	@Override
+	public void format(Formatter formatter) {
+		formatter.format(this);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java
new file mode 100644
index 0000000..778844e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java
@@ -0,0 +1,66 @@
+/*
+ * 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.configuration.description;
+
+/**
+ * Element that represents a link in the {@link Description}.
+ */
+public class LinkElement implements InlineElement {
+	private final String link;
+	private final String text;
+
+	/**
+	 * Creates a link with a given url and description.
+	 *
+	 * @param link address that this link should point to
+	 * @param text a description for that link, that should be used in text
+	 * @return link representation
+	 */
+	public static LinkElement link(String link, String text) {
+		return new LinkElement(link, text);
+	}
+
+	/**
+	 * Creates a link with a given url. This url will be used as a description for that link.
+	 *
+	 * @param link address that this link should point to
+	 * @return link representation
+	 */
+	public static LinkElement link(String link) {
+		return new LinkElement(link, link);
+	}
+
+	public String getLink() {
+		return link;
+	}
+
+	public String getText() {
+		return text;
+	}
+
+	private LinkElement(String link, String text) {
+		this.link = link;
+		this.text = text;
+	}
+
+	@Override
+	public void format(Formatter formatter) {
+		formatter.format(this);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java
new file mode 100644
index 0000000..1dea3ab
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java
@@ -0,0 +1,59 @@
+/*
+ * 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.configuration.description;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Represents a list in the {@link Description}.
+ */
+public class ListElement implements BlockElement {
+
+	private final List<InlineElement> entries;
+
+	/**
+	 * Creates a list with blocks of text. For example:
+	 * <pre>{@code
+	 * .list(
+	 * 	text("this is first element of list"),
+	 * 	text("this is second element of list with a %s", link("https://link"))
+	 * )
+	 * }</pre>
+	 *
+	 * @param elements list of this list entries
+	 * @return list representation
+	 */
+	public static ListElement list(InlineElement... elements) {
+		return new ListElement(Arrays.asList(elements));
+	}
+
+	public List<InlineElement> getEntries() {
+		return entries;
+	}
+
+	private ListElement(List<InlineElement> entries) {
+		this.entries = entries;
+	}
+
+	@Override
+	public void format(Formatter formatter) {
+		formatter.format(this);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java
new file mode 100644
index 0000000..aeb1d7a
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java
@@ -0,0 +1,73 @@
+/*
+ * 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.configuration.description;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Represents a text block in the {@link Description}.
+ */
+public class TextElement implements BlockElement, InlineElement {
+	private final String format;
+	private final List<InlineElement> elements;
+
+	/**
+	 * Creates a block of text with placeholders ("%s") that will be replaced with proper string representation of
+	 * given {@link InlineElement}. For example:
+	 *
+	 * <p>{@code text("This is a text with a link %s", link("https://somepage", "to here"))}
+	 *
+	 * @param format   text with placeholders for elements
+	 * @param elements elements to be put in the text
+	 * @return block of text
+	 */
+	public static TextElement text(String format, InlineElement... elements) {
+		return new TextElement(format, Arrays.asList(elements));
+	}
+
+	/**
+	 * Creates a simple block of text.
+	 *
+	 * @param text a simple block of text
+	 * @return block of text
+	 */
+	public static TextElement text(String text) {
+		return new TextElement(text, Collections.emptyList());
+	}
+
+	public String getFormat() {
+		return format;
+	}
+
+	public List<InlineElement> getElements() {
+		return elements;
+	}
+
+	private TextElement(String format, List<InlineElement> elements) {
+		this.format = format;
+		this.elements = elements;
+	}
+
+	@Override
+	public void format(Formatter formatter) {
+		formatter.format(this);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java b/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java
new file mode 100644
index 0000000..fcdb155
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.configuration.description;
+
+import org.junit.Test;
+
+import static org.apache.flink.configuration.description.LinkElement.link;
+import static org.apache.flink.configuration.description.TextElement.text;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link Description} and formatting with {@link HtmlFormatter}.
+ */
+public class DescriptionHtmlTest {
+	@Test
+	public void testDescriptionWithLink() {
+		Description description = Description.builder()
+			.text("This is a text with a link %s", link("https://somepage", "to here"))
+			.build();
+
+		String formattedDescription = new HtmlFormatter().format(description);
+
+		assertEquals("This is a text with a link <a href=\"https://somepage\">" +
+			"to here</a>", formattedDescription);
+	}
+
+	@Test
+	public void testDescriptionWithPercents() {
+		Description description = Description.builder()
+			.text("This is a text that has some percentage value of 20%.")
+			.build();
+
+		String formattedDescription = new HtmlFormatter().format(description);
+
+		assertEquals("This is a text that has some percentage value of 20%.", formattedDescription);
+	}
+
+	@Test
+	public void testDescriptionWithMultipleLinks() {
+		Description description = Description.builder()
+			.text("This is a text with a link %s and another %s", link("https://somepage", "to here"),
+				link("https://link"))
+			.build();
+
+		String formattedDescription = new HtmlFormatter().format(description);
+
+		assertEquals("This is a text with a link <a href=\"https://somepage\">to here</a> and another " +
+			"<a href=\"https://link\">https://link</a>", formattedDescription);
+	}
+
+	@Test
+	public void testDescriptionWithList() {
+		Description description = Description.builder()
+			.text("This is some list: ")
+			.list(
+				link("http://first_link"),
+				text("this is second element of list with a %s", link("https://link")))
+			.build();
+
+		String formattedDescription = new HtmlFormatter().format(description);
+
+		assertEquals(
+			"This is some list: <ul><li><a href=\"http://first_link\">http://first_link" +
+			"</a></li><li>this is second element of list " +
+				"with a <a href=\"https://link\">https://link</a></li></ul>",
+			formattedDescription);
+	}
+
+	@Test
+	public void testDescriptionWithLineBreak() {
+		Description description = Description.builder()
+			.text("This is first line.")
+			.linebreak()
+			.text("This is second line.")
+			.build();
+
+		String formattedDescription = new HtmlFormatter().format(description);
+
+		assertEquals(
+			"This is first line.<br/>This is second line.",
+			formattedDescription);
+	}
+
+	@Test
+	public void testDescriptionWithListAndEscaping() {
+		Description description = Description.builder()
+			.text("This is some list: ")
+			.list(
+				text("this is first element with illegal character '>' and '<'")
+			)
+			.build();
+
+		String formattedDescription = new HtmlFormatter().format(description);
+
+		assertEquals(
+			"This is some list: <ul><li>this is first element with illegal character '&gt;' and '&lt;'</li></ul>",
+			formattedDescription);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
----------------------------------------------------------------------
diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
index d333719..d0f665a 100644
--- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
+++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
@@ -24,6 +24,8 @@ import org.apache.flink.annotation.docs.ConfigGroups;
 import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.description.Formatter;
+import org.apache.flink.configuration.description.HtmlFormatter;
 import org.apache.flink.util.function.ThrowingConsumer;
 
 import java.io.IOException;
@@ -76,6 +78,7 @@ public class ConfigOptionsDocGenerator {
 	private static final String CLASS_PREFIX_GROUP = "classPrefix";
 	private static final Pattern CLASS_NAME_PATTERN = Pattern.compile("(?<" + CLASS_NAME_GROUP + ">(?<" + CLASS_PREFIX_GROUP + ">[a-zA-Z]*)(?:Options|Config|Parameters))(?:\\.java)?");
 
+	private static final Formatter formatter = new HtmlFormatter();
 	/**
 	 * This method generates html tables from set of classes containing {@link ConfigOption ConfigOptions}.
 	 *
@@ -253,7 +256,7 @@ public class ConfigOptionsDocGenerator {
 			"        <tr>\n" +
 			"            <td><h5>" + escapeCharacters(option.key()) + "</h5></td>\n" +
 			"            <td style=\"word-wrap: break-word;\">" + escapeCharacters(addWordBreakOpportunities(defaultValue)) + "</td>\n" +
-			"            <td>" + escapeCharacters(option.description()) + "</td>\n" +
+			"            <td>" + formatter.format(option.description()) + "</td>\n" +
 			"        </tr>\n";
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java
----------------------------------------------------------------------
diff --git a/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java b/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java
index 3a53046..89d6f63 100644
--- a/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java
+++ b/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java
@@ -33,8 +33,8 @@ public enum Utils {
 	public static String escapeCharacters(String value) {
 		return value
 			.replaceAll("<wbr>", TEMPORARY_PLACEHOLDER)
-			.replaceAll("<", "&#60;")
-			.replaceAll(">", "&#62;")
+			.replaceAll("<", "&lt;")
+			.replaceAll(">", "&gt;")
 			.replaceAll(TEMPORARY_PLACEHOLDER, "<wbr>");
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java
index 26cba80..a0eade6 100644
--- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java
+++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java
@@ -24,6 +24,8 @@ import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.description.Formatter;
+import org.apache.flink.configuration.description.HtmlFormatter;
 import org.apache.flink.docs.configuration.data.TestCommonOptions;
 import org.apache.flink.util.FileUtils;
 
@@ -235,6 +237,7 @@ public class ConfigOptionsDocGeneratorTest {
 		};
 
 		ConfigOptionsDocGenerator.generateCommonSection(projectRootDir, outputDirectory, locations, "src/test/java");
+		Formatter formatter = new HtmlFormatter();
 
 		String expected =
 			"<table class=\"table table-bordered\">\n" +
@@ -249,12 +252,12 @@ public class ConfigOptionsDocGeneratorTest {
 			"        <tr>\n" +
 			"            <td><h5>" + TestCommonOptions.COMMON_POSITIONED_OPTION.key() + "</h5></td>\n" +
 			"            <td style=\"word-wrap: break-word;\">" + TestCommonOptions.COMMON_POSITIONED_OPTION.defaultValue() + "</td>\n" +
-			"            <td>" + TestCommonOptions.COMMON_POSITIONED_OPTION.description() + "</td>\n" +
+			"            <td>" + formatter.format(TestCommonOptions.COMMON_POSITIONED_OPTION.description()) + "</td>\n" +
 			"        </tr>\n" +
 			"        <tr>\n" +
 			"            <td><h5>" + TestCommonOptions.COMMON_OPTION.key() + "</h5></td>\n" +
 			"            <td style=\"word-wrap: break-word;\">" + TestCommonOptions.COMMON_OPTION.defaultValue() + "</td>\n" +
-			"            <td>" + TestCommonOptions.COMMON_OPTION.description() + "</td>\n" +
+			"            <td>" + formatter.format(TestCommonOptions.COMMON_OPTION.description()) + "</td>\n" +
 			"        </tr>\n" +
 			"    </tbody>\n" +
 			"</table>\n";

http://git-wip-us.apache.org/repos/asf/flink/blob/f576381e/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
----------------------------------------------------------------------
diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
index 2d20212..264d2ed 100644
--- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
+++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
@@ -20,6 +20,8 @@ package org.apache.flink.docs.configuration;
 
 import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.description.Formatter;
+import org.apache.flink.configuration.description.HtmlFormatter;
 
 import org.jsoup.Jsoup;
 import org.jsoup.nodes.Document;
@@ -55,6 +57,8 @@ import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.stri
  */
 public class ConfigOptionsDocsCompletenessITCase {
 
+	private static final Formatter htmlFormatter = new HtmlFormatter();
+
 	@Test
 	public void testCommonSectionCompleteness() throws IOException, ClassNotFoundException {
 		Map<String, DocumentedOption> documentedOptions = parseDocumentedCommonOptions();
@@ -173,7 +177,11 @@ public class ConfigOptionsDocsCompletenessITCase {
 			.map(tableRow -> {
 				String key = tableRow.child(0).text();
 				String defaultValue = tableRow.child(1).text();
-				String description = tableRow.child(2).text();
+				String description = tableRow.child(2)
+					.childNodes()
+					.stream()
+					.map(Object::toString)
+					.collect(Collectors.joining());
 				return new DocumentedOption(key, defaultValue, description, file.getName(file.getNameCount() - 1));
 			})
 			.collect(Collectors.toList());
@@ -189,7 +197,7 @@ public class ConfigOptionsDocsCompletenessITCase {
 					if (predicate.test(option)) {
 						String key = option.option.key();
 						String defaultValue = stringifyDefault(option);
-						String description = option.option.description();
+						String description = htmlFormatter.format(option.option.description());
 						ExistingOption duplicate = existingOptions.put(key, new ExistingOption(key, defaultValue, description, optionsClass));
 						if (duplicate != null) {
 							// multiple documented options have the same key