You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ew...@apache.org on 2016/08/07 01:14:14 UTC

kafka git commit: KAFKA-3809: Auto-generate documentation for topic-level configuration

Repository: kafka
Updated Branches:
  refs/heads/trunk 3dafb81da -> 778fcd0e9


KAFKA-3809: Auto-generate documentation for topic-level configuration

ijuma said that it would make sense to split out this work from KAFKA-3234, since KAFKA-3234 had both a mechanical change (generating docs) as well as a change requiring discussion (deprecating/renaming config options).

jjkoshy, I hope you don't mind that I took over this work. It's been 3 months since the last activity on KAFKA-3234, so I thought it would be okay to take over.

This work is essentially is the first 5-6 commits from Joel's https://github.com/apache/kafka/pull/907. However, since I'm not very experienced with git, I didn't do a direct merge/rebase, but instead largely hand-merged it. I did some minor cleanup. All credit goes to Joel, all blame goes to me. :)

For reference, I attached the auto-generated configuration.html file (as a PDF, because github won't let me attache html).
[configuration.pdf](https://github.com/apache/kafka/files/323901/configuration.pdf)

This is my first time writing Scala, so let me know if there are any changes needed.

I don't know who is the right person to review this. ijuma, can you help me redirect this to the appropriate person? Thanks.

Author: James Cheng <jy...@yahoo.com>

Reviewers: Ismael Juma <is...@juma.me.uk>, Joel Koshy <jj...@gmail.com>, Ewen Cheslack-Postava <ew...@confluent.io>

Closes #1527 from wushujames/generate_topic_docs


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

Branch: refs/heads/trunk
Commit: 778fcd0e9192c8de4c17f23189029d5c1bf98e2a
Parents: 3dafb81
Author: James Cheng <jy...@yahoo.com>
Authored: Sat Aug 6 18:14:53 2016 -0700
Committer: Ewen Cheslack-Postava <me...@ewencp.org>
Committed: Sat Aug 6 18:14:53 2016 -0700

----------------------------------------------------------------------
 build.gradle                                    |  10 +-
 .../apache/kafka/common/config/ConfigDef.java   |  77 ++++----
 core/src/main/scala/kafka/log/LogConfig.scala   | 188 ++++++++++++++-----
 .../main/scala/kafka/server/KafkaConfig.scala   |  16 +-
 docs/configuration.html                         | 100 +---------
 5 files changed, 212 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/778fcd0e/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 5889f71..6edcfdd 100644
--- a/build.gradle
+++ b/build.gradle
@@ -488,9 +488,17 @@ project(':core') {
     standardOutput = new File(generatedDocsDir, "kafka_config.html").newOutputStream()
   }
 
+  task genTopicConfigDocs(type: JavaExec) {
+    classpath = sourceSets.main.runtimeClasspath
+    main = 'kafka.log.LogConfig'
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "topic_config.html").newOutputStream()
+  }
+
   task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs',
                                'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs',
-                               ':connect:runtime:genConnectConfigDocs', ':streams:genStreamsConfigDocs'], type: Tar) {
+                               'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', 
+                               ':streams:genStreamsConfigDocs'], type: Tar) {
     classifier = 'site-docs'
     compression = Compression.GZIP
     from project.file("../docs")

http://git-wip-us.apache.org/repos/asf/kafka/blob/778fcd0e/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index fe9c209..073f474 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -872,46 +872,57 @@ public class ConfigDef {
         }
     }
 
+    protected List<String> headers() {
+        return Arrays.asList("Name", "Description", "Type", "Default", "Valid Values", "Importance");
+    }
+
+    protected String getConfigValue(ConfigKey key, String headerName) {
+        switch (headerName) {
+            case "Name":
+                return key.name;
+            case "Description":
+                return key.documentation;
+            case "Type":
+                return key.type.toString().toLowerCase(Locale.ROOT);
+            case "Default":
+                if (key.hasDefault()) {
+                    if (key.defaultValue == null)
+                        return "null";
+                    else if (key.type == Type.STRING && key.defaultValue.toString().isEmpty())
+                        return "\"\"";
+                    else
+                        return key.defaultValue.toString();
+                } else
+                    return "";
+            case "Valid Values":
+                return key.validator != null ? key.validator.toString() : "";
+            case "Importance":
+                return key.importance.toString().toLowerCase(Locale.ROOT);
+            default:
+                throw new RuntimeException("Can't find value for header '" + headerName + "' in " + key.name);
+        }
+    }
+    
     public String toHtmlTable() {
         List<ConfigKey> configs = sortedConfigs();
         StringBuilder b = new StringBuilder();
         b.append("<table class=\"data-table\"><tbody>\n");
         b.append("<tr>\n");
-        b.append("<th>Name</th>\n");
-        b.append("<th>Description</th>\n");
-        b.append("<th>Type</th>\n");
-        b.append("<th>Default</th>\n");
-        b.append("<th>Valid Values</th>\n");
-        b.append("<th>Importance</th>\n");
+        // print column headers
+        for (String headerName : headers()) {
+            b.append("<th>");
+            b.append(headerName);
+            b.append("</th>\n");
+        }
         b.append("</tr>\n");
         for (ConfigKey def : configs) {
             b.append("<tr>\n");
-            b.append("<td>");
-            b.append(def.name);
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.documentation);
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.type.toString().toLowerCase(Locale.ROOT));
-            b.append("</td>");
-            b.append("<td>");
-            if (def.hasDefault()) {
-                if (def.defaultValue == null)
-                    b.append("null");
-                else if (def.type == Type.STRING && def.defaultValue.toString().isEmpty())
-                    b.append("\"\"");
-                else
-                    b.append(def.defaultValue);
-            } else
-                b.append("");
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.validator != null ? def.validator.toString() : "");
-            b.append("</td>");
-            b.append("<td>");
-            b.append(def.importance.toString().toLowerCase(Locale.ROOT));
-            b.append("</td>");
+            // print column values
+            for (String headerName : headers()) {
+                b.append("<td>");
+                b.append(getConfigValue(def, headerName));
+                b.append("</td>");
+            }
             b.append("</tr>\n");
         }
         b.append("</tbody></table>");
@@ -963,7 +974,7 @@ public class ConfigDef {
      * Get a list of configs sorted into "natural" order: listing required fields first, then
      * ordering by importance, and finally by name.
      */
-    private List<ConfigKey> sortedConfigs() {
+    protected List<ConfigKey> sortedConfigs() {
         // sort first required fields, then by importance, then name
         List<ConfigKey> configs = new ArrayList<>(this.configKeys.values());
         Collections.sort(configs, new Comparator<ConfigKey>() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/778fcd0e/core/src/main/scala/kafka/log/LogConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
index 31e62b4..1b1ecb7 100755
--- a/core/src/main/scala/kafka/log/LogConfig.scala
+++ b/core/src/main/scala/kafka/log/LogConfig.scala
@@ -30,6 +30,9 @@ import org.apache.kafka.common.record.TimestampType
 import org.apache.kafka.common.utils.Utils
 import java.util.Locale
 
+import scala.collection.mutable
+import org.apache.kafka.common.config.ConfigDef.ConfigKey
+
 object Defaults {
   val SegmentSize = kafka.server.Defaults.LogSegmentBytes
   val SegmentMs = kafka.server.Defaults.LogRollHours * 60 * 60 * 1000L
@@ -57,7 +60,7 @@ object Defaults {
 case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props, false) {
   /**
    * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig
-   * should also go in copyKafkaConfigToLog.
+   * should also go in [[kafka.server.KafkaServer.copyKafkaConfigToLog]].
    */
   val segmentSize = getInt(LogConfig.SegmentBytesProp)
   val segmentMs = getLong(LogConfig.SegmentMsProp)
@@ -88,7 +91,7 @@ case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfi
 object LogConfig {
 
   def main(args: Array[String]) {
-    System.out.println(configDef.toHtmlTable)
+    println(configDef.toHtmlTable)
   }
 
   val Delete = "delete"
@@ -116,67 +119,160 @@ object LogConfig {
   val MessageTimestampTypeProp = "message.timestamp.type"
   val MessageTimestampDifferenceMaxMsProp = "message.timestamp.difference.max.ms"
 
-  val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log"
-  val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled"
-  val SegmentJitterMsDoc = "The maximum random jitter subtracted from segmentMs to avoid thundering herds of segment" +
-    " rolling"
-  val FlushIntervalDoc = "The number of messages that can be written to the log before a flush is forced"
-  val FlushMsDoc = "The amount of time the log can have dirty data before a flush is forced"
-  val RetentionSizeDoc = "The approximate total number of bytes this log can use"
-  val RetentionMsDoc = "The approximate maximum age of the last segment that is retained"
-  val MaxIndexSizeDoc = "The maximum size of an index file"
-  val MaxMessageSizeDoc = "The maximum size of a message"
-  val IndexIntervalDoc = "The approximate number of bytes between index entries"
+  val SegmentSizeDoc = "This configuration controls the segment file size for " +
+    "the log. Retention and cleaning is always done a file at a time so a larger " +
+    "segment size means fewer files but less granular control over retention."
+  val SegmentMsDoc = "This configuration controls the period of time after " +
+    "which Kafka will force the log to roll even if the segment file isn't full " +
+    "to ensure that retention can delete or compact old data."
+  val SegmentJitterMsDoc = "The maximum random jitter subtracted from the scheduled segment roll time to avoid" +
+    " thundering herds of segment rolling"
+  val FlushIntervalDoc = "This setting allows specifying an interval at which we " +
+    "will force an fsync of data written to the log. For example if this was set to 1 " +
+    "we would fsync after every message; if it were 5 we would fsync after every five " +
+    "messages. In general we recommend you not set this and use replication for " +
+    "durability and allow the operating system's background flush capabilities as it " +
+    "is more efficient. This setting can be overridden on a per-topic basis (see <a " +
+    "href=\"#topic-config\">the per-topic configuration section</a>)."
+  val FlushMsDoc = "This setting allows specifying a time interval at which we will " +
+    "force an fsync of data written to the log. For example if this was set to 1000 " +
+    "we would fsync after 1000 ms had passed. In general we recommend you not set " +
+    "this and use replication for durability and allow the operating system's background " +
+    "flush capabilities as it is more efficient."
+  val RetentionSizeDoc = "This configuration controls the maximum size a log can grow " +
+    "to before we will discard old log segments to free up space if we are using the " +
+    "\"delete\" retention policy. By default there is no size limit only a time limit."
+  val RetentionMsDoc = "This configuration controls the maximum time we will retain a " +
+    "log before we will discard old log segments to free up space if we are using the " +
+    "\"delete\" retention policy. This represents an SLA on how soon consumers must read " +
+    "their data."
+  val MaxIndexSizeDoc = "This configuration controls the size of the index that maps " +
+    "offsets to file positions. We preallocate this index file and shrink it only after log " +
+    "rolls. You generally should not need to change this setting."
+  val MaxMessageSizeDoc = "This is largest message size Kafka will allow to be appended. Note that if you increase" +
+    " this size you must also increase your consumer's fetch size so they can fetch messages this large."
+  val IndexIntervalDoc = "This setting controls how frequently Kafka adds an index " +
+    "entry to it's offset index. The default setting ensures that we index a message " +
+    "roughly every 4096 bytes. More indexing allows reads to jump closer to the exact " +
+    "position in the log but makes the index larger. You probably don't need to change " +
+    "this."
   val FileDeleteDelayMsDoc = "The time to wait before deleting a file from the filesystem"
-  val DeleteRetentionMsDoc = "The time to retain delete markers in the log. Only applicable for logs that are being" +
-    " compacted."
-  val MinCleanableRatioDoc = "The ratio of bytes that are available for cleaning to the bytes already cleaned"
-  val CompactDoc = "Should old segments in this log be deleted or deduplicated?"
-  val UncleanLeaderElectionEnableDoc = "Indicates whether unclean leader election is enabled"
-  val MinInSyncReplicasDoc = "If number of insync replicas drops below this number, we stop accepting writes with" +
-    " -1 (or all) required acks"
+  val DeleteRetentionMsDoc = "The amount of time to retain delete tombstone markers " +
+    "for <a href=\"#compaction\">log compacted</a> topics. This setting also gives a bound " +
+    "on the time in which a consumer must complete a read if they begin from offset 0 " +
+    "to ensure that they get a valid snapshot of the final stage (otherwise delete " +
+    "tombstones may be collected before they complete their scan)."
+  val MinCleanableRatioDoc = "This configuration controls how frequently the log " +
+    "compactor will attempt to clean the log (assuming <a href=\"#compaction\">log " +
+    "compaction</a> is enabled). By default we will avoid cleaning a log where more than " +
+    "50% of the log has been compacted. This ratio bounds the maximum space wasted in " +
+    "the log by duplicates (at 50% at most 50% of the log could be duplicates). A " +
+    "higher ratio will mean fewer, more efficient cleanings but will mean more wasted " +
+    "space in the log."
+  val CompactDoc = "A string that is either \"delete\" or \"compact\". This string " +
+    "designates the retention policy to use on old log segments. The default policy " +
+    "(\"delete\") will discard old segments when their retention time or size limit has " +
+    "been reached. The \"compact\" setting will enable <a href=\"#compaction\">log " +
+    "compaction</a> on the topic."
+  val UncleanLeaderElectionEnableDoc = "Indicates whether to enable replicas not in the ISR set to be elected as" +
+    " leader as a last resort, even though doing so may result in data loss"
+  val MinInSyncReplicasDoc = KafkaConfig.MinInSyncReplicasDoc
   val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the " +
     "standard compression codecs ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to " +
     "no compression; and 'producer' which means retain the original compression codec set by the producer."
   val PreAllocateEnableDoc ="Should pre allocate file when create new segment?"
   val MessageFormatVersionDoc = KafkaConfig.LogMessageFormatVersionDoc
   val MessageTimestampTypeDoc = KafkaConfig.LogMessageTimestampTypeDoc
-  val MessageTimestampDifferenceMaxMsDoc = KafkaConfig.LogMessageTimestampDifferenceMaxMsDoc
+  val MessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " +
+    "a message and the timestamp specified in the message. If message.timestamp.type=CreateTime, a message will be rejected " +
+    "if the difference in timestamp exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."  
+
+  private class LogConfigDef extends ConfigDef {
+
+    private final val serverDefaultConfigNames = mutable.Map[String, String]()
+
+    def define(name: String, defType: ConfigDef.Type, defaultValue: Any, validator: ConfigDef.Validator,
+               importance: ConfigDef.Importance, doc: String, serverDefaultConfigName: String): LogConfigDef = {
+      super.define(name, defType, defaultValue, validator, importance, doc)
+      serverDefaultConfigNames.put(name, serverDefaultConfigName)
+      this
+    }
+
+    def define(name: String, defType: ConfigDef.Type, defaultValue: Any, importance: ConfigDef.Importance,
+               documentation: String, serverDefaultConfigName: String): LogConfigDef = {
+      super.define(name, defType, defaultValue, importance, documentation)
+      serverDefaultConfigNames.put(name, serverDefaultConfigName)
+      this
+    }
+
+    def define(name: String, defType: ConfigDef.Type, importance: ConfigDef.Importance, documentation: String,
+               serverDefaultConfigName: String): LogConfigDef = {
+      super.define(name, defType, importance, documentation)
+      serverDefaultConfigNames.put(name, serverDefaultConfigName)
+      this
+    }
+
+    override def headers = List("Name", "Description", "Type", "Default", "Valid Values", "Server Default Property", "Importance").asJava
+
+    override def getConfigValue(key: ConfigKey, headerName: String): String = {
+      headerName match {
+        case "Server Default Property" => serverDefaultConfigNames.get(key.name).get
+        case _ => super.getConfigValue(key, headerName)
+      }
+    }
+  }
 
-  private val configDef = {
+  private val configDef: LogConfigDef = {
     import org.apache.kafka.common.config.ConfigDef.Importance._
     import org.apache.kafka.common.config.ConfigDef.Range._
     import org.apache.kafka.common.config.ConfigDef.Type._
     import org.apache.kafka.common.config.ConfigDef.ValidString._
 
-    new ConfigDef()
-      .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinMessageOverhead), MEDIUM, SegmentSizeDoc)
-      .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(0), MEDIUM, SegmentMsDoc)
-      .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc)
-      .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc)
-      .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(0), MEDIUM, FlushIntervalDoc)
-      .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc)
+    new LogConfigDef()
+      .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinMessageOverhead), MEDIUM,
+        SegmentSizeDoc, KafkaConfig.LogSegmentBytesProp)
+      .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(0), MEDIUM, SegmentMsDoc,
+        KafkaConfig.LogRollTimeMillisProp)
+      .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc,
+        KafkaConfig.LogRollTimeJitterMillisProp)
+      .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc,
+        KafkaConfig.LogIndexSizeMaxBytesProp)
+      .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(0), MEDIUM, FlushIntervalDoc,
+        KafkaConfig.LogFlushIntervalMessagesProp)
+      .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc,
+        KafkaConfig.LogFlushIntervalMsProp)
       // can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize
-      .define(RetentionBytesProp, LONG, Defaults.RetentionSize, MEDIUM, RetentionSizeDoc)
+      .define(RetentionBytesProp, LONG, Defaults.RetentionSize, MEDIUM, RetentionSizeDoc,
+        KafkaConfig.LogRetentionBytesProp)
       // can be negative. See kafka.log.LogManager.cleanupExpiredSegments
-      .define(RetentionMsProp, LONG, Defaults.RetentionMs, MEDIUM, RetentionMsDoc)
-      .define(MaxMessageBytesProp, INT, Defaults.MaxMessageSize, atLeast(0), MEDIUM, MaxMessageSizeDoc)
-      .define(IndexIntervalBytesProp, INT, Defaults.IndexInterval, atLeast(0), MEDIUM,  IndexIntervalDoc)
-      .define(DeleteRetentionMsProp, LONG, Defaults.DeleteRetentionMs, atLeast(0), MEDIUM, DeleteRetentionMsDoc)
-      .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc)
+      .define(RetentionMsProp, LONG, Defaults.RetentionMs, MEDIUM, RetentionMsDoc,
+        KafkaConfig.LogRetentionTimeMillisProp)
+      .define(MaxMessageBytesProp, INT, Defaults.MaxMessageSize, atLeast(0), MEDIUM, MaxMessageSizeDoc,
+        KafkaConfig.MessageMaxBytesProp)
+      .define(IndexIntervalBytesProp, INT, Defaults.IndexInterval, atLeast(0), MEDIUM, IndexIntervalDoc,
+        KafkaConfig.LogIndexIntervalBytesProp)
+      .define(DeleteRetentionMsProp, LONG, Defaults.DeleteRetentionMs, atLeast(0), MEDIUM,
+        DeleteRetentionMsDoc, KafkaConfig.LogCleanerDeleteRetentionMsProp)
+      .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc,
+        KafkaConfig.LogDeleteDelayMsProp)
       .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM,
-        MinCleanableRatioDoc)
-      .define(CleanupPolicyProp, STRING, Defaults.Compact, in(Compact, Delete), MEDIUM,
-        CompactDoc)
+        MinCleanableRatioDoc, KafkaConfig.LogCleanerMinCleanRatioProp)
+      .define(CleanupPolicyProp, STRING, Defaults.Compact, in(Compact, Delete), MEDIUM, CompactDoc,
+        KafkaConfig.LogCleanupPolicyProp)
       .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable,
-        MEDIUM, UncleanLeaderElectionEnableDoc)
-      .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc)
-      .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc)
-      .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable,
-        MEDIUM, PreAllocateEnableDoc)
-      .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, MEDIUM, MessageFormatVersionDoc)
-      .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, MEDIUM, MessageTimestampTypeDoc)
-      .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc)
+        MEDIUM, UncleanLeaderElectionEnableDoc, KafkaConfig.UncleanLeaderElectionEnableProp)
+      .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc,
+        KafkaConfig.MinInSyncReplicasProp)
+      .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*),
+        MEDIUM, CompressionTypeDoc, KafkaConfig.CompressionTypeProp)
+      .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, MEDIUM, PreAllocateEnableDoc,
+        KafkaConfig.LogPreAllocateProp)
+      .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, MEDIUM, MessageFormatVersionDoc,
+        KafkaConfig.LogMessageFormatVersionProp)
+      .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, MEDIUM, MessageTimestampTypeDoc,
+        KafkaConfig.LogMessageTimestampTypeProp)
+      .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs,
+        atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc, KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)
   }
 
   def apply(): LogConfig = LogConfig(new Properties())

http://git-wip-us.apache.org/repos/asf/kafka/blob/778fcd0e/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index f9a12a9..6655681 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -444,14 +444,24 @@ object KafkaConfig {
     "Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check ApiVersion for more details. By setting a particular message format version, the " +
     "user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " +
     "will cause consumers with older versions to break as they will receive messages with a format that they don't understand."
+
   val LogMessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either " +
     "`CreateTime` or `LogAppendTime`"
+
   val LogMessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " +
-    "a message and the timestamp specified in the message. If message.timestamp.type=CreateTime, a message will be rejected " +
-    "if the difference in timestamp exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."
+    "a message and the timestamp specified in the message. If log.message.timestamp.type=CreateTime, a message will be rejected " +
+    "if the difference in timestamp exceeds this threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime."
   val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown"
   val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server"
-  val MinInSyncReplicasDoc = "define the minimum number of replicas in ISR needed to satisfy a produce request with acks=all (or -1)"
+  val MinInSyncReplicasDoc = "When a producer sets acks to \"all\" (or \"-1\"), " +
+    "min.insync.replicas specifies the minimum number of replicas that must acknowledge " +
+    "a write for the write to be considered successful. If this minimum cannot be met, " +
+    "then the producer will raise an exception (either NotEnoughReplicas or " +
+    "NotEnoughReplicasAfterAppend).<br>When used together, min.insync.replicas and acks " +
+    "allow you to enforce greater durability guarantees. A typical scenario would be to " +
+    "create a topic with a replication factor of 3, set min.insync.replicas to 2, and " +
+    "produce with acks of \"all\". This will ensure that the producer raises an exception " +
+    "if a majority of replicas do not receive a write."
   /** ********* Replication configuration ***********/
   val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels"
   val ControllerMessageQueueSizeDoc = "The buffer size for controller-to-broker-channels"

http://git-wip-us.apache.org/repos/asf/kafka/blob/778fcd0e/docs/configuration.html
----------------------------------------------------------------------
diff --git a/docs/configuration.html b/docs/configuration.html
index f9bd1e4..9c39ae8 100644
--- a/docs/configuration.html
+++ b/docs/configuration.html
@@ -34,7 +34,7 @@ Topic-level configurations and defaults are discussed in more detail <a href="#t
 
 <a id="topic-config" href="#topic-config">Topic-level configuration</a>
 
-Configurations pertinent to topics have both a global default as well an optional per-topic override. If no per-topic configuration is given the global default is used. The override can be set at topic creation time by giving one or more <code>--config</code> options. This example creates a topic named <i>my-topic</i> with a custom max message size and flush rate:
+Configurations pertinent to topics have both a server default as well an optional per-topic override. If no per-topic configuration is given the server default is used. The override can be set at topic creation time by giving one or more <code>--config</code> options. This example creates a topic named <i>my-topic</i> with a custom max message size and flush rate:
 <pre>
 <b> &gt; bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic my-topic --partitions 1
         --replication-factor 1 --config max.message.bytes=64000 --config flush.messages=1</b>
@@ -51,101 +51,9 @@ To remove an override you can do
     --delete-config max.message.bytes</b>
 </pre>
 
-The following are the topic-level configurations. The server's default configuration for this property is given under the Server Default Property heading, setting this default in the server config allows you to change the default given to topics that have no override specified.
-<table class="data-table">
-<tbody>
-    <tr>
-        <th>Property</th>
-        <th>Default</th>
-        <th>Server Default Property</th>
-        <th>Description</th>
-    </tr>
-    <tr>
-      <td>cleanup.policy</td>
-      <td>delete</td>
-      <td>log.cleanup.policy</td>
-      <td>A string that is either "delete" or "compact". This string designates the retention policy to use on old log segments. The default policy ("delete") will discard old segments when their retention time or size limit has been reached. The "compact" setting will enable <a href="#compaction">log compaction</a> on the topic.</td>
-    </tr>
-    <tr>
-      <td>delete.retention.ms</td>
-      <td>86400000 (24 hours)</td>
-      <td>log.cleaner.delete.retention.ms</td>
-      <td>The amount of time to retain delete tombstone markers for <a href="#compaction">log compacted</a> topics. This setting also gives a bound on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise delete tombstones may be collected before they complete their scan).</td>
-    </tr>
-    <tr>
-      <td>flush.messages</td>
-      <td>None</td>
-      <td>log.flush.interval.messages</td>
-      <td>This setting allows specifying an interval at which we will force an fsync of data written to the log. For example if this was set to 1 we would fsync after every message; if it were 5 we would fsync after every five messages. In general we recommend you not set this and use replication for durability and allow the operating system's background flush capabilities as it is more efficient. This setting can be overridden on a per-topic basis (see <a href="#topic-config">the per-topic configuration section</a>).</td>
-    </tr>
-    <tr>
-      <td>flush.ms</td>
-      <td>None</td>
-      <td>log.flush.interval.ms</td>
-      <td>This setting allows specifying a time interval at which we will force an fsync of data written to the log. For example if this was set to 1000 we would fsync after 1000 ms had passed. In general we recommend you not set this and use replication for durability and allow the operating system's background flush capabilities as it is more efficient.</td>
-    </tr>
-    <tr>
-      <td>index.interval.bytes</td>
-      <td>4096</td>
-      <td>log.index.interval.bytes</td>
-      <td>This setting controls how frequently Kafka adds an index entry to it's offset index. The default setting ensures that we index a message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact position in the log but makes the index larger. You probably don't need to change this.</td>
-    </tr>
-    <tr>
-      <td>max.message.bytes</td>
-      <td>1,000,000</td>
-      <td>message.max.bytes</td>
-      <td>This is largest message size Kafka will allow to be appended to this topic. Note that if you increase this size you must also increase your consumer's fetch size so they can fetch messages this large.</td>
-    </tr>
-    <tr>
-      <td>min.cleanable.dirty.ratio</td>
-      <td>0.5</td>
-      <td>log.cleaner.min.cleanable.ratio</td>
-      <td>This configuration controls how frequently the log compactor will attempt to clean the log (assuming <a href="#compaction">log compaction</a> is enabled). By default we will avoid cleaning a log where more than 50% of the log has been compacted. This ratio bounds the maximum space wasted in the log by duplicates (at 50% at most 50% of the log could be duplicates). A higher ratio will mean fewer, more efficient cleanings but will mean more wasted space in the log.</td>
-    </tr>
-    <tr>
-      <td>min.insync.replicas</td>
-      <td>1</td>
-      <td>min.insync.replicas</td>
-      <td>When a producer sets acks to "all", min.insync.replicas specifies the minimum number of replicas that must acknowledge a write for the write to be considered successful. If this minimum cannot be met, then the producer will raise an exception (either NotEnoughReplicas or NotEnoughReplicasAfterAppend).
-      When used together, min.insync.replicas and acks allow you to enforce greater durability guarantees. A typical scenario would be to create a topic with a replication factor of 3, set min.insync.replicas to 2, and produce with acks of "all". This will ensure that the producer raises an exception if a majority of replicas do not receive a write.</td>
-    </tr>
-    <tr>
-      <td>retention.bytes</td>
-      <td>None</td>
-      <td>log.retention.bytes</td>
-      <td>This configuration controls the maximum size a log can grow to before we will discard old log segments to free up space if we are using the "delete" retention policy. By default there is no size limit only a time limit.</td>
-    </tr>
-    <tr>
-      <td>retention.ms</td>
-      <td>7 days</td>
-      <td>log.retention.minutes</td>
-      <td>This configuration controls the maximum time we will retain a log before we will discard old log segments to free up space if we are using the "delete" retention policy. This represents an SLA on how soon consumers must read their data.</td>
-    </tr>
-    <tr>
-      <td>segment.bytes</td>
-      <td>1 GB</td>
-      <td>log.segment.bytes</td>
-      <td>This configuration controls the segment file size for the log. Retention and cleaning is always done a file at a time so a larger segment size means fewer files but less granular control over retention.</td>
-    </tr>
-    <tr>
-      <td>segment.index.bytes</td>
-      <td>10 MB</td>
-      <td>log.index.size.max.bytes</td>
-      <td>This configuration controls the size of the index that maps offsets to file positions. We preallocate this index file and shrink it only after log rolls. You generally should not need to change this setting.</td>
-    </tr>
-    <tr>
-      <td>segment.ms</td>
-      <td>7 days</td>
-      <td>log.roll.hours</td>
-      <td>This configuration controls the period of time after which Kafka will force the log to roll even if the segment file isn't full to ensure that retention can delete or compact old data.</td>
-    </tr>
-    <tr>
-      <td>segment.jitter.ms</td>
-      <td>0</td>
-      <td>log.roll.jitter.{ms,hours}</td>
-      <td>The maximum jitter to subtract from logRollTimeMillis.</td>
-    </tr>
-</table>
+The following are the topic-level configurations. The server's default configuration for this property is given under the Server Default Property heading. A given server default config value only applies to a topic if it does not have an explicit topic config override.
+
+<!--#include virtual="generated/topic_config.html" -->
 
 <h3><a id="producerconfigs" href="#producerconfigs">3.2 Producer Configs</a></h3>