You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/04/18 16:19:00 UTC

[jira] [Commented] (KAFKA-3365) Add a documentation field for types and update doc generation

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

ASF GitHub Bot commented on KAFKA-3365:
---------------------------------------

hachikuji closed pull request #4735: KAFKA-3365 Add a documentation field for types and update doc generation
URL: https://github.com/apache/kafka/pull/4735
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/build.gradle b/build.gradle
index 5e4c35643c2..4f2eb15051e 100644
--- a/build.gradle
+++ b/build.gradle
@@ -634,6 +634,13 @@ project(':core') {
     standardOutput = new File(generatedDocsDir, "protocol_errors.html").newOutputStream()
   }
 
+  task genProtocolTypesDocs(type: JavaExec) {
+    classpath = sourceSets.main.runtimeClasspath
+    main = 'org.apache.kafka.common.protocol.types.Type'
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "protocol_types.html").newOutputStream()
+  }
+
   task genProtocolApiKeyDocs(type: JavaExec) {
     classpath = sourceSets.main.runtimeClasspath
     main = 'org.apache.kafka.common.protocol.ApiKeys'
@@ -697,7 +704,7 @@ project(':core') {
     standardOutput = new File(generatedDocsDir, "producer_metrics.html").newOutputStream()
   }
 
-  task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs',
+  task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolTypesDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs',
                                'genAdminClientConfigDocs', 'genProducerConfigDocs', 'genConsumerConfigDocs',
                                'genKafkaConfigDocs', 'genTopicConfigDocs',
                                ':connect:runtime:genConnectConfigDocs', ':connect:runtime:genConnectTransformationDocs',
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
index 4213ecd08f0..6609dfd5196 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
@@ -16,12 +16,16 @@
  */
 package org.apache.kafka.common.protocol.types;
 
+import org.apache.kafka.common.protocol.types.Type.DocumentedType;
+
 import java.nio.ByteBuffer;
 
 /**
  * Represents a type for an array of a particular type
  */
-public class ArrayOf extends Type {
+public class ArrayOf extends DocumentedType {
+
+    private static final String ARRAY_TYPE_NAME = "ARRAY";
 
     private final Type type;
     private final boolean nullable;
@@ -93,7 +97,7 @@ public Type type() {
 
     @Override
     public String toString() {
-        return "ARRAY(" + type + ")";
+        return ARRAY_TYPE_NAME + "(" + type + ")";
     }
 
     @Override
@@ -110,4 +114,18 @@ public String toString() {
             throw new SchemaException("Not an Object[].");
         }
     }
+
+    @Override
+    public String typeName() {
+        return ARRAY_TYPE_NAME;
+    }
+
+    @Override
+    public String documentation() {
+        return "Represents a sequence of objects of a given type T. " +
+                "Type T can be either a primitive type (e.g. " + STRING + ") or a structure. " +
+                "First, the length N is given as an " + INT32 + ". Then N instances of type T follow. " +
+                "A null array is represented with a length of -1. " +
+                "In protocol documentation an array of T instances is referred to as [T].";
+    }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
index faa1540b499..cbcd4491d37 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
@@ -193,6 +193,4 @@ public void visit(Schema schema) {}
         public void visit(ArrayOf array) {}
         public void visit(Type field) {}
     }
-
-
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
index 57d31f459fb..85916d57d2d 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
@@ -63,6 +63,29 @@ public boolean isNullable() {
         return false;
     }
 
+    /**
+     * A Type that can return its description for documentation purposes.
+     */
+    public static abstract class DocumentedType extends Type {
+
+        /**
+         * Short name of the type to identify it in documentation;
+         * @return the name of the type
+         */
+        public abstract String typeName();
+
+        /**
+         * Documentation of the Type.
+         *
+         * @return details about valid values, representation
+         */
+        public abstract String documentation();
+
+        @Override
+        public String toString() {
+            return typeName();
+        }
+    }
     /**
      * The Boolean type represents a boolean value in a byte by using
      * the value of 0 to represent false, and 1 to represent true.
@@ -70,7 +93,7 @@ public boolean isNullable() {
      * If for some reason a value that is not 0 or 1 is read,
      * then any non-zero value will return true.
      */
-    public static final Type BOOLEAN = new Type() {
+    public static final DocumentedType BOOLEAN = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             if ((Boolean) o)
@@ -91,7 +114,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "BOOLEAN";
         }
 
@@ -102,9 +125,16 @@ public Boolean validate(Object item) {
             else
                 throw new SchemaException(item + " is not a Boolean.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents a boolean value in a byte. " +
+                    "Values 0 and 1 are used to represent false and true respectively. " +
+                    "When reading a boolean value, any non-zero value is considered true.";
+        }
     };
 
-    public static final Type INT8 = new Type() {
+    public static final DocumentedType INT8 = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             buffer.put((Byte) o);
@@ -121,7 +151,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "INT8";
         }
 
@@ -132,9 +162,14 @@ public Byte validate(Object item) {
             else
                 throw new SchemaException(item + " is not a Byte.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between -2<sup>7</sup> and 2<sup>7</sup>-1 inclusive.";
+        }
     };
 
-    public static final Type INT16 = new Type() {
+    public static final DocumentedType INT16 = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             buffer.putShort((Short) o);
@@ -151,7 +186,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "INT16";
         }
 
@@ -162,9 +197,15 @@ public Short validate(Object item) {
             else
                 throw new SchemaException(item + " is not a Short.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between -2<sup>15</sup> and 2<sup>15</sup>-1 inclusive. " +
+                    "The values are encoded using two bytes in network byte order (big-endian).";
+        }
     };
 
-    public static final Type INT32 = new Type() {
+    public static final DocumentedType INT32 = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             buffer.putInt((Integer) o);
@@ -181,7 +222,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "INT32";
         }
 
@@ -192,9 +233,15 @@ public Integer validate(Object item) {
             else
                 throw new SchemaException(item + " is not an Integer.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between -2<sup>31</sup> and 2<sup>31</sup>-1 inclusive. " +
+                    "The values are encoded using four bytes in network byte order (big-endian).";
+        }
     };
 
-    public static final Type UNSIGNED_INT32 = new Type() {
+    public static final DocumentedType UNSIGNED_INT32 = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             ByteUtils.writeUnsignedInt(buffer, (long) o);
@@ -211,7 +258,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "UINT32";
         }
 
@@ -222,9 +269,15 @@ public Long validate(Object item) {
             else
                 throw new SchemaException(item + " is not a Long.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between 0 and 2<sup>32</sup>-1 inclusive. " +
+                    "The values are encoded using four bytes in network byte order (big-endian).";
+        }
     };
 
-    public static final Type INT64 = new Type() {
+    public static final DocumentedType INT64 = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             buffer.putLong((Long) o);
@@ -241,7 +294,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "INT64";
         }
 
@@ -252,9 +305,15 @@ public Long validate(Object item) {
             else
                 throw new SchemaException(item + " is not a Long.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between -2<sup>63</sup> and 2<sup>63</sup>-1 inclusive. " +
+                    "The values are encoded using eight bytes in network byte order (big-endian).";
+        }
     };
 
-    public static final Type STRING = new Type() {
+    public static final DocumentedType STRING = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             byte[] bytes = Utils.utf8((String) o);
@@ -282,7 +341,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "STRING";
         }
 
@@ -293,9 +352,16 @@ public String validate(Object item) {
             else
                 throw new SchemaException(item + " is not a String.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents a sequence of characters. First the length N is given as an " + INT16 +
+                    ". Then N bytes follow which are the UTF-8 encoding of the character sequence. " +
+                    "Length must not be negative.";
+        }
     };
 
-    public static final Type NULLABLE_STRING = new Type() {
+    public static final DocumentedType NULLABLE_STRING = new DocumentedType() {
         @Override
         public boolean isNullable() {
             return true;
@@ -336,7 +402,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "NULLABLE_STRING";
         }
 
@@ -350,9 +416,16 @@ public String validate(Object item) {
             else
                 throw new SchemaException(item + " is not a String.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents a sequence of characters or null. For non-null strings, first the length N is given as an " + INT16 +
+                    ". Then N bytes follow which are the UTF-8 encoding of the character sequence. " +
+                    "A null value is encoded with length of -1 and there are no following bytes.";
+        }
     };
 
-    public static final Type BYTES = new Type() {
+    public static final DocumentedType BYTES = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             ByteBuffer arg = (ByteBuffer) o;
@@ -383,7 +456,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "BYTES";
         }
 
@@ -394,9 +467,15 @@ public ByteBuffer validate(Object item) {
             else
                 throw new SchemaException(item + " is not a java.nio.ByteBuffer.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents a raw sequence of bytes. First the length N is given as an " + INT32 +
+                    ". Then N bytes follow.";
+        }
     };
 
-    public static final Type NULLABLE_BYTES = new Type() {
+    public static final DocumentedType NULLABLE_BYTES = new DocumentedType() {
         @Override
         public boolean isNullable() {
             return true;
@@ -440,7 +519,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "NULLABLE_BYTES";
         }
 
@@ -454,9 +533,15 @@ public ByteBuffer validate(Object item) {
 
             throw new SchemaException(item + " is not a java.nio.ByteBuffer.");
         }
+
+        @Override
+        public String documentation() {
+            return "Represents a raw sequence of bytes or null. For non-null values, first the length N is given as an " + INT32 +
+                    ". Then N bytes follow. A null value is encoded with length of -1 and there are no following bytes.";
+        }
     };
 
-    public static final Type RECORDS = new Type() {
+    public static final DocumentedType RECORDS = new DocumentedType() {
         @Override
         public boolean isNullable() {
             return true;
@@ -486,7 +571,7 @@ public int sizeOf(Object o) {
         }
 
         @Override
-        public String toString() {
+        public String typeName() {
             return "RECORDS";
         }
 
@@ -500,9 +585,16 @@ public Records validate(Object item) {
 
             throw new SchemaException(item + " is not an instance of " + Records.class.getName());
         }
+
+        @Override
+        public String documentation() {
+            return "Represents a sequence of Kafka records as " + NULLABLE_BYTES + ". " +
+                    "For a detailed description of records see " +
+                    "<a href=\"/documentation/#messageformat\">Message Sets</a>.";
+        }
     };
 
-    public static final Type VARINT = new Type() {
+    public static final DocumentedType VARINT = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             ByteUtils.writeVarint((Integer) o, buffer);
@@ -520,7 +612,7 @@ public Integer validate(Object item) {
             throw new SchemaException(item + " is not an integer");
         }
 
-        public String toString() {
+        public String typeName() {
             return "VARINT";
         }
 
@@ -528,9 +620,16 @@ public String toString() {
         public int sizeOf(Object o) {
             return ByteUtils.sizeOfVarint((Integer) o);
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between -2<sup>31</sup> and 2<sup>31</sup>-1 inclusive. " +
+                    "Encoding follows the variable-length zig-zag encoding from " +
+                    " <a href=\"http://code.google.com/apis/protocolbuffers/docs/encoding.html\"> Google Protocol Buffers</a>.";
+        }
     };
 
-    public static final Type VARLONG = new Type() {
+    public static final DocumentedType VARLONG = new DocumentedType() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
             ByteUtils.writeVarlong((Long) o, buffer);
@@ -548,7 +647,7 @@ public Long validate(Object item) {
             throw new SchemaException(item + " is not a long");
         }
 
-        public String toString() {
+        public String typeName() {
             return "VARLONG";
         }
 
@@ -556,6 +655,43 @@ public String toString() {
         public int sizeOf(Object o) {
             return ByteUtils.sizeOfVarlong((Long) o);
         }
+
+        @Override
+        public String documentation() {
+            return "Represents an integer between -2<sup>63</sup> and 2<sup>63</sup>-1 inclusive. " +
+                    "Encoding follows the variable-length zig-zag encoding from " +
+                    " <a href=\"http://code.google.com/apis/protocolbuffers/docs/encoding.html\"> Google Protocol Buffers</a>.";
+        }
     };
 
+    private static String toHtml() {
+
+        DocumentedType[] types = {
+            BOOLEAN, INT8, INT16, INT32, INT64,
+            UNSIGNED_INT32, VARINT, VARLONG,
+            STRING, NULLABLE_STRING, BYTES, NULLABLE_BYTES,
+            RECORDS, new ArrayOf(STRING)};
+        final StringBuilder b = new StringBuilder();
+        b.append("<table class=\"data-table\"><tbody>\n");
+        b.append("<tr>");
+        b.append("<th>Type</th>\n");
+        b.append("<th>Description</th>\n");
+        b.append("</tr>\n");
+        for (DocumentedType type : types) {
+            b.append("<tr>");
+            b.append("<td>");
+            b.append(type.typeName());
+            b.append("</td>");
+            b.append("<td>");
+            b.append(type.documentation());
+            b.append("</td>");
+            b.append("</tr>\n");
+        }
+        b.append("</table>\n");
+        return b.toString();
+    }
+
+    public static void main(String[] args) {
+        System.out.println(toHtml());
+    }
 }
diff --git a/docs/protocol.html b/docs/protocol.html
index 85f413349eb..eaf920446b1 100644
--- a/docs/protocol.html
+++ b/docs/protocol.html
@@ -162,18 +162,7 @@ <h4><a id="protocol_details" href="#protocol_details">The Protocol</a></h4>
 <h5><a id="protocol_types" href="#protocol_types">Protocol Primitive Types</a></h5>
 
 <p>The protocol is built out of the following primitive types.</p>
-
-<p><b>Fixed Width Primitives</b><p>
-
-<p>int8, int16, int32, int64 - Signed integers with the given precision (in bits) stored in big endian order.</p>
-
-<p><b>Variable Length Primitives</b><p>
-
-<p>bytes, string - These types consist of a signed integer giving a length N followed by N bytes of content. A length of -1 indicates null. string uses an int16 for its size, and bytes uses an int32.</p>
-
-<p><b>Arrays</b><p>
-
-<p>This is a notation for handling repeated structures. These will always be encoded as an int32 size containing the length N followed by N repetitions of the structure which can itself be made up of other primitive types. In the BNF grammars below we will show an array of a structure foo as [foo].</p>
+<!--#include virtual="generated/protocol_types.html" -->
 
 <h5><a id="protocol_grammar" href="#protocol_grammar">Notes on reading the request format grammars</a></h5>
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Add a documentation field for types and update doc generation
> -------------------------------------------------------------
>
>                 Key: KAFKA-3365
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3365
>             Project: Kafka
>          Issue Type: Sub-task
>            Reporter: Grant Henke
>            Assignee: Andras Beni
>            Priority: Major
>
> Currently the type class does not allow a documentation field. This means we can't auto generate a high level documentation summary for each type in the protocol. Adding this field and updating the generated output would be valuable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)