kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gwens...@apache.org
Subject kafka git commit: KAFKA-3361: Initial protocol documentation page and generation (0.9)
Date Thu, 10 Mar 2016 06:02:29 GMT
Repository: kafka
Updated Branches:
  refs/heads/0.9.0 f91d993a5 -> fcb86d58f


KAFKA-3361: Initial protocol documentation page and generation (0.9)

- Moves all generated docs under /docs/generated
- Generates docs for Protocol, Errors, and ApiKeys
- Adds new protocol.html page

Author: Grant Henke <granthenke@gmail.com>

Reviewers: Gwen Shapira

Closes #1038 from granthenke/protocol-docs-0.9


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

Branch: refs/heads/0.9.0
Commit: fcb86d58f302172b652b674052b3800bdf86d476
Parents: f91d993
Author: Grant Henke <granthenke@gmail.com>
Authored: Wed Mar 9 22:02:14 2016 -0800
Committer: Gwen Shapira <cshapi@gmail.com>
Committed: Wed Mar 9 22:02:14 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   6 +-
 build.gradle                                    |  47 +++++-
 .../apache/kafka/common/protocol/ApiKeys.java   |  26 +++
 .../apache/kafka/common/protocol/Errors.java    |  35 +++-
 .../apache/kafka/common/protocol/Protocol.java  | 166 +++++++++++++++++++
 .../kafka/common/protocol/types/Field.java      |   4 +
 docs/configuration.html                         |   8 +-
 docs/protocol.html                              | 163 ++++++++++++++++++
 8 files changed, 437 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 5a8ba0f..50e1f85 100644
--- a/.gitignore
+++ b/.gitignore
@@ -37,7 +37,5 @@ results
 tests/results
 .ducktape
 tests/.ducktape
-docs/producer_config.html
-docs/consumer_config.html
-docs/kafka_config.html
-docs/connect_config.html
+
+docs/generated/

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index a259cb8..19ac665 100644
--- a/build.gradle
+++ b/build.gradle
@@ -56,6 +56,8 @@ ext {
     mavenUrl = project.hasProperty('mavenUrl') ? project.mavenUrl : ''
     mavenUsername = project.hasProperty('mavenUsername') ? project.mavenUsername : ''
     mavenPassword = project.hasProperty('mavenPassword') ? project.mavenPassword : ''
+
+    generatedDocsDir = new File("${project.rootDir}/docs/generated")
 }
 
 apply from: file('wrapper.gradle')
@@ -303,25 +305,51 @@ project(':core') {
     into "$buildDir/dependant-libs-${scalaVersion}"
   }
 
-  tasks.create(name: "genProducerConfigDocs", dependsOn:jar, type: JavaExec) {
+  task genProtocolErrorDocs(type: JavaExec) {
+    classpath = sourceSets.main.runtimeClasspath
+    main = 'org.apache.kafka.common.protocol.Errors'
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "protocol_errors.html").newOutputStream()
+  }
+
+  task genProtocolApiKeyDocs(type: JavaExec) {
+    classpath = sourceSets.main.runtimeClasspath
+    main = 'org.apache.kafka.common.protocol.ApiKeys'
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "protocol_api_keys.html").newOutputStream()
+  }
+
+  task genProtocolMessageDocs(type: JavaExec) {
+    classpath = sourceSets.main.runtimeClasspath
+    main = 'org.apache.kafka.common.protocol.Protocol'
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "protocol_messages.html").newOutputStream()
+  }
+
+  task genProducerConfigDocs(type: JavaExec) {
     classpath = sourceSets.main.runtimeClasspath
     main = 'org.apache.kafka.clients.producer.ProducerConfig'
-    standardOutput = new File('docs/producer_config.html').newOutputStream()
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "producer_config.html").newOutputStream()
   }
 
-  tasks.create(name: "genConsumerConfigDocs", dependsOn:jar, type: JavaExec) {
+  task genConsumerConfigDocs(type: JavaExec) {
     classpath = sourceSets.main.runtimeClasspath
     main = 'org.apache.kafka.clients.consumer.ConsumerConfig'
-    standardOutput = new File('docs/consumer_config.html').newOutputStream()
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "consumer_config.html").newOutputStream()
   }
 
-  tasks.create(name: "genKafkaConfigDocs", dependsOn:jar, type: JavaExec) {
+  task genKafkaConfigDocs(type: JavaExec) {
     classpath = sourceSets.main.runtimeClasspath
     main = 'kafka.server.KafkaConfig'
-    standardOutput = new File('docs/kafka_config.html').newOutputStream()
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "kafka_config.html").newOutputStream()
   }
 
-  task siteDocsTar(dependsOn: ['genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs',
':connect:runtime:genConnectConfigDocs'], type: Tar) {
+  task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs',
+                               'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs',
+                               ':connect:runtime:genConnectConfigDocs'], type: Tar) {
     classifier = 'site-docs'
     compression = Compression.GZIP
     from project.file("../docs")
@@ -764,10 +792,11 @@ project(':connect:runtime') {
   }
   test.dependsOn('checkstyleMain', 'checkstyleTest')
 
-  tasks.create(name: "genConnectConfigDocs", dependsOn:jar, type: JavaExec) {
+  task genConnectConfigDocs(type: JavaExec) {
     classpath = sourceSets.main.runtimeClasspath
     main = 'org.apache.kafka.connect.runtime.distributed.DistributedConfig'
-    standardOutput = new File('docs/connect_config.html').newOutputStream()
+    if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
+    standardOutput = new File(generatedDocsDir, "connect_config.html").newOutputStream()
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index 5bd3c96..6084583 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -67,4 +67,30 @@ public enum ApiKeys {
     public static ApiKeys forId(int id) {
         return codeToType[id];
     }
+
+    private static String toHtml() {
+        final StringBuilder b = new StringBuilder();
+        b.append("<table class=\"data-table\"><tbody>\n");
+        b.append("<tr>");
+        b.append("<th>Name</th>\n");
+        b.append("<th>Key</th>\n");
+        b.append("</tr>");
+        for (ApiKeys key : ApiKeys.values()) {
+            b.append("<tr>\n");
+            b.append("<td>");
+            b.append(key.name);
+            b.append("</td>");
+            b.append("<td>");
+            b.append(key.id);
+            b.append("</td>");
+            b.append("</tr>\n");
+        }
+        b.append("</table>\n");
+        return b.toString();
+    }
+
+    public static void main(String[] args) {
+        System.out.println(toHtml());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index dd4f39c..0c021b5 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -27,7 +27,7 @@ import org.slf4j.LoggerFactory;
 /**
  * This class contains all the client-server errors--those errors that must be sent from
the server to the client. These
  * are thus part of the protocol. The names can be changed but the error code cannot.
- * 
+ *
  * Do not add exceptions that occur only on the client or only on the server here.
  */
 public enum Errors {
@@ -159,4 +159,37 @@ public enum Errors {
         Errors error = classToError.get(t.getClass());
         return error == null ? UNKNOWN : error;
     }
+
+    private static String toHtml() {
+        final StringBuilder b = new StringBuilder();
+        b.append("<table class=\"data-table\"><tbody>\n");
+        b.append("<tr>");
+        b.append("<th>Error</th>\n");
+        b.append("<th>Code</th>\n");
+        b.append("<th>Retriable</th>\n");
+        b.append("<th>Description</th>\n");
+        b.append("</tr>\n");
+        for (Errors error : Errors.values()) {
+            b.append("<tr>");
+            b.append("<td>");
+            b.append(error.name());
+            b.append("</td>");
+            b.append("<td>");
+            b.append(error.code());
+            b.append("</td>");
+            b.append("<td>");
+            b.append(error.exception() != null && error.exception() instanceof RetriableException
? "True" : "False");
+            b.append("</td>");
+            b.append("<td>");
+            b.append(error.exception() != null ? error.exception().getMessage() : "");
+            b.append("</td>");
+            b.append("</tr>\n");
+        }
+        b.append("</table>\n");
+        return b.toString();
+    }
+
+    public static void main(String[] args) {
+        System.out.println(toHtml());
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 48c64c2..26448c6 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -19,6 +19,12 @@ package org.apache.kafka.common.protocol;
 import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Type;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
 
 import static org.apache.kafka.common.protocol.types.Type.BYTES;
 import static org.apache.kafka.common.protocol.types.Type.INT16;
@@ -708,4 +714,164 @@ public class Protocol {
                         + " but " + RESPONSES[api.id].length + " response versions.");
     }
 
+    private static String indentString(int size) {
+        StringBuilder b = new StringBuilder(size);
+        for (int i = 0; i < size; i++)
+            b.append(" ");
+        return b.toString();
+    }
+
+    private static void schemaToBnfHtml(Schema schema, StringBuilder b, int indentSize) {
+        final String indentStr = indentString(indentSize);
+        final Map<String, Type> subTypes = new LinkedHashMap<>();
+
+        // Top level fields
+        for (Field field: schema.fields()) {
+            if (field.type instanceof ArrayOf) {
+                b.append("[");
+                b.append(field.name);
+                b.append("] ");
+                Type innerType = ((ArrayOf) field.type).type();
+                if (innerType instanceof Schema && !subTypes.containsKey(field.name))
+                    subTypes.put(field.name, (Schema) innerType);
+            } else if (field.type instanceof Schema) {
+                b.append(field.name);
+                b.append(" ");
+                if (!subTypes.containsKey(field.name))
+                    subTypes.put(field.name, (Schema) field.type);
+            } else {
+                b.append(field.name);
+                b.append(" ");
+                if (!subTypes.containsKey(field.name))
+                    subTypes.put(field.name, field.type);
+            }
+        }
+        b.append("\n");
+
+        // Sub Types/Schemas
+        for (Map.Entry<String, Type> entry: subTypes.entrySet()) {
+            if (entry.getValue() instanceof Schema) {
+                // Complex Schema Type
+                b.append(indentStr);
+                b.append(entry.getKey());
+                b.append(" => ");
+                schemaToBnfHtml((Schema) entry.getValue(), b, indentSize + 2);
+            } else {
+                // Standard Field Type
+                b.append(indentStr);
+                b.append(entry.getKey());
+                b.append(" => ");
+                b.append(entry.getValue());
+                b.append("\n");
+            }
+        }
+    }
+
+    private static void populateSchemaFields(Schema schema, Set<Field> fields) {
+        for (Field field: schema.fields()) {
+            fields.add(field);
+            if (field.type instanceof ArrayOf) {
+                Type innerType = ((ArrayOf) field.type).type();
+                if (innerType instanceof Schema)
+                    populateSchemaFields((Schema) innerType, fields);
+            } else if (field.type instanceof Schema)
+                populateSchemaFields((Schema) field.type, fields);
+        }
+    }
+
+    private static void schemaToFieldTableHtml(Schema schema, StringBuilder b) {
+        Set<Field> fields = new LinkedHashSet<>();
+        populateSchemaFields(schema, fields);
+
+        b.append("<table class=\"data-table\"><tbody>\n");
+        b.append("<tr>");
+        b.append("<th>Field</th>\n");
+        b.append("<th>Description</th>\n");
+        b.append("</tr>");
+        for (Field field : fields) {
+            b.append("<tr>\n");
+            b.append("<td>");
+            b.append(field.name);
+            b.append("</td>");
+            b.append("<td>");
+            b.append(field.doc);
+            b.append("</td>");
+            b.append("</tr>\n");
+        }
+        b.append("</table>\n");
+    }
+
+    public static String toHtml() {
+        final StringBuilder b = new StringBuilder();
+        b.append("<h5>Headers:</h5>\n");
+
+        b.append("<pre>");
+        b.append("Request Header => ");
+        schemaToBnfHtml(REQUEST_HEADER, b, 2);
+        b.append("</pre>\n");
+        schemaToFieldTableHtml(REQUEST_HEADER, b);
+
+        b.append("<pre>");
+        b.append("Response Header => ");
+        schemaToBnfHtml(RESPONSE_HEADER, b, 2);
+        b.append("</pre>\n");
+        schemaToFieldTableHtml(RESPONSE_HEADER, b);
+
+        for (ApiKeys key : ApiKeys.values()) {
+            // Key
+            b.append("<h5>");
+            b.append(key.name);
+            b.append(" API (Key: ");
+            b.append(key.id);
+            b.append("):</h5>\n\n");
+            // Requests
+            b.append("<b>Requests:</b><br>\n");
+            Schema[] requests = REQUESTS[key.id];
+            for (int i = 0; i < requests.length; i++) {
+                Schema schema = requests[i];
+                // Schema
+                if (schema != null) {
+                    b.append("<p>");
+                    // Version header
+                    b.append("<pre>");
+                    b.append(key.name);
+                    b.append(" Request (Version: ");
+                    b.append(i);
+                    b.append(") => ");
+                    schemaToBnfHtml(requests[i], b, 2);
+                    b.append("</pre>");
+                    schemaToFieldTableHtml(requests[i], b);
+                }
+                b.append("</p>\n");
+            }
+
+            // Responses
+            b.append("<b>Responses:</b><br>\n");
+            Schema[] responses = RESPONSES[key.id];
+            for (int i = 0; i < responses.length; i++) {
+                Schema schema = responses[i];
+                // Schema
+                if (schema != null) {
+                    b.append("<p>");
+                    // Version header
+                    b.append("<pre>");
+                    b.append(key.name);
+                    b.append(" Response (Version: ");
+                    b.append(i);
+                    b.append(") => ");
+                    schemaToBnfHtml(responses[i], b, 2);
+                    b.append("</pre>");
+                    schemaToFieldTableHtml(responses[i], b);
+                }
+                b.append("</p>\n");
+            }
+        }
+
+        return b.toString();
+    }
+
+    public static void main(String[] args) {
+        System.out.println(toHtml());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
index 1eb1195..cab7bf4 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
@@ -66,4 +66,8 @@ public class Field {
         return type;
     }
 
+    public Schema schema() {
+        return schema;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/docs/configuration.html
----------------------------------------------------------------------
diff --git a/docs/configuration.html b/docs/configuration.html
index b633540..40aded5 100644
--- a/docs/configuration.html
+++ b/docs/configuration.html
@@ -28,7 +28,7 @@ The essential configurations are the following:
 
 Topic-level configurations and defaults are discussed in more detail <a href="#topic-config">below</a>.
 
-<!--#include virtual="kafka_config.html" -->
+<!--#include virtual="generated/kafka_config.html" -->
 
 <p>More details about broker configuration can be found in the scala class <code>kafka.server.KafkaConfig</code>.</p>
 
@@ -150,7 +150,7 @@ The following are the topic-level configurations. The server's default
configura
 <h3><a id="producerconfigs" href="#producerconfigs">3.2 Producer Configs</a></h3>
 
 Below is the configuration of the Java producer:
-<!--#include virtual="producer_config.html" -->
+<!--#include virtual="generated/producer_config.html" -->
 
 <p>
     For those interested in the legacy Scala producer configs, information can be found <a
href="http://kafka.apache.org/082/documentation.html#producerconfigs">
@@ -330,7 +330,7 @@ The essential old consumer configurations are the following:
 
 <h4><a id="newconsumerconfigs" href="#newconsumerconfigs">3.3.2 New Consumer
Configs</a></h4>
 Since 0.9.0.0 we have been working on a replacement for our existing simple and high-level
consumers. The code is considered beta quality. Below is the configuration for the new consumer:
-<!--#include virtual="consumer_config.html" -->
+<!--#include virtual="generated/consumer_config.html" -->
 
 <h3><a id="connectconfigs" href="#connectconfigs">3.4 Kafka Connect Configs</a></h3>
-<!--#include virtual="connect_config.html" -->
+<!--#include virtual="generated/connect_config.html" -->

http://git-wip-us.apache.org/repos/asf/kafka/blob/fcb86d58/docs/protocol.html
----------------------------------------------------------------------
diff --git a/docs/protocol.html b/docs/protocol.html
new file mode 100644
index 0000000..98923aa
--- /dev/null
+++ b/docs/protocol.html
@@ -0,0 +1,163 @@
+<h3><a id="protocol" href="#protocol">Kafka Wire Protocol</a></h3>
+
+<p>This document covers the wire protocol implemented in Kafka. It is meant to give
a readable guide to the protocol that covers the available requests, their binary format,
and the proper way to make use of them to implement a client. This document assumes you understand
the basic design and terminology described <a href="https://kafka.apache.org/documentation.html#design">here</a></p>
+
+<ul class="toc">
+    <li><a href="#protocol_preliminaries">Preliminaries</a>
+        <ul>
+            <li><a href="#protocol_network">Network</a>
+            <li><a href="#protocol_partitioning">Partitioning and bootstrapping</a>
+            <li><a href="#protocol_partitioning_strategies">Partitioning Strategies</a>
+            <li><a href="#protocol_batching">Batching</a>
+            <li><a href="#protocol_compatibility">Versioning and Compatibility</a>
+        </ul>
+    </li>
+    <li><a href="#protocol_details">The Protocol</a>
+        <ul>
+            <li><a href="#protocol_types">Protocol Primitive Types</a>
+            <li><a href="#protocol_grammar">Notes on reading the request format
grammars</a>
+            <li><a href="#protocol_common">Common Request and Response Structure</a>
+            <li><a href="#protocol_message_sets">Message Sets</a>
+        </ul>
+    </li>
+    <li><a href="#protocol_constants">Constants</a>
+        <ul>
+            <li><a href="#protocol_error_codes">Error Codes</a>
+            <li><a href="#protocol_api_keys">Api Keys</a>
+        </ul>
+    </li>
+    <li><a href="#protocol_messages">The Messages</a></li>
+    <li><a href="#protocol_philosophy">Some Common Philosophical Questions</a></li>
+</ul>
+
+<h4><a id="protocol_preliminaries" href="#protocol_preliminaries">Preliminaries</a></h4>
+
+<h5><a id="protocol_network" href="#protocol_network">Network</a></h5>
+
+<p>Kafka uses a binary protocol over TCP. The protocol defines all apis as request
response message pairs. All messages are size delimited and are made up of the following primitive
types.</p>
+
+<p>The client initiates a socket connection and then writes a sequence of request messages
and reads back the corresponding response message. No handshake is required on connection
or disconnection. TCP is happier if you maintain persistent connections used for many requests
to amortize the cost of the TCP handshake, but beyond this penalty connecting is pretty cheap.</p>
+
+<p>The client will likely need to maintain a connection to multiple brokers, as data
is partitioned and the clients will need to talk to the server that has their data. However
it should not generally be necessary to maintain multiple connections to a single broker from
a single client instance (i.e. connection pooling).</p>
+
+<p>The server guarantees that on a single TCP connection, requests will be processed
in the order they are sent and responses will return in that order as well. The broker's request
processing allows only a single in-flight request per connection in order to guarantee this
ordering. Note that clients can (and ideally should) use non-blocking IO to implement request
pipelining and achieve higher throughput. i.e., clients can send requests even while awaiting
responses for preceding requests since the outstanding requests will be buffered in the underlying
OS socket buffer. All requests are initiated by the client, and result in a corresponding
response message from the server except where noted.</p>
+
+<p>The server has a configurable maximum limit on request size and any request that
exceeds this limit will result in the socket being disconnected.</p>
+
+<h5><a id="protocol_partitioning" href="#protocol_partitioning">Partitioning
and bootstrapping</a></h5>
+
+<p>Kafka is a partitioned system so not all servers have the complete data set. Instead
recall that topics are split into a pre-defined number of partitions, P, and each partition
is replicated with some replication factor, N. Topic partitions themselves are just ordered
"commit logs" numbered 0, 1, ..., P.</p>
+
+<p>All systems of this nature have the question of how a particular piece of data is
assigned to a particular partition. Kafka clients directly control this assignment, the brokers
themselves enforce no particular semantics of which messages should be published to a particular
partition. Rather, to publish messages the client directly addresses messages to a particular
partition, and when fetching messages, fetches from a particular partition. If two clients
want to use the same partitioning scheme they must use the same method to compute the mapping
of key to partition.</p>
+
+<p>These requests to publish or fetch data must be sent to the broker that is currently
acting as the leader for a given partition. This condition is enforced by the broker, so a
request for a particular partition to the wrong broker will result in an the NotLeaderForPartition
error code (described below).</p>
+
+<p>How can the client find out which topics exist, what partitions they have, and which
brokers currently host those partitions so that it can direct its requests to the right hosts?
This information is dynamic, so you can't just configure each client with some static mapping
file. Instead all Kafka brokers can answer a metadata request that describes the current state
of the cluster: what topics there are, which partitions those topics have, which broker is
the leader for those partitions, and the host and port information for these brokers.</p>
+
+<p>In other words, the client needs to somehow find one broker and that broker will
tell the client about all the other brokers that exist and what partitions they host. This
first broker may itself go down so the best practice for a client implementation is to take
a list of two or three urls to bootstrap from. The user can then choose to use a load balancer
or just statically configure two or three of their kafka hosts in the clients.</p>
+
+<p>The client does not need to keep polling to see if the cluster has changed; it can
fetch metadata once when it is instantiated cache that metadata until it receives an error
indicating that the metadata is out of date. This error can come in two forms: (1) a socket
error indicating the client cannot communicate with a particular broker, (2) an error code
in the response to a request indicating that this broker no longer hosts the partition for
which data was requested.</p>
+<ol>
+    <li>Cycle through a list of "bootstrap" kafka urls until we find one we can connect
to. Fetch cluster metadata.</li>
+    <li>Process fetch or produce requests, directing them to the appropriate broker
based on the topic/partitions they send to or fetch from.</li>
+    <li>If we get an appropriate error, refresh the metadata and try again.</li>
+</ol>
+
+<h5><a id="protocol_partitioning_strategies" href="#protocol_partitioning_strategies">Partitioning
Strategies</a></h5>
+
+<p>As mentioned above the assignment of messages to partitions is something the producing
client controls. That said, how should this functionality be exposed to the end-user?</p>
+
+<p>Partitioning really serves two purposes in Kafka:</p>
+<ol>
+    <li>It balances data and request load over brokers</li>
+    <li>It serves as a way to divvy up processing among consumer processes while allowing
local state and preserving order within the partition. We call this semantic partitioning.</li>
+</ol>
+
+<p>For a given use case you may care about only one of these or both.</p>
+
+<p>To accomplish simple load balancing a simple approach would be for the client to
just round robin requests over all brokers. Another alternative, in an environment where there
are many more producers than brokers, would be to have each client chose a single partition
at random and publish to that. This later strategy will result in far fewer TCP connections.</p>
+
+<p>Semantic partitioning means using some key in the message to assign messages to
partitions. For example if you were processing a click message stream you might want to partition
the stream by the user id so that all data for a particular user would go to a single consumer.
To accomplish this the client can take a key associated with the message and use some hash
of this key to choose the partition to which to deliver the message.</p>
+
+<h5><a id="protocol_batching" href="#protocol_batching">Batching</a></h5>
+
+<p>Our apis encourage batching small things together for efficiency. We have found
this is a very significant performance win. Both our API to send messages and our API to fetch
messages always work with a sequence of messages not a single message to encourage this. A
clever client can make use of this and support an "asynchronous" mode in which it batches
together messages sent individually and sends them in larger clumps. We go even further with
this and allow the batching across multiple topics and partitions, so a produce request may
contain data to append to many partitions and a fetch request may pull data from many partitions
all at once.</p>
+
+<p>The client implementer can choose to ignore this and send everything one at a time
if they like.</p>
+
+<h5><a id="protocol_compatibility" href="#protocol_compatibility">Versioning
and Compatibility</a></h5>
+
+<p>The protocol is designed to enable incremental evolution in a backward compatible
fashion. Our versioning is on a per-api basis, each version consisting of a request and response
pair. Each request contains an API key that identifies the API being invoked and a version
number that indicates the format of the request and the expected format of the response.</p>
+
+<p>The intention is that clients would implement a particular version of the protocol,
and indicate this version in their requests. Our goal is primarily to allow API evolution
in an environment where downtime is not allowed and clients and servers cannot all be changed
at once.</p>
+
+<p>The server will reject requests with a version it does not support, and will always
respond to the client with exactly the protocol format it expects based on the version it
included in its request. The intended upgrade path is that new features would first be rolled
out on the server (with the older clients not making use of them) and then as newer clients
are deployed these new features would gradually be taken advantage of.</p>
+
+<p>Currently all versions are baselined at 0, as we evolve these APIs we will indicate
the format for each version individually.</p>
+
+<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>
+
+<h5><a id="protocol_grammar" href="#protocol_grammar">Notes on reading the request
format grammars</a></h5>
+
+<p>The <a href="https://en.wikipedia.org/wiki/Backus%E2%80%93Naur_Form">BNF</a>s
below give an exact context free grammar for the request and response binary format. The BNF
is intentionally not compact in order to give human-readable name. As always in a BNF a sequence
of productions indicates concatenation. When there are multiple possible productions these
are separated with '|' and may be enclosed in parenthesis for grouping. The top-level definition
is always given first and subsequent sub-parts are indented.</p>
+
+<h5><a id="protocol_common" href="#protocol_common">Common Request and Response
Structure</a></h5>
+
+<p>All requests and responses originate from the following grammar which will be incrementally
describe through the rest of this document:</p>
+
+<pre>
+RequestOrResponse => Size (RequestMessage | ResponseMessage)
+Size => int32
+</pre>
+
+<table class="data-table"><tbody>
+<tr><th>Field</th><th>Description</th></tr>
+<tr><td>message_size</td><td>The message_size field gives the size
of the subsequent request or response message in bytes. The client can read requests by first
reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes
of the request.</td></tr>
+</table>
+
+<h5><a id="protocol_message_sets" href="#protocol_message_sets">Message Sets</a></h5>
+
+<p>A description of the message set format can be found <a href="https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-Messagesets">here</a>.
(KAFKA-3368)</p>
+
+<h4><a id="protocol_constants" href="#protocol_constants">Constants</a></h4>
+
+<h5><a id="protocol_error_codes" href="#protocol_error_codes">Error Codes</a></h5>
+<p>We use numeric codes to indicate what problem occurred on the server. These can
be translated by the client into exceptions or whatever the appropriate error handling mechanism
in the client language. Here is a table of the error codes currently in use:</p>
+<!--#include virtual="generated/protocol_errors.html" -->
+
+<h5><a id="protocol_api_keys" href="#protocol_api_keys">Api Keys</a></h5>
+<p>The following are the numeric codes that the ApiKey in the request can take for
each of the below request types.</p>
+<!--#include virtual="generated/protocol_api_keys.html" -->
+
+<h4><a id="protocol_messages" href="#protocol_messages">The Messages</a></h4>
+
+<p>This section gives details on each of the individual API Messages, their usage,
their binary format, and the meaning of their fields.</p>
+<!--#include virtual="generated/protocol_messages.html" -->
+
+<h4><a id="protocol_philosophy" href="#protocol_philosophy">Some Common Philosophical
Questions</a></h4>
+
+<p>Some people have asked why we don't use HTTP. There are a number of reasons, the
best is that client implementors can make use of some of the more advanced TCP features--the
ability to multiplex requests, the ability to simultaneously poll many connections, etc. We
have also found HTTP libraries in many languages to be surprisingly shabby.</p>
+
+<p>Others have asked if maybe we shouldn't support many different protocols. Prior
experience with this was that it makes it very hard to add and test new features if they have
to be ported across many protocol implementations. Our feeling is that most users don't really
see multiple protocols as a feature, they just want a good reliable client in the language
of their choice.</p>
+
+<p>Another question is why we don't adopt XMPP, STOMP, AMQP or an existing protocol.
The answer to this varies by protocol, but in general the problem is that the protocol does
determine large parts of the implementation and we couldn't do what we are doing if we didn't
have control over the protocol. Our belief is that it is possible to do better than existing
messaging systems have in providing a truly distributed messaging system, and to do this we
need to build something that works differently.</p>
+
+<p>A final question is why we don't use a system like Protocol Buffers or Thrift to
define our request messages. These packages excel at helping you to managing lots and lots
of serialized messages. However we have only a few messages. Support across languages is somewhat
spotty (depending on the package). Finally the mapping between binary log format and wire
protocol is something we manage somewhat carefully and this would not be possible with these
systems. Finally we prefer the style of versioning APIs explicitly and checking this to inferring
new values as nulls as it allows more nuanced control of compatibility.</p>
+
+


Mime
View raw message