This is the multi-page printable view of this section. Click here to print.

Return to the regular view of this page.

Design

1 - Design

Motivation

We designed Kafka to be able to act as a unified platform for handling all the real-time data feeds a large company might have. To do this we had to think through a fairly broad set of use cases.

It would have to have high-throughput to support high volume event streams such as real-time log aggregation.

It would need to deal gracefully with large data backlogs to be able to support periodic data loads from offline systems.

It also meant the system would have to handle low-latency delivery to handle more traditional messaging use-cases.

We wanted to support partitioned, distributed, real-time processing of these feeds to create new, derived feeds. This motivated our partitioning and consumer model.

Finally in cases where the stream is fed into other data systems for serving, we knew the system would have to be able to guarantee fault-tolerance in the presence of machine failures.

Supporting these uses led us to a design with a number of unique elements, more akin to a database log than a traditional messaging system. We will outline some elements of the design in the following sections.

Persistence

Don’t fear the filesystem!

Kafka relies heavily on the filesystem for storing and caching messages. There is a general perception that “disks are slow” which makes people skeptical that a persistent structure can offer competitive performance. In fact disks are both much slower and much faster than people expect depending on how they are used; and a properly designed disk structure can often be as fast as the network.

The key fact about disk performance is that the throughput of hard drives has been diverging from the latency of a disk seek for the last decade. As a result the performance of linear writes on a JBOD configuration with six 7200rpm SATA RAID-5 array is about 600MB/sec but the performance of random writes is only about 100k/sec–a difference of over 6000X. These linear reads and writes are the most predictable of all usage patterns, and are heavily optimized by the operating system. A modern operating system provides read-ahead and write-behind techniques that prefetch data in large block multiples and group smaller logical writes into large physical writes. A further discussion of this issue can be found in this ACM Queue article; they actually find that sequential disk access can in some cases be faster than random memory access!

To compensate for this performance divergence, modern operating systems have become increasingly aggressive in their use of main memory for disk caching. A modern OS will happily divert all free memory to disk caching with little performance penalty when the memory is reclaimed. All disk reads and writes will go through this unified cache. This feature cannot easily be turned off without using direct I/O, so even if a process maintains an in-process cache of the data, this data will likely be duplicated in OS pagecache, effectively storing everything twice.

Furthermore, we are building on top of the JVM, and anyone who has spent any time with Java memory usage knows two things:

  1. The memory overhead of objects is very high, often doubling the size of the data stored (or worse).
  2. Java garbage collection becomes increasingly fiddly and slow as the in-heap data increases.

As a result of these factors using the filesystem and relying on pagecache is superior to maintaining an in-memory cache or other structure–we at least double the available cache by having automatic access to all free memory, and likely double again by storing a compact byte structure rather than individual objects. Doing so will result in a cache of up to 28-30GB on a 32GB machine without GC penalties. Furthermore, this cache will stay warm even if the service is restarted, whereas the in-process cache will need to be rebuilt in memory (which for a 10GB cache may take 10 minutes) or else it will need to start with a completely cold cache (which likely means terrible initial performance). This also greatly simplifies the code as all logic for maintaining coherency between the cache and filesystem is now in the OS, which tends to do so more efficiently and more correctly than one-off in-process attempts. If your disk usage favors linear reads then read-ahead is effectively pre-populating this cache with useful data on each disk read.

This suggests a design which is very simple: rather than maintain as much as possible in-memory and flush it all out to the filesystem in a panic when we run out of space, we invert that. All data is immediately written to a persistent log on the filesystem without necessarily flushing to disk. In effect this just means that it is transferred into the kernel’s pagecache.

This style of pagecache-centric design is described in an article on the design of Varnish here (along with a healthy dose of arrogance).

Constant Time Suffices

The persistent data structure used in messaging systems are often a per-consumer queue with an associated BTree or other general-purpose random access data structures to maintain metadata about messages. BTrees are the most versatile data structure available, and make it possible to support a wide variety of transactional and non-transactional semantics in the messaging system. They do come with a fairly high cost, though: Btree operations are O(log N). Normally O(log N) is considered essentially equivalent to constant time, but this is not true for disk operations. Disk seeks come at 10 ms a pop, and each disk can do only one seek at a time so parallelism is limited. Hence even a handful of disk seeks leads to very high overhead. Since storage systems mix very fast cached operations with very slow physical disk operations, the observed performance of tree structures is often superlinear as data increases with fixed cache–i.e. doubling your data makes things much worse than twice as slow.

Intuitively a persistent queue could be built on simple reads and appends to files as is commonly the case with logging solutions. This structure has the advantage that all operations are O(1) and reads do not block writes or each other. This has obvious performance advantages since the performance is completely decoupled from the data size–one server can now take full advantage of a number of cheap, low-rotational speed 1+TB SATA drives. Though they have poor seek performance, these drives have acceptable performance for large reads and writes and come at 1/3 the price and 3x the capacity.

Having access to virtually unlimited disk space without any performance penalty means that we can provide some features not usually found in a messaging system. For example, in Kafka, instead of attempting to delete messages as soon as they are consumed, we can retain messages for a relatively long period (say a week). This leads to a great deal of flexibility for consumers, as we will describe.

Efficiency

We have put significant effort into efficiency. One of our primary use cases is handling web activity data, which is very high volume: each page view may generate dozens of writes. Furthermore, we assume each message published is read by at least one consumer (often many), hence we strive to make consumption as cheap as possible.

We have also found, from experience building and running a number of similar systems, that efficiency is a key to effective multi-tenant operations. If the downstream infrastructure service can easily become a bottleneck due to a small bump in usage by the application, such small changes will often create problems. By being very fast we help ensure that the application will tip-over under load before the infrastructure. This is particularly important when trying to run a centralized service that supports dozens or hundreds of applications on a centralized cluster as changes in usage patterns are a near-daily occurrence.

We discussed disk efficiency in the previous section. Once poor disk access patterns have been eliminated, there are two common causes of inefficiency in this type of system: too many small I/O operations, and excessive byte copying.

The small I/O problem happens both between the client and the server and in the server’s own persistent operations.

To avoid this, our protocol is built around a “message set” abstraction that naturally groups messages together. This allows network requests to group messages together and amortize the overhead of the network roundtrip rather than sending a single message at a time. The server in turn appends chunks of messages to its log in one go, and the consumer fetches large linear chunks at a time.

This simple optimization produces orders of magnitude speed up. Batching leads to larger network packets, larger sequential disk operations, contiguous memory blocks, and so on, all of which allows Kafka to turn a bursty stream of random message writes into linear writes that flow to the consumers.

The other inefficiency is in byte copying. At low message rates this is not an issue, but under load the impact is significant. To avoid this we employ a standardized binary message format that is shared by the producer, the broker, and the consumer (so data chunks can be transferred without modification between them).

The message log maintained by the broker is itself just a directory of files, each populated by a sequence of message sets that have been written to disk in the same format used by the producer and consumer. Maintaining this common format allows optimization of the most important operation: network transfer of persistent log chunks. Modern unix operating systems offer a highly optimized code path for transferring data out of pagecache to a socket; in Linux this is done with the sendfile system call.

To understand the impact of sendfile, it is important to understand the common data path for transfer of data from file to socket:

  1. The operating system reads data from the disk into pagecache in kernel space
  2. The application reads the data from kernel space into a user-space buffer
  3. The application writes the data back into kernel space into a socket buffer
  4. The operating system copies the data from the socket buffer to the NIC buffer where it is sent over the network

This is clearly inefficient, there are four copies and two system calls. Using sendfile, this re-copying is avoided by allowing the OS to send the data from pagecache to the network directly. So in this optimized path, only the final copy to the NIC buffer is needed.

We expect a common use case to be multiple consumers on a topic. Using the zero-copy optimization above, data is copied into pagecache exactly once and reused on each consumption instead of being stored in memory and copied out to user-space every time it is read. This allows messages to be consumed at a rate that approaches the limit of the network connection.

This combination of pagecache and sendfile means that on a Kafka cluster where the consumers are mostly caught up you will see no read activity on the disks whatsoever as they will be serving data entirely from cache.

For more background on the sendfile and zero-copy support in Java, see this article.

End-to-end Batch Compression

In some cases the bottleneck is actually not CPU or disk but network bandwidth. This is particularly true for a data pipeline that needs to send messages between data centers over a wide-area network. Of course, the user can always compress its messages one at a time without any support needed from Kafka, but this can lead to very poor compression ratios as much of the redundancy is due to repetition between messages of the same type (e.g. field names in JSON or user agents in web logs or common string values). Efficient compression requires compressing multiple messages together rather than compressing each message individually.

Kafka supports this with an efficient batching format. A batch of messages can be clumped together compressed and sent to the server in this form. This batch of messages will be written in compressed form and will remain compressed in the log and will only be decompressed by the consumer.

Kafka supports GZIP, Snappy, LZ4 and ZStandard compression protocols. More details on compression can be found here.

The Producer

Load balancing

The producer sends data directly to the broker that is the leader for the partition without any intervening routing tier. To help the producer do this all Kafka nodes can answer a request for metadata about which servers are alive and where the leaders for the partitions of a topic are at any given time to allow the producer to appropriately direct its requests.

The client controls which partition it publishes messages to. This can be done at random, implementing a kind of random load balancing, or it can be done by some semantic partitioning function. We expose the interface for semantic partitioning by allowing the user to specify a key to partition by and using this to hash to a partition (there is also an option to override the partition function if need be). For example if the key chosen was a user id then all data for a given user would be sent to the same partition. This in turn will allow consumers to make locality assumptions about their consumption. This style of partitioning is explicitly designed to allow locality-sensitive processing in consumers.

Asynchronous send

Batching is one of the big drivers of efficiency, and to enable batching the Kafka producer will attempt to accumulate data in memory and to send out larger batches in a single request. The batching can be configured to accumulate no more than a fixed number of messages and to wait no longer than some fixed latency bound (say 64k or 10 ms). This allows the accumulation of more bytes to send, and few larger I/O operations on the servers. This buffering is configurable and gives a mechanism to trade off a small amount of additional latency for better throughput.

Details on configuration and the api for the producer can be found elsewhere in the documentation.

The Consumer

The Kafka consumer works by issuing “fetch” requests to the brokers leading the partitions it wants to consume. The consumer specifies its offset in the log with each request and receives back a chunk of log beginning from that position. The consumer thus has significant control over this position and can rewind it to re-consume data if need be.

Push vs. pull

An initial question we considered is whether consumers should pull data from brokers or brokers should push data to the consumer. In this respect Kafka follows a more traditional design, shared by most messaging systems, where data is pushed to the broker from the producer and pulled from the broker by the consumer. Some logging-centric systems, such as Scribe and Apache Flume, follow a very different push-based path where data is pushed downstream. There are pros and cons to both approaches. However, a push-based system has difficulty dealing with diverse consumers as the broker controls the rate at which data is transferred. The goal is generally for the consumer to be able to consume at the maximum possible rate; unfortunately, in a push system this means the consumer tends to be overwhelmed when its rate of consumption falls below the rate of production (a denial of service attack, in essence). A pull-based system has the nicer property that the consumer simply falls behind and catches up when it can. This can be mitigated with some kind of backoff protocol by which the consumer can indicate it is overwhelmed, but getting the rate of transfer to fully utilize (but never over-utilize) the consumer is trickier than it seems. Previous attempts at building systems in this fashion led us to go with a more traditional pull model.

Another advantage of a pull-based system is that it lends itself to aggressive batching of data sent to the consumer. A push-based system must choose to either send a request immediately or accumulate more data and then send it later without knowledge of whether the downstream consumer will be able to immediately process it. If tuned for low latency, this will result in sending a single message at a time only for the transfer to end up being buffered anyway, which is wasteful. A pull-based design fixes this as the consumer always pulls all available messages after its current position in the log (or up to some configurable max size). So one gets optimal batching without introducing unnecessary latency.

The deficiency of a naive pull-based system is that if the broker has no data the consumer may end up polling in a tight loop, effectively busy-waiting for data to arrive. To avoid this we have parameters in our pull request that allow the consumer request to block in a “long poll” waiting until data arrives (and optionally waiting until a given number of bytes is available to ensure large transfer sizes).

You could imagine other possible designs which would be only pull, end-to-end. The producer would locally write to a local log, and brokers would pull from that with consumers pulling from them. A similar type of “store-and-forward” producer is often proposed. This is intriguing but we felt not very suitable for our target use cases which have thousands of producers. Our experience running persistent data systems at scale led us to feel that involving thousands of disks in the system across many applications would not actually make things more reliable and would be a nightmare to operate. And in practice we have found that we can run a pipeline with strong SLAs at large scale without a need for producer persistence.

Consumer Position

Keeping track of what has been consumed is, surprisingly, one of the key performance points of a messaging system.

Most messaging systems keep metadata about what messages have been consumed on the broker. That is, as a message is handed out to a consumer, the broker either records that fact locally immediately or it may wait for acknowledgement from the consumer. This is a fairly intuitive choice, and indeed for a single machine server it is not clear where else this state could go. Since the data structures used for storage in many messaging systems scale poorly, this is also a pragmatic choice–since the broker knows what is consumed it can immediately delete it, keeping the data size small.

What is perhaps not obvious is that getting the broker and consumer to come into agreement about what has been consumed is not a trivial problem. If the broker records a message as consumed immediately every time it is handed out over the network, then if the consumer fails to process the message (say because it crashes or the request times out or whatever) that message will be lost. To solve this problem, many messaging systems add an acknowledgement feature which means that messages are only marked as sent not consumed when they are sent; the broker waits for a specific acknowledgement from the consumer to record the message as consumed. This strategy fixes the problem of losing messages, but creates new problems. First of all, if the consumer processes the message but fails before it can send an acknowledgement then the message will be consumed twice. The second problem is around performance, now the broker must keep multiple states about every single message (first to lock it so it is not given out a second time, and then to mark it as permanently consumed so that it can be removed). Tricky problems must be dealt with, like what to do with messages that are sent but never acknowledged.

Kafka handles this differently. Our topic is divided into a set of totally ordered partitions, each of which is consumed by exactly one consumer within each subscribing consumer group at any given time. This means that the position of a consumer in each partition is just a single integer, the offset of the next message to consume. This makes the state about what has been consumed very small, just one number for each partition. This state can be periodically checkpointed. This makes the equivalent of message acknowledgements very cheap.

There is a side benefit of this decision. A consumer can deliberately rewind back to an old offset and re-consume data. This violates the common contract of a queue, but turns out to be an essential feature for many consumers. For example, if the consumer code has a bug and is discovered after some messages are consumed, the consumer can re-consume those messages once the bug is fixed.

Offline Data Load

Scalable persistence allows for the possibility of consumers that only periodically consume such as batch data loads that periodically bulk-load data into an offline system such as Hadoop or a relational data warehouse.

In the case of Hadoop we parallelize the data load by splitting the load over individual map tasks, one for each node/topic/partition combination, allowing full parallelism in the loading. Hadoop provides the task management, and tasks which fail can restart without danger of duplicate data–they simply restart from their original position.

Message Delivery Semantics

Now that we understand a little about how producers and consumers work, let’s discuss the semantic guarantees Kafka provides between producer and consumer. Clearly there are multiple possible message delivery guarantees that could be provided:

  • At most once –Messages may be lost but are never redelivered.
  • At least once –Messages are never lost but may be redelivered.
  • Exactly once –this is what people actually want, each message is delivered once and only once.

It’s worth noting that this breaks down into two problems: the durability guarantees for publishing a message and the guarantees when consuming a message.

Many systems claim to provide “exactly once” delivery semantics, but it is important to read the fine print, most of these claims are misleading (i.e. they don’t translate to the case where consumers or producers can fail, cases where there are multiple consumer processes, or cases where data written to disk can be lost).

Kafka’s semantics are straight-forward. When publishing a message we have a notion of the message being “committed” to the log. Once a published message is committed it will not be lost as long as one broker that replicates the partition to which this message was written remains “alive”. The definition of committed message, alive partition as well as a description of which types of failures we attempt to handle will be described in more detail in the next section. For now let’s assume a perfect, lossless broker and try to understand the guarantees to the producer and consumer. If a producer attempts to publish a message and experiences a network error it cannot be sure if this error happened before or after the message was committed. This is similar to the semantics of inserting into a database table with an autogenerated key.

Prior to 0.11.0.0, if a producer failed to receive a response indicating that a message was committed, it had little choice but to resend the message. This provides at-least-once delivery semantics since the message may be written to the log again during resending if the original request had in fact succeeded. Since 0.11.0.0, the Kafka producer also supports an idempotent delivery option which guarantees that resending will not result in duplicate entries in the log. To achieve this, the broker assigns each producer an ID and deduplicates messages using a sequence number that is sent by the producer along with every message. Also beginning with 0.11.0.0, the producer supports the ability to send messages to multiple topic partitions using transaction-like semantics: i.e. either all messages are successfully written or none of them are. The main use case for this is exactly-once processing between Kafka topics (described below).

Not all use cases require such strong guarantees. For uses which are latency sensitive we allow the producer to specify the durability level it desires. If the producer specifies that it wants to wait on the message being committed this can take on the order of 10 ms. However the producer can also specify that it wants to perform the send completely asynchronously or that it wants to wait only until the leader (but not necessarily the followers) have the message.

Now let’s describe the semantics from the point-of-view of the consumer. All replicas have the exact same log with the same offsets. The consumer controls its position in this log. If the consumer never crashed it could just store this position in memory, but if the consumer fails and we want this topic partition to be taken over by another process the new process will need to choose an appropriate position from which to start processing. Let’s say the consumer reads some messages – it has several options for processing the messages and updating its position.

  1. It can read the messages, then save its position in the log, and finally process the messages. In this case there is a possibility that the consumer process crashes after saving its position but before saving the output of its message processing. In this case the process that took over processing would start at the saved position even though a few messages prior to that position had not been processed. This corresponds to “at-most-once” semantics as in the case of a consumer failure messages may not be processed.
  2. It can read the messages, process the messages, and finally save its position. In this case there is a possibility that the consumer process crashes after processing messages but before saving its position. In this case when the new process takes over the first few messages it receives will already have been processed. This corresponds to the “at-least-once” semantics in the case of consumer failure. In many cases messages have a primary key and so the updates are idempotent (receiving the same message twice just overwrites a record with another copy of itself).

So what about exactly once semantics (i.e. the thing you actually want)? When consuming from a Kafka topic and producing to another topic (as in a Kafka Streams application), we can leverage the new transactional producer capabilities in 0.11.0.0 that were mentioned above. The consumer’s position is stored as a message in a topic, so we can write the offset to Kafka in the same transaction as the output topics receiving the processed data. If the transaction is aborted, the consumer’s position will revert to its old value and the produced data on the output topics will not be visible to other consumers, depending on their “isolation level.” In the default “read_uncommitted” isolation level, all messages are visible to consumers even if they were part of an aborted transaction, but in “read_committed,” the consumer will only return messages from transactions which were committed (and any messages which were not part of a transaction).

When writing to an external system, the limitation is in the need to coordinate the consumer’s position with what is actually stored as output. The classic way of achieving this would be to introduce a two-phase commit between the storage of the consumer position and the storage of the consumers output. But this can be handled more simply and generally by letting the consumer store its offset in the same place as its output. This is better because many of the output systems a consumer might want to write to will not support a two-phase commit. As an example of this, consider a Kafka Connect connector which populates data in HDFS along with the offsets of the data it reads so that it is guaranteed that either data and offsets are both updated or neither is. We follow similar patterns for many other data systems which require these stronger semantics and for which the messages do not have a primary key to allow for deduplication.

So effectively Kafka supports exactly-once delivery in Kafka Streams, and the transactional producer/consumer can be used generally to provide exactly-once delivery when transferring and processing data between Kafka topics. Exactly-once delivery for other destination systems generally requires cooperation with such systems, but Kafka provides the offset which makes implementing this feasible (see also Kafka Connect). Otherwise, Kafka guarantees at-least-once delivery by default, and allows the user to implement at-most-once delivery by disabling retries on the producer and committing offsets in the consumer prior to processing a batch of messages.

Replication

Kafka replicates the log for each topic’s partitions across a configurable number of servers (you can set this replication factor on a topic-by-topic basis). This allows automatic failover to these replicas when a server in the cluster fails so messages remain available in the presence of failures.

Other messaging systems provide some replication-related features, but, in our (totally biased) opinion, this appears to be a tacked-on thing, not heavily used, and with large downsides: replicas are inactive, throughput is heavily impacted, it requires fiddly manual configuration, etc. Kafka is meant to be used with replication by default–in fact we implement un-replicated topics as replicated topics where the replication factor is one.

The unit of replication is the topic partition. Under non-failure conditions, each partition in Kafka has a single leader and zero or more followers. The total number of replicas including the leader constitute the replication factor. All reads and writes go to the leader of the partition. Typically, there are many more partitions than brokers and the leaders are evenly distributed among brokers. The logs on the followers are identical to the leader’s log–all have the same offsets and messages in the same order (though, of course, at any given time the leader may have a few as-yet unreplicated messages at the end of its log).

Followers consume messages from the leader just as a normal Kafka consumer would and apply them to their own log. Having the followers pull from the leader has the nice property of allowing the follower to naturally batch together log entries they are applying to their log.

As with most distributed systems automatically handling failures requires having a precise definition of what it means for a node to be “alive”. For Kafka node liveness has two conditions

  1. A node must be able to maintain its session with ZooKeeper (via ZooKeeper’s heartbeat mechanism)
  2. If it is a follower it must replicate the writes happening on the leader and not fall “too far” behind We refer to nodes satisfying these two conditions as being “in sync” to avoid the vagueness of “alive” or “failed”. The leader keeps track of the set of “in sync” nodes. If a follower dies, gets stuck, or falls behind, the leader will remove it from the list of in sync replicas. The determination of stuck and lagging replicas is controlled by the replica.lag.time.max.ms configuration.

In distributed systems terminology we only attempt to handle a “fail/recover” model of failures where nodes suddenly cease working and then later recover (perhaps without knowing that they have died). Kafka does not handle so-called “Byzantine” failures in which nodes produce arbitrary or malicious responses (perhaps due to bugs or foul play).

We can now more precisely define that a message is considered committed when all in sync replicas for that partition have applied it to their log. Only committed messages are ever given out to the consumer. This means that the consumer need not worry about potentially seeing a message that could be lost if the leader fails. Producers, on the other hand, have the option of either waiting for the message to be committed or not, depending on their preference for tradeoff between latency and durability. This preference is controlled by the acks setting that the producer uses. Note that topics have a setting for the “minimum number” of in-sync replicas that is checked when the producer requests acknowledgment that a message has been written to the full set of in-sync replicas. If a less stringent acknowledgement is requested by the producer, then the message can be committed, and consumed, even if the number of in-sync replicas is lower than the minimum (e.g. it can be as low as just the leader).

The guarantee that Kafka offers is that a committed message will not be lost, as long as there is at least one in sync replica alive, at all times.

Kafka will remain available in the presence of node failures after a short fail-over period, but may not remain available in the presence of network partitions.

Replicated Logs: Quorums, ISRs, and State Machines (Oh my!)

At its heart a Kafka partition is a replicated log. The replicated log is one of the most basic primitives in distributed data systems, and there are many approaches for implementing one. A replicated log can be used by other systems as a primitive for implementing other distributed systems in the state-machine style.

A replicated log models the process of coming into consensus on the order of a series of values (generally numbering the log entries 0, 1, 2, …). There are many ways to implement this, but the simplest and fastest is with a leader who chooses the ordering of values provided to it. As long as the leader remains alive, all followers need to only copy the values and ordering the leader chooses.

Of course if leaders didn’t fail we wouldn’t need followers! When the leader does die we need to choose a new leader from among the followers. But followers themselves may fall behind or crash so we must ensure we choose an up-to-date follower. The fundamental guarantee a log replication algorithm must provide is that if we tell the client a message is committed, and the leader fails, the new leader we elect must also have that message. This yields a tradeoff: if the leader waits for more followers to acknowledge a message before declaring it committed then there will be more potentially electable leaders.

If you choose the number of acknowledgements required and the number of logs that must be compared to elect a leader such that there is guaranteed to be an overlap, then this is called a Quorum.

A common approach to this tradeoff is to use a majority vote for both the commit decision and the leader election. This is not what Kafka does, but let’s explore it anyway to understand the tradeoffs. Let’s say we have 2 f +1 replicas. If f +1 replicas must receive a message prior to a commit being declared by the leader, and if we elect a new leader by electing the follower with the most complete log from at least f +1 replicas, then, with no more than f failures, the leader is guaranteed to have all committed messages. This is because among any f +1 replicas, there must be at least one replica that contains all committed messages. That replica’s log will be the most complete and therefore will be selected as the new leader. There are many remaining details that each algorithm must handle (such as precisely defined what makes a log more complete, ensuring log consistency during leader failure or changing the set of servers in the replica set) but we will ignore these for now.

This majority vote approach has a very nice property: the latency is dependent on only the fastest servers. That is, if the replication factor is three, the latency is determined by the faster follower not the slower one.

There are a rich variety of algorithms in this family including ZooKeeper’s Zab, Raft, and Viewstamped Replication. The most similar academic publication we are aware of to Kafka’s actual implementation is PacificA from Microsoft.

The downside of majority vote is that it doesn’t take many failures to leave you with no electable leaders. To tolerate one failure requires three copies of the data, and to tolerate two failures requires five copies of the data. In our experience having only enough redundancy to tolerate a single failure is not enough for a practical system, but doing every write five times, with 5x the disk space requirements and 1/5th the throughput, is not very practical for large volume data problems. This is likely why quorum algorithms more commonly appear for shared cluster configuration such as ZooKeeper but are less common for primary data storage. For example in HDFS the namenode’s high-availability feature is built on a majority-vote-based journal, but this more expensive approach is not used for the data itself.

Kafka takes a slightly different approach to choosing its quorum set. Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas (ISR) that are caught-up to the leader. Only members of this set are eligible for election as leader. A write to a Kafka partition is not considered committed until all in-sync replicas have received the write. This ISR set is persisted to ZooKeeper whenever it changes. Because of this, any replica in the ISR is eligible to be elected leader. This is an important factor for Kafka’s usage model where there are many partitions and ensuring leadership balance is important. With this ISR model and f+1 replicas, a Kafka topic can tolerate f failures without losing committed messages.

For most use cases we hope to handle, we think this tradeoff is a reasonable one. In practice, to tolerate f failures, both the majority vote and the ISR approach will wait for the same number of replicas to acknowledge before committing a message (e.g. to survive one failure a majority quorum needs three replicas and one acknowledgement and the ISR approach requires two replicas and one acknowledgement). The ability to commit without the slowest servers is an advantage of the majority vote approach. However, we think it is ameliorated by allowing the client to choose whether they block on the message commit or not, and the additional throughput and disk space due to the lower required replication factor is worth it.

Another important design distinction is that Kafka does not require that crashed nodes recover with all their data intact. It is not uncommon for replication algorithms in this space to depend on the existence of “stable storage” that cannot be lost in any failure-recovery scenario without potential consistency violations. There are two primary problems with this assumption. First, disk errors are the most common problem we observe in real operation of persistent data systems and they often do not leave data intact. Secondly, even if this were not a problem, we do not want to require the use of fsync on every write for our consistency guarantees as this can reduce performance by two to three orders of magnitude. Our protocol for allowing a replica to rejoin the ISR ensures that before rejoining, it must fully re-sync again even if it lost unflushed data in its crash.

Unclean leader election: What if they all die?

Note that Kafka’s guarantee with respect to data loss is predicated on at least one replica remaining in sync. If all the nodes replicating a partition die, this guarantee no longer holds.

However a practical system needs to do something reasonable when all the replicas die. If you are unlucky enough to have this occur, it is important to consider what will happen. There are two behaviors that could be implemented:

  1. Wait for a replica in the ISR to come back to life and choose this replica as the leader (hopefully it still has all its data).
  2. Choose the first replica (not necessarily in the ISR) that comes back to life as the leader.

This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become leader, then its log becomes the source of truth even though it is not guaranteed to have every committed message. By default from version 0.11.0.0, Kafka chooses the first strategy and favor waiting for a consistent replica. This behavior can be changed using configuration property unclean.leader.election.enable, to support use cases where uptime is preferable to consistency.

This dilemma is not specific to Kafka. It exists in any quorum-based scheme. For example in a majority voting scheme, if a majority of servers suffer a permanent failure, then you must either choose to lose 100% of your data or violate consistency by taking what remains on an existing server as your new source of truth.

Availability and Durability Guarantees

When writing to Kafka, producers can choose whether they wait for the message to be acknowledged by 0,1 or all (-1) replicas. Note that “acknowledgement by all replicas” does not guarantee that the full set of assigned replicas have received the message. By default, when acks=all, acknowledgement happens as soon as all the current in-sync replicas have received the message. For example, if a topic is configured with only two replicas and one fails (i.e., only one in sync replica remains), then writes that specify acks=all will succeed. However, these writes could be lost if the remaining replica also fails. Although this ensures maximum availability of the partition, this behavior may be undesirable to some users who prefer durability over availability. Therefore, we provide two topic-level configurations that can be used to prefer message durability over availability:

  1. Disable unclean leader election - if all replicas become unavailable, then the partition will remain unavailable until the most recent leader becomes available again. This effectively prefers unavailability over the risk of message loss. See the previous section on Unclean Leader Election for clarification.
  2. Specify a minimum ISR size - the partition will only accept writes if the size of the ISR is above a certain minimum, in order to prevent the loss of messages that were written to just a single replica, which subsequently becomes unavailable. This setting only takes effect if the producer uses acks=all and guarantees that the message will be acknowledged by at least this many in-sync replicas. This setting offers a trade-off between consistency and availability. A higher setting for minimum ISR size guarantees better consistency since the message is guaranteed to be written to more replicas which reduces the probability that it will be lost. However, it reduces availability since the partition will be unavailable for writes if the number of in-sync replicas drops below the minimum threshold.

Replica Management

The above discussion on replicated logs really covers only a single log, i.e. one topic partition. However a Kafka cluster will manage hundreds or thousands of these partitions. We attempt to balance partitions within a cluster in a round-robin fashion to avoid clustering all partitions for high-volume topics on a small number of nodes. Likewise we try to balance leadership so that each node is the leader for a proportional share of its partitions.

It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all partitions a node hosted when that node failed. Instead, we elect one of the brokers as the “controller”. This controller detects failures at the broker level and is responsible for changing the leader of all affected partitions in a failed broker. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number of partitions. If the controller fails, one of the surviving brokers will become the new controller.

Log Compaction

Log compaction ensures that Kafka will always retain at least the last known value for each message key within the log of data for a single topic partition. It addresses use cases and scenarios such as restoring state after application crashes or system failure, or reloading caches after application restarts during operational maintenance. Let’s dive into these use cases in more detail and then describe how compaction works.

So far we have described only the simpler approach to data retention where old log data is discarded after a fixed period of time or when the log reaches some predetermined size. This works well for temporal event data such as logging where each record stands alone. However an important class of data streams are the log of changes to keyed, mutable data (for example, the changes to a database table).

Let’s discuss a concrete example of such a stream. Say we have a topic containing user email addresses; every time a user updates their email address we send a message to this topic using their user id as the primary key. Now say we send the following messages over some time period for a user with id 123, each message corresponding to a change in email address (messages for other ids are omitted):

        123 => bill@microsoft.com
                .
                .
                .
        123 => bill@gatesfoundation.org
                .
                .
                .
        123 => bill@gmail.com

Log compaction gives us a more granular retention mechanism so that we are guaranteed to retain at least the last update for each primary key (e.g. bill@gmail.com). By doing this we guarantee that the log contains a full snapshot of the final value for every key not just keys that changed recently. This means downstream consumers can restore their own state off this topic without us having to retain a complete log of all changes.

Let’s start by looking at a few use cases where this is useful, then we’ll see how it can be used.

  1. Database change subscription. It is often necessary to have a data set in multiple data systems, and often one of these systems is a database of some kind (either a RDBMS or perhaps a new-fangled key-value store). For example you might have a database, a cache, a search cluster, and a Hadoop cluster. Each change to the database will need to be reflected in the cache, the search cluster, and eventually in Hadoop. In the case that one is only handling the real-time updates you only need recent log. But if you want to be able to reload the cache or restore a failed search node you may need a complete data set.
  2. Event sourcing. This is a style of application design which co-locates query processing with application design and uses a log of changes as the primary store for the application.
  3. Journaling for high-availability. A process that does local computation can be made fault-tolerant by logging out changes that it makes to its local state so another process can reload these changes and carry on if it should fail. A concrete example of this is handling counts, aggregations, and other “group by”-like processing in a stream query system. Samza, a real-time stream-processing framework, uses this feature for exactly this purpose. In each of these cases one needs primarily to handle the real-time feed of changes, but occasionally, when a machine crashes or data needs to be re-loaded or re-processed, one needs to do a full load. Log compaction allows feeding both of these use cases off the same backing topic. This style of usage of a log is described in more detail in this blog post.

The general idea is quite simple. If we had infinite log retention, and we logged each change in the above cases, then we would have captured the state of the system at each time from when it first began. Using this complete log, we could restore to any point in time by replaying the first N records in the log. This hypothetical complete log is not very practical for systems that update a single record many times as the log will grow without bound even for a stable dataset. The simple log retention mechanism which throws away old updates will bound space but the log is no longer a way to restore the current state–now restoring from the beginning of the log no longer recreates the current state as old updates may not be captured at all.

Log compaction is a mechanism to give finer-grained per-record retention, rather than the coarser-grained time-based retention. The idea is to selectively remove records where we have a more recent update with the same primary key. This way the log is guaranteed to have at least the last state for each key.

This retention policy can be set per-topic, so a single cluster can have some topics where retention is enforced by size or time and other topics where retention is enforced by compaction.

This functionality is inspired by one of LinkedIn’s oldest and most successful pieces of infrastructure–a database changelog caching service called Databus. Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts as a source-of-truth store so it is useful even in situations where the upstream data source would not otherwise be replayable.

Log Compaction Basics

Here is a high-level picture that shows the logical structure of a Kafka log with the offset for each message.

The head of the log is identical to a traditional Kafka log. It has dense, sequential offsets and retains all messages. Log compaction adds an option for handling the tail of the log. The picture above shows a log with a compacted tail. Note that the messages in the tail of the log retain the original offset assigned when they were first written–that never changes. Note also that all offsets remain valid positions in the log, even if the message with that offset has been compacted away; in this case this position is indistinguishable from the next highest offset that does appear in the log. For example, in the picture above the offsets 36, 37, and 38 are all equivalent positions and a read beginning at any of these offsets would return a message set beginning with 38.

Compaction also allows for deletes. A message with a key and a null payload will be treated as a delete from the log. This delete marker will cause any prior message with that key to be removed (as would any new message with that key), but delete markers are special in that they will themselves be cleaned out of the log after a period of time to free up space. The point in time at which deletes are no longer retained is marked as the “delete retention point” in the above diagram.

The compaction is done in the background by periodically recopying log segments. Cleaning does not block reads and can be throttled to use no more than a configurable amount of I/O throughput to avoid impacting producers and consumers. The actual process of compacting a log segment looks something like this:

What guarantees does log compaction provide?

Log compaction guarantees the following:

  1. Any consumer that stays caught-up to within the head of the log will see every message that is written; these messages will have sequential offsets. The topic’s min.compaction.lag.ms can be used to guarantee the minimum length of time must pass after a message is written before it could be compacted. I.e. it provides a lower bound on how long each message will remain in the (uncompacted) head.
  2. Ordering of messages is always maintained. Compaction will never re-order messages, just remove some.
  3. The offset for a message never changes. It is the permanent identifier for a position in the log.
  4. Any consumer progressing from the start of the log will see at least the final state of all records in the order they were written. Additionally, all delete markers for deleted records will be seen, provided the consumer reaches the head of the log in a time period less than the topic’s delete.retention.ms setting (the default is 24 hours). In other words: since the removal of delete markers happens concurrently with reads, it is possible for a consumer to miss delete markers if it lags by more than delete.retention.ms.

Log Compaction Details

Log compaction is handled by the log cleaner, a pool of background threads that recopy log segment files, removing records whose key appears in the head of the log. Each compactor thread works as follows:

  1. It chooses the log that has the highest ratio of log head to log tail
  2. It creates a succinct summary of the last offset for each key in the head of the log
  3. It recopies the log from beginning to end removing keys which have a later occurrence in the log. New, clean segments are swapped into the log immediately so the additional disk space required is just one additional log segment (not a fully copy of the log).
  4. The summary of the log head is essentially just a space-compact hash table. It uses exactly 24 bytes per entry. As a result with 8GB of cleaner buffer one cleaner iteration can clean around 366GB of log head (assuming 1k messages).

Configuring The Log Cleaner

The log cleaner is enabled by default. This will start the pool of cleaner threads. To enable log cleaning on a particular topic you can add the log-specific property

  log.cleanup.policy=compact

This can be done either at topic creation time or using the alter topic command.

The log cleaner can be configured to retain a minimum amount of the uncompacted “head” of the log. This is enabled by setting the compaction time lag.

  log.cleaner.min.compaction.lag.ms

This can be used to prevent messages newer than a minimum message age from being subject to compaction. If not set, all log segments are eligible for compaction except for the last segment, i.e. the one currently being written to. The active segment will not be compacted even if all of its messages are older than the minimum compaction time lag.

Further cleaner configurations are described here.

Quotas

Kafka cluster has the ability to enforce quotas on requests to control the broker resources used by clients. Two types of client quotas can be enforced by Kafka brokers for each group of clients sharing a quota:

  1. Network bandwidth quotas define byte-rate thresholds (since 0.9)
  2. Request rate quotas define CPU utilization thresholds as a percentage of network and I/O threads (since 0.11)

Why are quotas necessary?

It is possible for producers and consumers to produce/consume very high volumes of data or generate requests at a very high rate and thus monopolize broker resources, cause network saturation and generally DOS other clients and the brokers themselves. Having quotas protects against these issues and is all the more important in large multi-tenant clusters where a small set of badly behaved clients can degrade user experience for the well behaved ones. In fact, when running Kafka as a service this even makes it possible to enforce API limits according to an agreed upon contract.

Client groups

The identity of Kafka clients is the user principal which represents an authenticated user in a secure cluster. In a cluster that supports unauthenticated clients, user principal is a grouping of unauthenticated users chosen by the broker using a configurable PrincipalBuilder. Client-id is a logical grouping of clients with a meaningful name chosen by the client application. The tuple (user, client-id) defines a secure logical group of clients that share both user principal and client-id.

Quotas can be applied to (user, client-id), user or client-id groups. For a given connection, the most specific quota matching the connection is applied. All connections of a quota group share the quota configured for the group. For example, if (user=“test-user”, client-id=“test-client”) has a produce quota of 10MB/sec, this is shared across all producer instances of user “test-user” with the client-id “test-client”.

Quota Configuration

Quota configuration may be defined for (user, client-id), user and client-id groups. It is possible to override the default quota at any of the quota levels that needs a higher (or even lower) quota. The mechanism is similar to the per-topic log config overrides. User and (user, client-id) quota overrides are written to ZooKeeper under /config/users and client-id quota overrides are written under /config/clients. These overrides are read by all brokers and are effective immediately. This lets us change quotas without having to do a rolling restart of the entire cluster. See here for details. Default quotas for each group may also be updated dynamically using the same mechanism.

The order of precedence for quota configuration is:

  1. /config/users//clients/
  2. /config/users//clients/
  3. /config/users/
  4. /config/users//clients/
  5. /config/users//clients/
  6. /config/users/
  7. /config/clients/
  8. /config/clients/

Broker properties (quota.producer.default, quota.consumer.default) can also be used to set defaults of network bandwidth quotas for client-id groups. These properties are being deprecated and will be removed in a later release. Default quotas for client-id can be set in Zookeeper similar to the other quota overrides and defaults.

Network Bandwidth Quotas

Network bandwidth quotas are defined as the byte rate threshold for each group of clients sharing a quota. By default, each unique client group receives a fixed quota in bytes/sec as configured by the cluster. This quota is defined on a per-broker basis. Each group of clients can publish/fetch a maximum of X bytes/sec per broker before clients are throttled.

Request Rate Quotas

Request rate quotas are defined as the percentage of time a client can utilize on request handler I/O threads and network threads of each broker within a quota window. A quota of n% represents n% of one thread, so the quota is out of a total capacity of ((num.io.threads + num.network.threads) * 100)%. Each group of clients may use a total percentage of upto n% across all I/O and network threads in a quota window before being throttled. Since the number of threads allocated for I/O and network threads are typically based on the number of cores available on the broker host, request rate quotas represent the total percentage of CPU that may be used by each group of clients sharing the quota.

Enforcement

By default, each unique client group receives a fixed quota as configured by the cluster. This quota is defined on a per-broker basis. Each client can utilize this quota per broker before it gets throttled. We decided that defining these quotas per broker is much better than having a fixed cluster wide bandwidth per client because that would require a mechanism to share client quota usage among all the brokers. This can be harder to get right than the quota implementation itself!

How does a broker react when it detects a quota violation? In our solution, the broker first computes the amount of delay needed to bring the violating client under its quota and returns a response with the delay immediately. In case of a fetch request, the response will not contain any data. Then, the broker mutes the channel to the client, not to process requests from the client anymore, until the delay is over. Upon receiving a response with a non-zero delay duration, the Kafka client will also refrain from sending further requests to the broker during the delay. Therefore, requests from a throttled client are effectively blocked from both sides. Even with older client implementations that do not respect the delay response from the broker, the back pressure applied by the broker via muting its socket channel can still handle the throttling of badly behaving clients. Those clients who sent further requests to the throttled channel will receive responses only after the delay is over.

Byte-rate and thread utilization are measured over multiple small windows (e.g. 30 windows of 1 second each) in order to detect and correct quota violations quickly. Typically, having large measurement windows (for e.g. 10 windows of 30 seconds each) leads to large bursts of traffic followed by long delays which is not great in terms of user experience.

2 - Protocol

Kafka protocol guide

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 here

  • Preliminaries
    • Network
    • Partitioning and bootstrapping
    • Partitioning Strategies
    • Batching
    • Versioning and Compatibility
  • The Protocol
    • Protocol Primitive Types
    • Notes on reading the request format grammars
    • Common Request and Response Structure
    • Record Batch
  • Constants
    • Error Codes
    • Api Keys
  • The Messages
  • Some Common Philosophical Questions

Preliminaries

Network

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.

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.

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).

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.

The server has a configurable maximum limit on request size and any request that exceeds this limit will result in the socket being disconnected.

Partitioning and bootstrapping

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.

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.

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).

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.

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.

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.

  1. Cycle through a list of “bootstrap” Kafka URLs until we find one we can connect to. Fetch cluster metadata.
  2. Process fetch or produce requests, directing them to the appropriate broker based on the topic/partitions they send to or fetch from.
  3. If we get an appropriate error, refresh the metadata and try again.

Partitioning Strategies

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?

Partitioning really serves two purposes in Kafka:

  1. It balances data and request load over brokers
  2. 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.

For a given use case you may care about only one of these or both.

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.

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.

Batching

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.

The client implementer can choose to ignore this and send everything one at a time if they like.

Versioning and Compatibility

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.

The intention is that clients will support a range of API versions. When communicating with a particular broker, a given client should use the highest API version supported by both and indicate this version in their requests.

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.

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.

Currently all versions are baselined at 0, as we evolve these APIs we will indicate the format for each version individually.

Retrieving Supported API versions

In order to work against multiple broker versions, clients need to know what versions of various APIs a broker supports. The broker exposes this information since 0.10.0.0 as described in KIP-35. Clients should use the supported API versions information to choose the highest API version supported by both client and broker. If no such version exists, an error should be reported to the user.

The following sequence may be used by a client to obtain supported API versions from a broker.

  1. Client sends ApiVersionsRequest to a broker after connection has been established with the broker. If SSL is enabled, this happens after SSL connection has been established.
  2. On receiving ApiVersionsRequest, a broker returns its full list of supported ApiKeys and versions regardless of current authentication state (e.g., before SASL authentication on an SASL listener, do note that no Kafka protocol requests may take place on a SSL listener before the SSL handshake is finished). If this is considered to leak information about the broker version a workaround is to use SSL with client authentication which is performed at an earlier stage of the connection where the ApiVersionRequest is not available. Also, note that broker versions older than 0.10.0.0 do not support this API and will either ignore the request or close connection in response to the request.
  3. If multiple versions of an API are supported by broker and client, clients are recommended to use the latest version supported by the broker and itself.
  4. Deprecation of a protocol version is done by marking an API version as deprecated in the protocol documentation.
  5. Supported API versions obtained from a broker are only valid for the connection on which that information is obtained. In the event of disconnection, the client should obtain the information from the broker again, as the broker might have been upgraded/downgraded in the mean time.

SASL Authentication Sequence

The following sequence is used for SASL authentication:

  1. Kafka ApiVersionsRequest may be sent by the client to obtain the version ranges of requests supported by the broker. This is optional.
  2. Kafka SaslHandshakeRequest containing the SASL mechanism for authentication is sent by the client. If the requested mechanism is not enabled in the server, the server responds with the list of supported mechanisms and closes the client connection. If the mechanism is enabled in the server, the server sends a successful response and continues with SASL authentication.
  3. The actual SASL authentication is now performed. If SaslHandshakeRequest version is v0, a series of SASL client and server tokens corresponding to the mechanism are sent as opaque packets without wrapping the messages with Kafka protocol headers. If SaslHandshakeRequest version is v1, the SaslAuthenticate request/response are used, where the actual SASL tokens are wrapped in the Kafka protocol. The error code in the final message from the broker will indicate if authentication succeeded or failed.
  4. If authentication succeeds, subsequent packets are handled as Kafka API requests. Otherwise, the client connection is closed.

For interoperability with 0.9.0.x clients, the first packet received by the server is handled as a SASL/GSSAPI client token if it is not a valid Kafka request. SASL/GSSAPI authentication is performed starting with this packet, skipping the first two steps above.

The Protocol

Protocol Primitive Types

The protocol is built out of the following primitive types.

TypeDescription
BOOLEANRepresents 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.
INT8Represents an integer between -27 and 27-1 inclusive.
INT16Represents an integer between -215 and 215-1 inclusive. The values are encoded using two bytes in network byte order (big-endian).
INT32Represents an integer between -231 and 231-1 inclusive. The values are encoded using four bytes in network byte order (big-endian).
INT64Represents an integer between -263 and 263-1 inclusive. The values are encoded using eight bytes in network byte order (big-endian).
UINT32Represents an integer between 0 and 232-1 inclusive. The values are encoded using four bytes in network byte order (big-endian).
VARINTRepresents an integer between -231 and 231-1 inclusive. Encoding follows the variable-length zig-zag encoding from Google Protocol Buffers.
VARLONGRepresents an integer between -263 and 263-1 inclusive. Encoding follows the variable-length zig-zag encoding from Google Protocol Buffers.
STRINGRepresents 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.
NULLABLE_STRINGRepresents 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.
BYTESRepresents a raw sequence of bytes. First the length N is given as an INT32. Then N bytes follow.
NULLABLE_BYTESRepresents 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.
RECORDSRepresents a sequence of Kafka records as NULLABLE_BYTES. For a detailed description of records see Message Sets.
ARRAYRepresents 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].

Notes on reading the request format grammars

The BNFs 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.

Common Request and Response Structure

All requests and responses originate from the following grammar which will be incrementally describe through the rest of this document:

RequestOrResponse => Size (RequestMessage | ResponseMessage)
  Size => int32
FieldDescription
message_sizeThe 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.

Record Batch

A description of the record batch format can be found here.

Constants

Error Codes

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:

ErrorCodeRetriableDescription
UNKNOWN_SERVER_ERROR-1FalseThe server experienced an unexpected error when processing the request.
NONE0False
OFFSET_OUT_OF_RANGE1FalseThe requested offset is not within the range of offsets maintained by the server.
CORRUPT_MESSAGE2TrueThis message has failed its CRC checksum, exceeds the valid size, has a null key for a compacted topic, or is otherwise corrupt.
UNKNOWN_TOPIC_OR_PARTITION3TrueThis server does not host this topic-partition.
INVALID_FETCH_SIZE4FalseThe requested fetch size is invalid.
LEADER_NOT_AVAILABLE5TrueThere is no leader for this topic-partition as we are in the middle of a leadership election.
NOT_LEADER_FOR_PARTITION6TrueThis server is not the leader for that topic-partition.
REQUEST_TIMED_OUT7TrueThe request timed out.
BROKER_NOT_AVAILABLE8FalseThe broker is not available.
REPLICA_NOT_AVAILABLE9FalseThe replica is not available for the requested topic-partition.
MESSAGE_TOO_LARGE10FalseThe request included a message larger than the max message size the server will accept.
STALE_CONTROLLER_EPOCH11FalseThe controller moved to another broker.
OFFSET_METADATA_TOO_LARGE12FalseThe metadata field of the offset request was too large.
NETWORK_EXCEPTION13TrueThe server disconnected before a response was received.
COORDINATOR_LOAD_IN_PROGRESS14TrueThe coordinator is loading and hence can't process requests.
COORDINATOR_NOT_AVAILABLE15TrueThe coordinator is not available.
NOT_COORDINATOR16TrueThis is not the correct coordinator.
INVALID_TOPIC_EXCEPTION17FalseThe request attempted to perform an operation on an invalid topic.
RECORD_LIST_TOO_LARGE18FalseThe request included message batch larger than the configured segment size on the server.
NOT_ENOUGH_REPLICAS19TrueMessages are rejected since there are fewer in-sync replicas than required.
NOT_ENOUGH_REPLICAS_AFTER_APPEND20TrueMessages are written to the log, but to fewer in-sync replicas than required.
INVALID_REQUIRED_ACKS21FalseProduce request specified an invalid value for required acks.
ILLEGAL_GENERATION22FalseSpecified group generation id is not valid.
INCONSISTENT_GROUP_PROTOCOL23FalseThe group member's supported protocols are incompatible with those of existing members or first group member tried to join with empty protocol type or empty protocol list.
INVALID_GROUP_ID24FalseThe configured groupId is invalid.
UNKNOWN_MEMBER_ID25FalseThe coordinator is not aware of this member.
INVALID_SESSION_TIMEOUT26FalseThe session timeout is not within the range allowed by the broker (as configured by group.min.session.timeout.ms and group.max.session.timeout.ms).
REBALANCE_IN_PROGRESS27FalseThe group is rebalancing, so a rejoin is needed.
INVALID_COMMIT_OFFSET_SIZE28FalseThe committing offset data size is not valid.
TOPIC_AUTHORIZATION_FAILED29FalseNot authorized to access topics: [Topic authorization failed.]
GROUP_AUTHORIZATION_FAILED30FalseNot authorized to access group: Group authorization failed.
CLUSTER_AUTHORIZATION_FAILED31FalseCluster authorization failed.
INVALID_TIMESTAMP32FalseThe timestamp of the message is out of acceptable range.
UNSUPPORTED_SASL_MECHANISM33FalseThe broker does not support the requested SASL mechanism.
ILLEGAL_SASL_STATE34FalseRequest is not valid given the current SASL state.
UNSUPPORTED_VERSION35FalseThe version of API is not supported.
TOPIC_ALREADY_EXISTS36FalseTopic with this name already exists.
INVALID_PARTITIONS37FalseNumber of partitions is below 1.
INVALID_REPLICATION_FACTOR38FalseReplication factor is below 1 or larger than the number of available brokers.
INVALID_REPLICA_ASSIGNMENT39FalseReplica assignment is invalid.
INVALID_CONFIG40FalseConfiguration is invalid.
NOT_CONTROLLER41TrueThis is not the correct controller for this cluster.
INVALID_REQUEST42FalseThis most likely occurs because of a request being malformed by the client library or the message was sent to an incompatible broker. See the broker logs for more details.
UNSUPPORTED_FOR_MESSAGE_FORMAT43FalseThe message format version on the broker does not support the request.
POLICY_VIOLATION44FalseRequest parameters do not satisfy the configured policy.
OUT_OF_ORDER_SEQUENCE_NUMBER45FalseThe broker received an out of order sequence number.
DUPLICATE_SEQUENCE_NUMBER46FalseThe broker received a duplicate sequence number.
INVALID_PRODUCER_EPOCH47FalseProducer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker.
INVALID_TXN_STATE48FalseThe producer attempted a transactional operation in an invalid state.
INVALID_PRODUCER_ID_MAPPING49FalseThe producer attempted to use a producer id which is not currently assigned to its transactional id.
INVALID_TRANSACTION_TIMEOUT50FalseThe transaction timeout is larger than the maximum value allowed by the broker (as configured by transaction.max.timeout.ms).
CONCURRENT_TRANSACTIONS51FalseThe producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing.
TRANSACTION_COORDINATOR_FENCED52FalseIndicates that the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer.
TRANSACTIONAL_ID_AUTHORIZATION_FAILED53FalseTransactional Id authorization failed.
SECURITY_DISABLED54FalseSecurity features are disabled.
OPERATION_NOT_ATTEMPTED55FalseThe broker did not attempt to execute this operation. This may happen for batched RPCs where some operations in the batch failed, causing the broker to respond without trying the rest.
KAFKA_STORAGE_ERROR56TrueDisk error when trying to access log file on the disk.
LOG_DIR_NOT_FOUND57FalseThe user-specified log directory is not found in the broker config.
SASL_AUTHENTICATION_FAILED58FalseSASL Authentication failed.
UNKNOWN_PRODUCER_ID59FalseThis exception is raised by the broker if it could not locate the producer metadata associated with the producerId in question. This could happen if, for instance, the producer's records were deleted because their retention time had elapsed. Once the last records of the producerId are removed, the producer's metadata is removed from the broker, and future appends by the producer will return this exception.
REASSIGNMENT_IN_PROGRESS60FalseA partition reassignment is in progress.
DELEGATION_TOKEN_AUTH_DISABLED61FalseDelegation Token feature is not enabled.
DELEGATION_TOKEN_NOT_FOUND62FalseDelegation Token is not found on server.
DELEGATION_TOKEN_OWNER_MISMATCH63FalseSpecified Principal is not valid Owner/Renewer.
DELEGATION_TOKEN_REQUEST_NOT_ALLOWED64FalseDelegation Token requests are not allowed on PLAINTEXT/1-way SSL channels and on delegation token authenticated channels.
DELEGATION_TOKEN_AUTHORIZATION_FAILED65FalseDelegation Token authorization failed.
DELEGATION_TOKEN_EXPIRED66FalseDelegation Token is expired.
INVALID_PRINCIPAL_TYPE67FalseSupplied principalType is not supported.
NON_EMPTY_GROUP68FalseThe group is not empty.
GROUP_ID_NOT_FOUND69FalseThe group id does not exist.
FETCH_SESSION_ID_NOT_FOUND70TrueThe fetch session ID was not found.
INVALID_FETCH_SESSION_EPOCH71TrueThe fetch session epoch is invalid.
LISTENER_NOT_FOUND72TrueThere is no listener on the leader broker that matches the listener on which metadata request was processed.
TOPIC_DELETION_DISABLED73FalseTopic deletion is disabled.
FENCED_LEADER_EPOCH74TrueThe leader epoch in the request is older than the epoch on the broker
UNKNOWN_LEADER_EPOCH75TrueThe leader epoch in the request is newer than the epoch on the broker
UNSUPPORTED_COMPRESSION_TYPE76FalseThe requesting client does not support the compression type of given partition.
STALE_BROKER_EPOCH77FalseBroker epoch has changed
OFFSET_NOT_AVAILABLE78TrueThe leader high watermark has not caught up from a recent leader election so the offsets cannot be guaranteed to be monotonically increasing
MEMBER_ID_REQUIRED79FalseThe group member needs to have a valid member id before actually entering a consumer group
PREFERRED_LEADER_NOT_AVAILABLE80TrueThe preferred leader was not available
GROUP_MAX_SIZE_REACHED81FalseConsumer group The consumer group has reached its max size. already has the configured maximum number of members.

Api Keys

The following are the numeric codes that the ApiKey in the request can take for each of the below request types.

NameKey
Produce0
Fetch1
ListOffsets2
Metadata3
LeaderAndIsr4
StopReplica5
UpdateMetadata6
ControlledShutdown7
OffsetCommit8
OffsetFetch9
FindCoordinator10
JoinGroup11
Heartbeat12
LeaveGroup13
SyncGroup14
DescribeGroups15
ListGroups16
SaslHandshake17
ApiVersions18
CreateTopics19
DeleteTopics20
DeleteRecords21
InitProducerId22
OffsetForLeaderEpoch23
AddPartitionsToTxn24
AddOffsetsToTxn25
EndTxn26
WriteTxnMarkers27
TxnOffsetCommit28
DescribeAcls29
CreateAcls30
DeleteAcls31
DescribeConfigs32
AlterConfigs33
AlterReplicaLogDirs34
DescribeLogDirs35
SaslAuthenticate36
CreatePartitions37
CreateDelegationToken38
RenewDelegationToken39
ExpireDelegationToken40
DescribeDelegationToken41
DeleteGroups42
ElectPreferredLeaders43

The Messages

This section gives details on each of the individual API Messages, their usage, their binary format, and the meaning of their fields.

Headers:
Request Header => api_key api_version correlation_id client_id 
  api_key => INT16
  api_version => INT16
  correlation_id => INT32
  client_id => NULLABLE_STRING
FieldDescription
api_keyThe id of the request type.
api_versionThe version of the API.
correlation_idA user-supplied integer value that will be passed back with the response
client_idA user specified identifier for the client making the request.
Response Header => correlation_id 
  correlation_id => INT32
FieldDescription
correlation_idThe user-supplied value passed in with the request
Produce API (Key: 0):
Requests:

Produce Request (Version: 0) => acks timeout [topic_data] 
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 1) => acks timeout [topic_data] 
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 2) => acks timeout [topic_data] 
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 3) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 4) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 5) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 6) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Produce Request (Version: 7) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
acksThe number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.
timeoutThe time to await a response in ms.
topic_datanull
topicName of topic
datanull
partitionTopic partition id
record_setnull

Responses:

Produce Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull

Produce Response (Version: 1) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Produce Response (Version: 2) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
log_append_timeThe timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Produce Response (Version: 3) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
log_append_timeThe timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Produce Response (Version: 4) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
log_append_timeThe timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Produce Response (Version: 5) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
log_append_timeThe timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.
log_start_offsetThe start offset of the log at the time this produce response was created
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Produce Response (Version: 6) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
log_append_timeThe timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.
log_start_offsetThe start offset of the log at the time this produce response was created
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Produce Response (Version: 7) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
base_offsetnull
log_append_timeThe timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.
log_start_offsetThe start offset of the log at the time this produce response was created
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

Fetch API (Key: 1):
Requests:

Fetch Request (Version: 0) => replica_id max_wait_time min_bytes [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 1) => replica_id max_wait_time min_bytes [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 2) => replica_id max_wait_time min_bytes [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 3) => replica_id max_wait_time min_bytes max_bytes [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 4) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 5) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
log_start_offsetEarliest available offset of the follower replica. The field is only used when request is sent by follower.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 6) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
log_start_offsetEarliest available offset of the follower replica. The field is only used when request is sent by follower.
partition_max_bytesMaximum bytes to fetch.

Fetch Request (Version: 7) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
session_idThe fetch session ID
session_epochThe fetch session epoch
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
log_start_offsetEarliest available offset of the follower replica. The field is only used when request is sent by follower.
partition_max_bytesMaximum bytes to fetch.
forgotten_topics_dataTopics to remove from the fetch session.
topicName of topic
partitionsPartitions to remove from the fetch session.

Fetch Request (Version: 8) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
session_idThe fetch session ID
session_epochThe fetch session epoch
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
fetch_offsetMessage offset.
log_start_offsetEarliest available offset of the follower replica. The field is only used when request is sent by follower.
partition_max_bytesMaximum bytes to fetch.
forgotten_topics_dataTopics to remove from the fetch session.
topicName of topic
partitionsPartitions to remove from the fetch session.

Fetch Request (Version: 9) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
session_idThe fetch session ID
session_epochThe fetch session epoch
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
current_leader_epochThe current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
fetch_offsetMessage offset.
log_start_offsetEarliest available offset of the follower replica. The field is only used when request is sent by follower.
partition_max_bytesMaximum bytes to fetch.
forgotten_topics_dataTopics to remove from the fetch session.
topicName of topic
partitionsPartitions to remove from the fetch session.

Fetch Request (Version: 10) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_time => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
max_wait_timeMaximum time in ms to wait for the response.
min_bytesMinimum bytes to accumulate in the response.
max_bytesMaximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
session_idThe fetch session ID
session_epochThe fetch session epoch
topicsTopics to fetch in the order provided.
topicName of topic
partitionsPartitions to fetch.
partitionTopic partition id
current_leader_epochThe current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
fetch_offsetMessage offset.
log_start_offsetEarliest available offset of the follower replica. The field is only used when request is sent by follower.
partition_max_bytesMaximum bytes to fetch.
forgotten_topics_dataTopics to remove from the fetch session.
topicName of topic
partitionsPartitions to remove from the fetch session.

Responses:

Fetch Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
      record_set => RECORDS
FieldDescription
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
record_setnull

Fetch Response (Version: 1) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
record_setnull

Fetch Response (Version: 2) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
record_setnull

Fetch Response (Version: 3) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
record_setnull

Fetch Response (Version: 4) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

Fetch Response (Version: 5) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        log_start_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
log_start_offsetEarliest available offset.
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

Fetch Response (Version: 6) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        log_start_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
log_start_offsetEarliest available offset.
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

Fetch Response (Version: 7) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        log_start_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
session_idThe fetch session ID
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
log_start_offsetEarliest available offset.
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

Fetch Response (Version: 8) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        log_start_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
session_idThe fetch session ID
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
log_start_offsetEarliest available offset.
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

Fetch Response (Version: 9) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        log_start_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
session_idThe fetch session ID
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
log_start_offsetEarliest available offset.
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

Fetch Response (Version: 10) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition_header record_set 
      partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] 
        partition => INT32
        error_code => INT16
        high_watermark => INT64
        last_stable_offset => INT64
        log_start_offset => INT64
        aborted_transactions => producer_id first_offset 
          producer_id => INT64
          first_offset => INT64
      record_set => RECORDS
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
session_idThe fetch session ID
responsesnull
topicName of topic
partition_responsesnull
partition_headernull
partitionTopic partition id
error_codeResponse error code
high_watermarkLast committed offset.
last_stable_offsetThe last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)
log_start_offsetEarliest available offset.
aborted_transactionsnull
producer_idThe producer id associated with the aborted transactions
first_offsetThe first offset in the aborted transaction
record_setnull

ListOffsets API (Key: 2):
Requests:

ListOffsets Request (Version: 0) => replica_id [topics] 
  replica_id => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition timestamp max_num_offsets 
      partition => INT32
      timestamp => INT64
      max_num_offsets => INT32
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
topicsTopics to list offsets.
topicName of topic
partitionsPartitions to list offsets.
partitionTopic partition id
timestampThe target timestamp for the partition.
max_num_offsetsMaximum offsets to return.

ListOffsets Request (Version: 1) => replica_id [topics] 
  replica_id => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition timestamp 
      partition => INT32
      timestamp => INT64
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
topicsTopics to list offsets.
topicName of topic
partitionsPartitions to list offsets.
partitionTopic partition id
timestampThe target timestamp for the partition.

ListOffsets Request (Version: 2) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition timestamp 
      partition => INT32
      timestamp => INT64
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to list offsets.
topicName of topic
partitionsPartitions to list offsets.
partitionTopic partition id
timestampThe target timestamp for the partition.

ListOffsets Request (Version: 3) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition timestamp 
      partition => INT32
      timestamp => INT64
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to list offsets.
topicName of topic
partitionsPartitions to list offsets.
partitionTopic partition id
timestampThe target timestamp for the partition.

ListOffsets Request (Version: 4) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch timestamp 
      partition => INT32
      current_leader_epoch => INT32
      timestamp => INT64
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to list offsets.
topicName of topic
partitionsPartitions to list offsets.
partitionTopic partition id
current_leader_epochThe current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
timestampThe target timestamp for the partition.

ListOffsets Request (Version: 5) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch timestamp 
      partition => INT32
      current_leader_epoch => INT32
      timestamp => INT64
FieldDescription
replica_idBroker id of the follower. For normal consumers, use -1.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records
topicsTopics to list offsets.
topicName of topic
partitionsPartitions to list offsets.
partitionTopic partition id
current_leader_epochThe current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
timestampThe target timestamp for the partition.

Responses:

ListOffsets Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code [offsets'] 
      partition => INT32
      error_code => INT16
      offsets' => INT64
FieldDescription
responsesThe listed offsets by topic
topicName of topic
partition_responsesThe listed offsets by partition
partitionTopic partition id
error_codeResponse error code
offsets'A list of offsets.

ListOffsets Response (Version: 1) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code timestamp offset 
      partition => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
FieldDescription
responsesThe listed offsets by topic
topicName of topic
partition_responsesThe listed offsets by partition
partitionTopic partition id
error_codeResponse error code
timestampThe timestamp associated with the returned offset
offsetThe offset found

ListOffsets Response (Version: 2) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code timestamp offset 
      partition => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesThe listed offsets by topic
topicName of topic
partition_responsesThe listed offsets by partition
partitionTopic partition id
error_codeResponse error code
timestampThe timestamp associated with the returned offset
offsetThe offset found

ListOffsets Response (Version: 3) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code timestamp offset 
      partition => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesThe listed offsets by topic
topicName of topic
partition_responsesThe listed offsets by partition
partitionTopic partition id
error_codeResponse error code
timestampThe timestamp associated with the returned offset
offsetThe offset found

ListOffsets Response (Version: 4) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code timestamp offset leader_epoch 
      partition => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesThe listed offsets by topic
topicName of topic
partition_responsesThe listed offsets by partition
partitionTopic partition id
error_codeResponse error code
timestampThe timestamp associated with the returned offset
offsetThe offset found
leader_epochThe leader epoch

ListOffsets Response (Version: 5) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code timestamp offset leader_epoch 
      partition => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesThe listed offsets by topic
topicName of topic
partition_responsesThe listed offsets by partition
partitionTopic partition id
error_codeResponse error code
timestampThe timestamp associated with the returned offset
offsetThe offset found
leader_epochThe leader epoch

Metadata API (Key: 3):
Requests:

Metadata Request (Version: 0) => [topics] 
  topics => STRING
FieldDescription
topicsAn array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics.

Metadata Request (Version: 1) => [topics] 
  topics => STRING
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.

Metadata Request (Version: 2) => [topics] 
  topics => STRING
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.

Metadata Request (Version: 3) => [topics] 
  topics => STRING
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.

Metadata Request (Version: 4) => [topics] allow_auto_topic_creation 
  topics => STRING
  allow_auto_topic_creation => BOOLEAN
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.
allow_auto_topic_creationIf this and the broker config auto.create.topics.enable are true, topics that don't exist will be created by the broker. Otherwise, no topics will be created by the broker.

Metadata Request (Version: 5) => [topics] allow_auto_topic_creation 
  topics => STRING
  allow_auto_topic_creation => BOOLEAN
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.
allow_auto_topic_creationIf this and the broker config auto.create.topics.enable are true, topics that don't exist will be created by the broker. Otherwise, no topics will be created by the broker.

Metadata Request (Version: 6) => [topics] allow_auto_topic_creation 
  topics => STRING
  allow_auto_topic_creation => BOOLEAN
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.
allow_auto_topic_creationIf this and the broker config auto.create.topics.enable are true, topics that don't exist will be created by the broker. Otherwise, no topics will be created by the broker.

Metadata Request (Version: 7) => [topics] allow_auto_topic_creation 
  topics => STRING
  allow_auto_topic_creation => BOOLEAN
FieldDescription
topicsAn array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.
allow_auto_topic_creationIf this and the broker config auto.create.topics.enable are true, topics that don't exist will be created by the broker. Otherwise, no topics will be created by the broker.

Responses:

Metadata Response (Version: 0) => [brokers] [topic_metadata] 
  brokers => node_id host port 
    node_id => INT32
    host => STRING
    port => INT32
  topic_metadata => error_code topic [partition_metadata] 
    error_code => INT16
    topic => STRING
    partition_metadata => error_code partition leader [replicas] [isr] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
FieldDescription
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.

Metadata Response (Version: 1) => [brokers] controller_id [topic_metadata] 
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader [replicas] [isr] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
FieldDescription
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.

Metadata Response (Version: 2) => [brokers] cluster_id controller_id [topic_metadata] 
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader [replicas] [isr] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
FieldDescription
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
cluster_idThe cluster id that this broker belongs to.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.

Metadata Response (Version: 3) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader [replicas] [isr] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
cluster_idThe cluster id that this broker belongs to.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.

Metadata Response (Version: 4) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader [replicas] [isr] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
cluster_idThe cluster id that this broker belongs to.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.

Metadata Response (Version: 5) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader [replicas] [isr] [offline_replicas] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
      offline_replicas => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
cluster_idThe cluster id that this broker belongs to.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.

Metadata Response (Version: 6) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader [replicas] [isr] [offline_replicas] 
      error_code => INT16
      partition => INT32
      leader => INT32
      replicas => INT32
      isr => INT32
      offline_replicas => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
cluster_idThe cluster id that this broker belongs to.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.

Metadata Response (Version: 7) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topic_metadata => error_code topic is_internal [partition_metadata] 
    error_code => INT16
    topic => STRING
    is_internal => BOOLEAN
    partition_metadata => error_code partition leader leader_epoch [replicas] [isr] [offline_replicas] 
      error_code => INT16
      partition => INT32
      leader => INT32
      leader_epoch => INT32
      replicas => INT32
      isr => INT32
      offline_replicas => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
brokersHost and port information for all brokers.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.
rackThe rack of the broker.
cluster_idThe cluster id that this broker belongs to.
controller_idThe broker id of the controller broker.
topic_metadataMetadata for requested topics
error_codeResponse error code
topicName of topic
is_internalIndicates if the topic is considered a Kafka internal topic
partition_metadataMetadata for each partition of the topic.
error_codeResponse error code
partitionTopic partition id
leaderThe id of the broker acting as leader for this partition.
leader_epochThe leader epoch
replicasThe set of all nodes that host this partition.
isrThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.

LeaderAndIsr API (Key: 4):
Requests:

LeaderAndIsr Request (Version: 0) => controller_id controller_epoch [partition_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_leaders => id host port 
    id => INT32
    host => STRING
    port => INT32
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
live_leadersLive leaders
idThe broker id
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

LeaderAndIsr Request (Version: 1) => controller_id controller_epoch [partition_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
    is_new => BOOLEAN
  live_leaders => id host port 
    id => INT32
    host => STRING
    port => INT32
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
is_newWhether the replica should have existed on the broker or not
live_leadersLive leaders
idThe broker id
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

LeaderAndIsr Request (Version: 2) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic [partition_states] 
    topic => STRING
    partition_states => partition controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new 
      partition => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      is_new => BOOLEAN
  live_leaders => id host port 
    id => INT32
    host => STRING
    port => INT32
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
broker_epochThe broker epoch
topic_statesTopic states
topicName of topic
partition_statesPartition states
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
is_newWhether the replica should have existed on the broker or not
live_leadersLive leaders
idThe broker id
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

Responses:

LeaderAndIsr Response (Version: 0) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
FieldDescription
error_codeResponse error code
partitionsResponse for the requests partitions
topicName of topic
partitionTopic partition id
error_codeResponse error code

LeaderAndIsr Response (Version: 1) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
FieldDescription
error_codeResponse error code
partitionsResponse for the requests partitions
topicName of topic
partitionTopic partition id
error_codeResponse error code

LeaderAndIsr Response (Version: 2) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
FieldDescription
error_codeResponse error code
partitionsResponse for the requests partitions
topicName of topic
partitionTopic partition id
error_codeResponse error code

StopReplica API (Key: 5):
Requests:

StopReplica Request (Version: 0) => controller_id controller_epoch delete_partitions [partitions] 
  controller_id => INT32
  controller_epoch => INT32
  delete_partitions => BOOLEAN
  partitions => topic partition 
    topic => STRING
    partition => INT32
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
delete_partitionsBoolean which indicates if replica's partitions must be deleted.
partitionsThe partitions
topicName of topic
partitionTopic partition id

StopReplica Request (Version: 1) => controller_id controller_epoch broker_epoch delete_partitions [partitions] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  delete_partitions => BOOLEAN
  partitions => topic [partition_ids] 
    topic => STRING
    partition_ids => INT32
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
broker_epochThe broker epoch
delete_partitionsBoolean which indicates if replica's partitions must be deleted.
partitionsThe partitions
topicName of topic
partition_idsThe partition ids of a topic

Responses:

StopReplica Response (Version: 0) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
FieldDescription
error_codeResponse error code
partitionsResponse for the requests partitions
topicName of topic
partitionTopic partition id
error_codeResponse error code

StopReplica Response (Version: 1) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
FieldDescription
error_codeResponse error code
partitionsResponse for the requests partitions
topicName of topic
partitionTopic partition id
error_codeResponse error code

UpdateMetadata API (Key: 6):
Requests:

UpdateMetadata Request (Version: 0) => controller_id controller_epoch [partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id host port 
    id => INT32
    host => STRING
    port => INT32
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
live_brokersLive broekrs
idThe broker id
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

UpdateMetadata Request (Version: 1) => controller_id controller_epoch [partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id [end_points] 
    id => INT32
    end_points => port host security_protocol_type 
      port => INT32
      host => STRING
      security_protocol_type => INT16
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
live_brokersLive broekrs
idThe broker id
end_pointsThe endpoints
portThe port on which the broker accepts requests.
hostThe hostname of the broker.
security_protocol_typeThe security protocol type.

UpdateMetadata Request (Version: 2) => controller_id controller_epoch [partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id [end_points] rack 
    id => INT32
    end_points => port host security_protocol_type 
      port => INT32
      host => STRING
      security_protocol_type => INT16
    rack => NULLABLE_STRING
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
live_brokersLive broekrs
idThe broker id
end_pointsThe endpoints
portThe port on which the broker accepts requests.
hostThe hostname of the broker.
security_protocol_typeThe security protocol type.
rackThe rack

UpdateMetadata Request (Version: 3) => controller_id controller_epoch [partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id [end_points] rack 
    id => INT32
    end_points => port host listener_name security_protocol_type 
      port => INT32
      host => STRING
      listener_name => STRING
      security_protocol_type => INT16
    rack => NULLABLE_STRING
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
live_brokersLive broekrs
idThe broker id
end_pointsThe endpoints
portThe port on which the broker accepts requests.
hostThe hostname of the broker.
listener_nameThe listener name.
security_protocol_typeThe security protocol type.
rackThe rack

UpdateMetadata Request (Version: 4) => controller_id controller_epoch [partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] 
    topic => STRING
    partition => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
    offline_replicas => INT32
  live_brokers => id [end_points] rack 
    id => INT32
    end_points => port host listener_name security_protocol_type 
      port => INT32
      host => STRING
      listener_name => STRING
      security_protocol_type => INT16
    rack => NULLABLE_STRING
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
partition_statesPartition states
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
offline_replicasThe offline replica ids
live_brokersLive broekrs
idThe broker id
end_pointsThe endpoints
portThe port on which the broker accepts requests.
hostThe hostname of the broker.
listener_nameThe listener name.
security_protocol_typeThe security protocol type.
rackThe rack

UpdateMetadata Request (Version: 5) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic [partition_states] 
    topic => STRING
    partition_states => partition controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] 
      partition => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      offline_replicas => INT32
  live_brokers => id [end_points] rack 
    id => INT32
    end_points => port host listener_name security_protocol_type 
      port => INT32
      host => STRING
      listener_name => STRING
      security_protocol_type => INT16
    rack => NULLABLE_STRING
FieldDescription
controller_idThe controller id
controller_epochThe controller epoch
broker_epochThe broker epoch
topic_statesTopic states
topicName of topic
partition_statesPartition states
partitionTopic partition id
controller_epochThe controller epoch
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
offline_replicasThe offline replica ids
live_brokersLive broekrs
idThe broker id
end_pointsThe endpoints
portThe port on which the broker accepts requests.
hostThe hostname of the broker.
listener_nameThe listener name.
security_protocol_typeThe security protocol type.
rackThe rack

Responses:

UpdateMetadata Response (Version: 0) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

UpdateMetadata Response (Version: 1) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

UpdateMetadata Response (Version: 2) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

UpdateMetadata Response (Version: 3) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

UpdateMetadata Response (Version: 4) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

UpdateMetadata Response (Version: 5) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

ControlledShutdown API (Key: 7):
Requests:

ControlledShutdown Request (Version: 0) => broker_id 
  broker_id => INT32
FieldDescription
broker_idThe id of the broker for which controlled shutdown has been requested.

ControlledShutdown Request (Version: 1) => broker_id 
  broker_id => INT32
FieldDescription
broker_idThe id of the broker for which controlled shutdown has been requested.

ControlledShutdown Request (Version: 2) => broker_id broker_epoch 
  broker_id => INT32
  broker_epoch => INT64
FieldDescription
broker_idThe id of the broker for which controlled shutdown has been requested.
broker_epochThe broker epoch

Responses:

ControlledShutdown Response (Version: 0) => error_code [partitions_remaining] 
  error_code => INT16
  partitions_remaining => topic partition 
    topic => STRING
    partition => INT32
FieldDescription
error_codeResponse error code
partitions_remainingThe partitions that the broker still leads.
topicName of topic
partitionTopic partition id

ControlledShutdown Response (Version: 1) => error_code [partitions_remaining] 
  error_code => INT16
  partitions_remaining => topic partition 
    topic => STRING
    partition => INT32
FieldDescription
error_codeResponse error code
partitions_remainingThe partitions that the broker still leads.
topicName of topic
partitionTopic partition id

ControlledShutdown Response (Version: 2) => error_code [partitions_remaining] 
  error_code => INT16
  partitions_remaining => topic partition 
    topic => STRING
    partition => INT32
FieldDescription
error_codeResponse error code
partitions_remainingThe partitions that the broker still leads.
topicName of topic
partitionTopic partition id

OffsetCommit API (Key: 8):
Requests:

OffsetCommit Request (Version: 0) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

OffsetCommit Request (Version: 1) => group_id generation_id member_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset timestamp metadata 
      partition => INT32
      offset => INT64
      timestamp => INT64
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
timestampTimestamp of the commit
metadataAny associated metadata the client wants to keep.

OffsetCommit Request (Version: 2) => group_id generation_id member_id retention_time [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time => INT64
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
retention_timeTime period in ms to retain the offset.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

OffsetCommit Request (Version: 3) => group_id generation_id member_id retention_time [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time => INT64
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
retention_timeTime period in ms to retain the offset.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

OffsetCommit Request (Version: 4) => group_id generation_id member_id retention_time [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time => INT64
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
retention_timeTime period in ms to retain the offset.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

OffsetCommit Request (Version: 5) => group_id generation_id member_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

OffsetCommit Request (Version: 6) => group_id generation_id member_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset leader_epoch metadata 
      partition => INT32
      offset => INT64
      leader_epoch => INT32
      metadata => NULLABLE_STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
leader_epochThe leader epoch, if provided is derived from the last consumed record. This is used by the consumer to check for log truncation and to ensure partition metadata is up to date following a group rebalance.
metadataAny associated metadata the client wants to keep.

Responses:

OffsetCommit Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 1) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 2) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 3) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 4) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 5) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 6) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for committed partitions
topicName of topic
partition_responsesResponses for committed partitions
partitionTopic partition id
error_codeResponse error code

OffsetFetch API (Key: 9):
Requests:

OffsetFetch Request (Version: 0) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition 
      partition => INT32
FieldDescription
group_idThe unique group identifier
topicsTopics to fetch offsets.
topicName of topic
partitionsPartitions to fetch offsets.
partitionTopic partition id

OffsetFetch Request (Version: 1) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition 
      partition => INT32
FieldDescription
group_idThe unique group identifier
topicsTopics to fetch offsets.
topicName of topic
partitionsPartitions to fetch offsets.
partitionTopic partition id

OffsetFetch Request (Version: 2) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition 
      partition => INT32
FieldDescription
group_idThe unique group identifier
topicsTopics to fetch offsets. If the topic array is null fetch offsets for all topics.
topicName of topic
partitionsPartitions to fetch offsets.
partitionTopic partition id

OffsetFetch Request (Version: 3) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition 
      partition => INT32
FieldDescription
group_idThe unique group identifier
topicsTopics to fetch offsets. If the topic array is null fetch offsets for all topics.
topicName of topic
partitionsPartitions to fetch offsets.
partitionTopic partition id

OffsetFetch Request (Version: 4) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition 
      partition => INT32
FieldDescription
group_idThe unique group identifier
topicsTopics to fetch offsets. If the topic array is null fetch offsets for all topics.
topicName of topic
partitionsPartitions to fetch offsets.
partitionTopic partition id

OffsetFetch Request (Version: 5) => group_id [topics] 
  group_id => STRING
  topics => topic [partitions] 
    topic => STRING
    partitions => partition 
      partition => INT32
FieldDescription
group_idThe unique group identifier
topicsTopics to fetch offsets. If the topic array is null fetch offsets for all topics.
topicName of topic
partitionsPartitions to fetch offsets.
partitionTopic partition id

Responses:

OffsetFetch Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
FieldDescription
responsesResponses by topic for fetched offsets
topicName of topic
partition_responsesResponses by partition for fetched offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.
error_codeResponse error code

OffsetFetch Response (Version: 1) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
FieldDescription
responsesResponses by topic for fetched offsets
topicName of topic
partition_responsesResponses by partition for fetched offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.
error_codeResponse error code

OffsetFetch Response (Version: 2) => [responses] error_code 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
FieldDescription
responsesResponses by topic for fetched offsets
topicName of topic
partition_responsesResponses by partition for fetched offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.
error_codeResponse error code
error_codeResponse error code

OffsetFetch Response (Version: 3) => throttle_time_ms [responses] error_code 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for fetched offsets
topicName of topic
partition_responsesResponses by partition for fetched offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.
error_codeResponse error code
error_codeResponse error code

OffsetFetch Response (Version: 4) => throttle_time_ms [responses] error_code 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for fetched offsets
topicName of topic
partition_responsesResponses by partition for fetched offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.
error_codeResponse error code
error_codeResponse error code

OffsetFetch Response (Version: 5) => throttle_time_ms [responses] error_code 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset leader_epoch metadata error_code 
      partition => INT32
      offset => INT64
      leader_epoch => INT32
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
responsesResponses by topic for fetched offsets
topicName of topic
partition_responsesResponses by partition for fetched offsets
partitionTopic partition id
offsetMessage offset to be committed
leader_epochThe leader epoch, if provided is derived from the last consumed record. This is used by the consumer to check for log truncation and to ensure partition metadata is up to date following a group rebalance.
metadataAny associated metadata the client wants to keep.
error_codeResponse error code
error_codeResponse error code

FindCoordinator API (Key: 10):
Requests:

FindCoordinator Request (Version: 0) => group_id 
  group_id => STRING
FieldDescription
group_idThe unique group identifier

FindCoordinator Request (Version: 1) => coordinator_key coordinator_type 
  coordinator_key => STRING
  coordinator_type => INT8
FieldDescription
coordinator_keyId to use for finding the coordinator (for groups, this is the groupId, for transactional producers, this is the transactional id)
coordinator_typeThe type of coordinator to find (0 = group, 1 = transaction)

FindCoordinator Request (Version: 2) => coordinator_key coordinator_type 
  coordinator_key => STRING
  coordinator_type => INT8
FieldDescription
coordinator_keyId to use for finding the coordinator (for groups, this is the groupId, for transactional producers, this is the transactional id)
coordinator_typeThe type of coordinator to find (0 = group, 1 = transaction)

Responses:

FindCoordinator Response (Version: 0) => error_code coordinator 
  error_code => INT16
  coordinator => node_id host port 
    node_id => INT32
    host => STRING
    port => INT32
FieldDescription
error_codeResponse error code
coordinatorHost and port information for the coordinator for a consumer group.
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

FindCoordinator Response (Version: 1) => throttle_time_ms error_code error_message coordinator 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  coordinator => node_id host port 
    node_id => INT32
    host => STRING
    port => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
error_messageResponse error message
coordinatorHost and port information for the coordinator
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

FindCoordinator Response (Version: 2) => throttle_time_ms error_code error_message coordinator 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  coordinator => node_id host port 
    node_id => INT32
    host => STRING
    port => INT32
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
error_messageResponse error message
coordinatorHost and port information for the coordinator
node_idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

JoinGroup API (Key: 11):
Requests:

JoinGroup Request (Version: 0) => group_id session_timeout member_id protocol_type [group_protocols] 
  group_id => STRING
  session_timeout => INT32
  member_id => STRING
  protocol_type => STRING
  group_protocols => protocol_name protocol_metadata 
    protocol_name => STRING
    protocol_metadata => BYTES
FieldDescription
group_idThe unique group identifier
session_timeoutThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
protocol_typeUnique name for class of protocols implemented by group
group_protocolsList of protocols that the member supports
protocol_namenull
protocol_metadatanull

JoinGroup Request (Version: 1) => group_id session_timeout rebalance_timeout member_id protocol_type [group_protocols] 
  group_id => STRING
  session_timeout => INT32
  rebalance_timeout => INT32
  member_id => STRING
  protocol_type => STRING
  group_protocols => protocol_name protocol_metadata 
    protocol_name => STRING
    protocol_metadata => BYTES
FieldDescription
group_idThe unique group identifier
session_timeoutThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms.
rebalance_timeoutThe maximum time that the coordinator will wait for each member to rejoin when rebalancing the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
protocol_typeUnique name for class of protocols implemented by group
group_protocolsList of protocols that the member supports
protocol_namenull
protocol_metadatanull

JoinGroup Request (Version: 2) => group_id session_timeout rebalance_timeout member_id protocol_type [group_protocols] 
  group_id => STRING
  session_timeout => INT32
  rebalance_timeout => INT32
  member_id => STRING
  protocol_type => STRING
  group_protocols => protocol_name protocol_metadata 
    protocol_name => STRING
    protocol_metadata => BYTES
FieldDescription
group_idThe unique group identifier
session_timeoutThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms.
rebalance_timeoutThe maximum time that the coordinator will wait for each member to rejoin when rebalancing the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
protocol_typeUnique name for class of protocols implemented by group
group_protocolsList of protocols that the member supports
protocol_namenull
protocol_metadatanull

JoinGroup Request (Version: 3) => group_id session_timeout rebalance_timeout member_id protocol_type [group_protocols] 
  group_id => STRING
  session_timeout => INT32
  rebalance_timeout => INT32
  member_id => STRING
  protocol_type => STRING
  group_protocols => protocol_name protocol_metadata 
    protocol_name => STRING
    protocol_metadata => BYTES
FieldDescription
group_idThe unique group identifier
session_timeoutThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms.
rebalance_timeoutThe maximum time that the coordinator will wait for each member to rejoin when rebalancing the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
protocol_typeUnique name for class of protocols implemented by group
group_protocolsList of protocols that the member supports
protocol_namenull
protocol_metadatanull

JoinGroup Request (Version: 4) => group_id session_timeout rebalance_timeout member_id protocol_type [group_protocols] 
  group_id => STRING
  session_timeout => INT32
  rebalance_timeout => INT32
  member_id => STRING
  protocol_type => STRING
  group_protocols => protocol_name protocol_metadata 
    protocol_name => STRING
    protocol_metadata => BYTES
FieldDescription
group_idThe unique group identifier
session_timeoutThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms.
rebalance_timeoutThe maximum time that the coordinator will wait for each member to rejoin when rebalancing the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
protocol_typeUnique name for class of protocols implemented by group
group_protocolsList of protocols that the member supports
protocol_namenull
protocol_metadatanull

Responses:

JoinGroup Response (Version: 0) => error_code generation_id group_protocol leader_id member_id [members] 
  error_code => INT16
  generation_id => INT32
  group_protocol => STRING
  leader_id => STRING
  member_id => STRING
  members => member_id member_metadata 
    member_id => STRING
    member_metadata => BYTES
FieldDescription
error_codeResponse error code
generation_idThe generation of the group.
group_protocolThe group protocol selected by the coordinator
leader_idThe leader of the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
membersnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_metadatanull

JoinGroup Response (Version: 1) => error_code generation_id group_protocol leader_id member_id [members] 
  error_code => INT16
  generation_id => INT32
  group_protocol => STRING
  leader_id => STRING
  member_id => STRING
  members => member_id member_metadata 
    member_id => STRING
    member_metadata => BYTES
FieldDescription
error_codeResponse error code
generation_idThe generation of the group.
group_protocolThe group protocol selected by the coordinator
leader_idThe leader of the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
membersnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_metadatanull

JoinGroup Response (Version: 2) => throttle_time_ms error_code generation_id group_protocol leader_id member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  group_protocol => STRING
  leader_id => STRING
  member_id => STRING
  members => member_id member_metadata 
    member_id => STRING
    member_metadata => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
generation_idThe generation of the group.
group_protocolThe group protocol selected by the coordinator
leader_idThe leader of the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
membersnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_metadatanull

JoinGroup Response (Version: 3) => throttle_time_ms error_code generation_id group_protocol leader_id member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  group_protocol => STRING
  leader_id => STRING
  member_id => STRING
  members => member_id member_metadata 
    member_id => STRING
    member_metadata => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
generation_idThe generation of the group.
group_protocolThe group protocol selected by the coordinator
leader_idThe leader of the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
membersnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_metadatanull

JoinGroup Response (Version: 4) => throttle_time_ms error_code generation_id group_protocol leader_id member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  group_protocol => STRING
  leader_id => STRING
  member_id => STRING
  members => member_id member_metadata 
    member_id => STRING
    member_metadata => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
generation_idThe generation of the group.
group_protocolThe group protocol selected by the coordinator
leader_idThe leader of the group
member_idThe member id assigned by the group coordinator or null if joining for the first time.
membersnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_metadatanull

Heartbeat API (Key: 12):
Requests:

Heartbeat Request (Version: 0) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.

Heartbeat Request (Version: 1) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.

Heartbeat Request (Version: 2) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.

Responses:

Heartbeat Response (Version: 0) => error_code 
  error_code => INT16
FieldDescription
error_codeResponse error code

Heartbeat Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code

Heartbeat Response (Version: 2) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code

LeaveGroup API (Key: 13):
Requests:

LeaveGroup Request (Version: 0) => group_id member_id 
  group_id => STRING
  member_id => STRING
FieldDescription
group_idThe ID of the group to leave.
member_idThe member ID to remove from the group.

LeaveGroup Request (Version: 1) => group_id member_id 
  group_id => STRING
  member_id => STRING
FieldDescription
group_idThe ID of the group to leave.
member_idThe member ID to remove from the group.

LeaveGroup Request (Version: 2) => group_id member_id 
  group_id => STRING
  member_id => STRING
FieldDescription
group_idThe ID of the group to leave.
member_idThe member ID to remove from the group.

Responses:

LeaveGroup Response (Version: 0) => error_code 
  error_code => INT16
FieldDescription
error_codeThe error code, or 0 if there was no error.

LeaveGroup Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.

LeaveGroup Response (Version: 2) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.

SyncGroup API (Key: 14):
Requests:

SyncGroup Request (Version: 0) => group_id generation_id member_id [group_assignment] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_assignment => member_id member_assignment 
    member_id => STRING
    member_assignment => BYTES
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
group_assignmentnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_assignmentnull

SyncGroup Request (Version: 1) => group_id generation_id member_id [group_assignment] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_assignment => member_id member_assignment 
    member_id => STRING
    member_assignment => BYTES
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
group_assignmentnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_assignmentnull

SyncGroup Request (Version: 2) => group_id generation_id member_id [group_assignment] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_assignment => member_id member_assignment 
    member_id => STRING
    member_assignment => BYTES
FieldDescription
group_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
group_assignmentnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_assignmentnull

Responses:

SyncGroup Response (Version: 0) => error_code member_assignment 
  error_code => INT16
  member_assignment => BYTES
FieldDescription
error_codeResponse error code
member_assignmentnull

SyncGroup Response (Version: 1) => throttle_time_ms error_code member_assignment 
  throttle_time_ms => INT32
  error_code => INT16
  member_assignment => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
member_assignmentnull

SyncGroup Response (Version: 2) => throttle_time_ms error_code member_assignment 
  throttle_time_ms => INT32
  error_code => INT16
  member_assignment => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
member_assignmentnull

DescribeGroups API (Key: 15):
Requests:

DescribeGroups Request (Version: 0) => [group_ids] 
  group_ids => STRING
FieldDescription
group_idsList of groupIds to request metadata for (an empty groupId array will return empty group metadata).

DescribeGroups Request (Version: 1) => [group_ids] 
  group_ids => STRING
FieldDescription
group_idsList of groupIds to request metadata for (an empty groupId array will return empty group metadata).

DescribeGroups Request (Version: 2) => [group_ids] 
  group_ids => STRING
FieldDescription
group_idsList of groupIds to request metadata for (an empty groupId array will return empty group metadata).

Responses:

DescribeGroups Response (Version: 0) => [groups] 
  groups => error_code group_id state protocol_type protocol [members] 
    error_code => INT16
    group_id => STRING
    state => STRING
    protocol_type => STRING
    protocol => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
FieldDescription
groupsnull
error_codeResponse error code
group_idThe unique group identifier
stateThe current state of the group (one of: Dead, Stable, CompletingRebalance, PreparingRebalance, or empty if there is no active group)
protocol_typeThe current group protocol type (will be empty if there is no active group)
protocolThe current group protocol (only provided if the group is Stable)
membersCurrent group members (only provided if the group is not Dead)
member_idThe member id assigned by the group coordinator or null if joining for the first time.
client_idThe client id used in the member's latest join group request
client_hostThe client host used in the request session corresponding to the member's join group.
member_metadataThe metadata corresponding to the current group protocol in use (will only be present if the group is stable).
member_assignmentThe current assignment provided by the group leader (will only be present if the group is stable).

DescribeGroups Response (Version: 1) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id state protocol_type protocol [members] 
    error_code => INT16
    group_id => STRING
    state => STRING
    protocol_type => STRING
    protocol => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
groupsnull
error_codeResponse error code
group_idThe unique group identifier
stateThe current state of the group (one of: Dead, Stable, CompletingRebalance, PreparingRebalance, or empty if there is no active group)
protocol_typeThe current group protocol type (will be empty if there is no active group)
protocolThe current group protocol (only provided if the group is Stable)
membersCurrent group members (only provided if the group is not Dead)
member_idThe member id assigned by the group coordinator or null if joining for the first time.
client_idThe client id used in the member's latest join group request
client_hostThe client host used in the request session corresponding to the member's join group.
member_metadataThe metadata corresponding to the current group protocol in use (will only be present if the group is stable).
member_assignmentThe current assignment provided by the group leader (will only be present if the group is stable).

DescribeGroups Response (Version: 2) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id state protocol_type protocol [members] 
    error_code => INT16
    group_id => STRING
    state => STRING
    protocol_type => STRING
    protocol => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
groupsnull
error_codeResponse error code
group_idThe unique group identifier
stateThe current state of the group (one of: Dead, Stable, CompletingRebalance, PreparingRebalance, or empty if there is no active group)
protocol_typeThe current group protocol type (will be empty if there is no active group)
protocolThe current group protocol (only provided if the group is Stable)
membersCurrent group members (only provided if the group is not Dead)
member_idThe member id assigned by the group coordinator or null if joining for the first time.
client_idThe client id used in the member's latest join group request
client_hostThe client host used in the request session corresponding to the member's join group.
member_metadataThe metadata corresponding to the current group protocol in use (will only be present if the group is stable).
member_assignmentThe current assignment provided by the group leader (will only be present if the group is stable).

ListGroups API (Key: 16):
Requests:

ListGroups Request (Version: 0) => 
FieldDescription

ListGroups Request (Version: 1) => 
FieldDescription

ListGroups Request (Version: 2) => 
FieldDescription

Responses:

ListGroups Response (Version: 0) => error_code [groups] 
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING
FieldDescription
error_codeResponse error code
groupsnull
group_idThe unique group identifier
protocol_typenull

ListGroups Response (Version: 1) => throttle_time_ms error_code [groups] 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
groupsnull
group_idThe unique group identifier
protocol_typenull

ListGroups Response (Version: 2) => throttle_time_ms error_code [groups] 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
groupsnull
group_idThe unique group identifier
protocol_typenull

SaslHandshake API (Key: 17):
Requests:

SaslHandshake Request (Version: 0) => mechanism 
  mechanism => STRING
FieldDescription
mechanismSASL Mechanism chosen by the client.

SaslHandshake Request (Version: 1) => mechanism 
  mechanism => STRING
FieldDescription
mechanismSASL Mechanism chosen by the client.

Responses:

SaslHandshake Response (Version: 0) => error_code [enabled_mechanisms] 
  error_code => INT16
  enabled_mechanisms => STRING
FieldDescription
error_codeResponse error code
enabled_mechanismsArray of mechanisms enabled in the server.

SaslHandshake Response (Version: 1) => error_code [enabled_mechanisms] 
  error_code => INT16
  enabled_mechanisms => STRING
FieldDescription
error_codeResponse error code
enabled_mechanismsArray of mechanisms enabled in the server.

ApiVersions API (Key: 18):
Requests:

ApiVersions Request (Version: 0) => 
FieldDescription

ApiVersions Request (Version: 1) => 
FieldDescription

ApiVersions Request (Version: 2) => 
FieldDescription

Responses:

ApiVersions Response (Version: 0) => error_code [api_versions] 
  error_code => INT16
  api_versions => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
FieldDescription
error_codeResponse error code
api_versionsAPI versions supported by the broker.
api_keyAPI key.
min_versionMinimum supported version.
max_versionMaximum supported version.

ApiVersions Response (Version: 1) => error_code [api_versions] throttle_time_ms 
  error_code => INT16
  api_versions => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
api_versionsAPI versions supported by the broker.
api_keyAPI key.
min_versionMinimum supported version.
max_versionMaximum supported version.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

ApiVersions Response (Version: 2) => error_code [api_versions] throttle_time_ms 
  error_code => INT16
  api_versions => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
api_versionsAPI versions supported by the broker.
api_keyAPI key.
min_versionMinimum supported version.
max_versionMaximum supported version.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

CreateTopics API (Key: 19):
Requests:

CreateTopics Request (Version: 0) => [create_topic_requests] timeout 
  create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries] 
    topic => STRING
    num_partitions => INT32
    replication_factor => INT16
    replica_assignment => partition [replicas] 
      partition => INT32
      replicas => INT32
    config_entries => config_name config_value 
      config_name => STRING
      config_value => NULLABLE_STRING
  timeout => INT32
FieldDescription
create_topic_requestsAn array of single topic creation requests. Can not have multiple entries for the same topic.
topicName of topic
num_partitionsNumber of partitions to be created. -1 indicates unset.
replication_factorReplication factor for the topic. -1 indicates unset.
replica_assignmentReplica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset.
partitionTopic partition id
replicasThe set of all nodes that should host this partition. The first replica in the list is the preferred leader.
config_entriesTopic level configuration for topic to be set.
config_nameConfiguration name
config_valueConfiguration value
timeoutThe time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately

CreateTopics Request (Version: 1) => [create_topic_requests] timeout validate_only 
  create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries] 
    topic => STRING
    num_partitions => INT32
    replication_factor => INT16
    replica_assignment => partition [replicas] 
      partition => INT32
      replicas => INT32
    config_entries => config_name config_value 
      config_name => STRING
      config_value => NULLABLE_STRING
  timeout => INT32
  validate_only => BOOLEAN
FieldDescription
create_topic_requestsAn array of single topic creation requests. Can not have multiple entries for the same topic.
topicName of topic
num_partitionsNumber of partitions to be created. -1 indicates unset.
replication_factorReplication factor for the topic. -1 indicates unset.
replica_assignmentReplica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset.
partitionTopic partition id
replicasThe set of all nodes that should host this partition. The first replica in the list is the preferred leader.
config_entriesTopic level configuration for topic to be set.
config_nameConfiguration name
config_valueConfiguration value
timeoutThe time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately
validate_onlyIf this is true, the request will be validated, but the topic won't be created.

CreateTopics Request (Version: 2) => [create_topic_requests] timeout validate_only 
  create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries] 
    topic => STRING
    num_partitions => INT32
    replication_factor => INT16
    replica_assignment => partition [replicas] 
      partition => INT32
      replicas => INT32
    config_entries => config_name config_value 
      config_name => STRING
      config_value => NULLABLE_STRING
  timeout => INT32
  validate_only => BOOLEAN
FieldDescription
create_topic_requestsAn array of single topic creation requests. Can not have multiple entries for the same topic.
topicName of topic
num_partitionsNumber of partitions to be created. -1 indicates unset.
replication_factorReplication factor for the topic. -1 indicates unset.
replica_assignmentReplica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset.
partitionTopic partition id
replicasThe set of all nodes that should host this partition. The first replica in the list is the preferred leader.
config_entriesTopic level configuration for topic to be set.
config_nameConfiguration name
config_valueConfiguration value
timeoutThe time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately
validate_onlyIf this is true, the request will be validated, but the topic won't be created.

CreateTopics Request (Version: 3) => [create_topic_requests] timeout validate_only 
  create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries] 
    topic => STRING
    num_partitions => INT32
    replication_factor => INT16
    replica_assignment => partition [replicas] 
      partition => INT32
      replicas => INT32
    config_entries => config_name config_value 
      config_name => STRING
      config_value => NULLABLE_STRING
  timeout => INT32
  validate_only => BOOLEAN
FieldDescription
create_topic_requestsAn array of single topic creation requests. Can not have multiple entries for the same topic.
topicName of topic
num_partitionsNumber of partitions to be created. -1 indicates unset.
replication_factorReplication factor for the topic. -1 indicates unset.
replica_assignmentReplica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset.
partitionTopic partition id
replicasThe set of all nodes that should host this partition. The first replica in the list is the preferred leader.
config_entriesTopic level configuration for topic to be set.
config_nameConfiguration name
config_valueConfiguration value
timeoutThe time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately
validate_onlyIf this is true, the request will be validated, but the topic won't be created.

Responses:

CreateTopics Response (Version: 0) => [topic_errors] 
  topic_errors => topic error_code 
    topic => STRING
    error_code => INT16
FieldDescription
topic_errorsAn array of per topic error codes.
topicName of topic
error_codeResponse error code

CreateTopics Response (Version: 1) => [topic_errors] 
  topic_errors => topic error_code error_message 
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
topic_errorsAn array of per topic errors.
topicName of topic
error_codeResponse error code
error_messageResponse error message

CreateTopics Response (Version: 2) => throttle_time_ms [topic_errors] 
  throttle_time_ms => INT32
  topic_errors => topic error_code error_message 
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_errorsAn array of per topic errors.
topicName of topic
error_codeResponse error code
error_messageResponse error message

CreateTopics Response (Version: 3) => throttle_time_ms [topic_errors] 
  throttle_time_ms => INT32
  topic_errors => topic error_code error_message 
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_errorsAn array of per topic errors.
topicName of topic
error_codeResponse error code
error_messageResponse error message

DeleteTopics API (Key: 20):
Requests:

DeleteTopics Request (Version: 0) => [topics] timeout 
  topics => STRING
  timeout => INT32
FieldDescription
topicsAn array of topics to be deleted.
timeoutThe time in ms to wait for a topic to be completely deleted on the controller node. Values <= 0 will trigger topic deletion and return immediately

DeleteTopics Request (Version: 1) => [topics] timeout 
  topics => STRING
  timeout => INT32
FieldDescription
topicsAn array of topics to be deleted.
timeoutThe time in ms to wait for a topic to be completely deleted on the controller node. Values <= 0 will trigger topic deletion and return immediately

DeleteTopics Request (Version: 2) => [topics] timeout 
  topics => STRING
  timeout => INT32
FieldDescription
topicsAn array of topics to be deleted.
timeoutThe time in ms to wait for a topic to be completely deleted on the controller node. Values <= 0 will trigger topic deletion and return immediately

DeleteTopics Request (Version: 3) => [topics] timeout 
  topics => STRING
  timeout => INT32
FieldDescription
topicsAn array of topics to be deleted.
timeoutThe time in ms to wait for a topic to be completely deleted on the controller node. Values <= 0 will trigger topic deletion and return immediately

Responses:

DeleteTopics Response (Version: 0) => [topic_error_codes] 
  topic_error_codes => topic error_code 
    topic => STRING
    error_code => INT16
FieldDescription
topic_error_codesAn array of per topic error codes.
topicName of topic
error_codeResponse error code

DeleteTopics Response (Version: 1) => throttle_time_ms [topic_error_codes] 
  throttle_time_ms => INT32
  topic_error_codes => topic error_code 
    topic => STRING
    error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_error_codesAn array of per topic error codes.
topicName of topic
error_codeResponse error code

DeleteTopics Response (Version: 2) => throttle_time_ms [topic_error_codes] 
  throttle_time_ms => INT32
  topic_error_codes => topic error_code 
    topic => STRING
    error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_error_codesAn array of per topic error codes.
topicName of topic
error_codeResponse error code

DeleteTopics Response (Version: 3) => throttle_time_ms [topic_error_codes] 
  throttle_time_ms => INT32
  topic_error_codes => topic error_code 
    topic => STRING
    error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_error_codesAn array of per topic error codes.
topicName of topic
error_codeResponse error code

DeleteRecords API (Key: 21):
Requests:

DeleteRecords Request (Version: 0) => [topics] timeout 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset 
      partition => INT32
      offset => INT64
  timeout => INT32
FieldDescription
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
offsetThe offset before which the messages will be deleted. -1 means high-watermark for the partition.
timeoutThe maximum time to await a response in ms.

DeleteRecords Request (Version: 1) => [topics] timeout 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset 
      partition => INT32
      offset => INT64
  timeout => INT32
FieldDescription
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
offsetThe offset before which the messages will be deleted. -1 means high-watermark for the partition.
timeoutThe maximum time to await a response in ms.

Responses:

DeleteRecords Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition low_watermark error_code 
      partition => INT32
      low_watermark => INT64
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
low_watermarkSmallest available offset of all live replicas
error_codeResponse error code

DeleteRecords Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition low_watermark error_code 
      partition => INT32
      low_watermark => INT64
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
low_watermarkSmallest available offset of all live replicas
error_codeResponse error code

InitProducerId API (Key: 22):
Requests:

InitProducerId Request (Version: 0) => transactional_id transaction_timeout_ms 
  transactional_id => NULLABLE_STRING
  transaction_timeout_ms => INT32
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
transaction_timeout_msThe time in ms to wait for before aborting idle transactions sent by this producer.

InitProducerId Request (Version: 1) => transactional_id transaction_timeout_ms 
  transactional_id => NULLABLE_STRING
  transaction_timeout_ms => INT32
FieldDescription
transactional_idThe transactional id or null if the producer is not transactional
transaction_timeout_msThe time in ms to wait for before aborting idle transactions sent by this producer.

Responses:

InitProducerId Response (Version: 0) => throttle_time_ms error_code producer_id producer_epoch 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.

InitProducerId Response (Version: 1) => throttle_time_ms error_code producer_id producer_epoch 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.

OffsetForLeaderEpoch API (Key: 23):
Requests:

OffsetForLeaderEpoch Request (Version: 0) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition leader_epoch 
      partition => INT32
      leader_epoch => INT32
FieldDescription
topicsAn array of topics to get epochs for
topicName of topic
partitionsAn array of partitions to get epochs for
partitionTopic partition id
leader_epochThe epoch to lookup an offset for.

OffsetForLeaderEpoch Request (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition leader_epoch 
      partition => INT32
      leader_epoch => INT32
FieldDescription
topicsAn array of topics to get epochs for
topicName of topic
partitionsAn array of partitions to get epochs for
partitionTopic partition id
leader_epochThe epoch to lookup an offset for.

OffsetForLeaderEpoch Request (Version: 2) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch leader_epoch 
      partition => INT32
      current_leader_epoch => INT32
      leader_epoch => INT32
FieldDescription
topicsAn array of topics to get epochs for
topicName of topic
partitionsAn array of partitions to get epochs for
partitionTopic partition id
current_leader_epochThe current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
leader_epochThe epoch to lookup an offset for.

Responses:

OffsetForLeaderEpoch Response (Version: 0) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition end_offset 
      error_code => INT16
      partition => INT32
      end_offset => INT64
FieldDescription
topicsAn array of topics for which we have leader offsets for some requested partition leader epoch
topicName of topic
partitionsAn array of offsets by partition
error_codeResponse error code
partitionTopic partition id
end_offsetThe end offset

OffsetForLeaderEpoch Response (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64
FieldDescription
topicsAn array of topics for which we have leader offsets for some requested partition leader epoch
topicName of topic
partitionsAn array of offsets by partition
error_codeResponse error code
partitionTopic partition id
leader_epochThe leader epoch
end_offsetThe end offset

OffsetForLeaderEpoch Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsAn array of topics for which we have leader offsets for some requested partition leader epoch
topicName of topic
partitionsAn array of offsets by partition
error_codeResponse error code
partitionTopic partition id
leader_epochThe leader epoch
end_offsetThe end offset

AddPartitionsToTxn API (Key: 24):
Requests:

AddPartitionsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsThe partitions to add to the transaction.
topicName of topic
partitionsnull

AddPartitionsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsThe partitions to add to the transaction.
topicName of topic
partitionsnull

Responses:

AddPartitionsToTxn Response (Version: 0) => throttle_time_ms [errors] 
  throttle_time_ms => INT32
  errors => topic [partition_errors] 
    topic => STRING
    partition_errors => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
errorsnull
topicName of topic
partition_errorsnull
partitionTopic partition id
error_codeResponse error code

AddPartitionsToTxn Response (Version: 1) => throttle_time_ms [errors] 
  throttle_time_ms => INT32
  errors => topic [partition_errors] 
    topic => STRING
    partition_errors => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
errorsnull
topicName of topic
partition_errorsnull
partitionTopic partition id
error_codeResponse error code

AddOffsetsToTxn API (Key: 25):
Requests:

AddOffsetsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch group_id 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => STRING
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
group_idThe unique group identifier

AddOffsetsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch group_id 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => STRING
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
group_idThe unique group identifier

Responses:

AddOffsetsToTxn Response (Version: 0) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code

AddOffsetsToTxn Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code

EndTxn API (Key: 26):
Requests:

EndTxn Request (Version: 0) => transactional_id producer_id producer_epoch transaction_result 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  transaction_result => BOOLEAN
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
transaction_resultThe result of the transaction (0 = ABORT, 1 = COMMIT)

EndTxn Request (Version: 1) => transactional_id producer_id producer_epoch transaction_result 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  transaction_result => BOOLEAN
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
transaction_resultThe result of the transaction (0 = ABORT, 1 = COMMIT)

Responses:

EndTxn Response (Version: 0) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code

EndTxn Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code

WriteTxnMarkers API (Key: 27):
Requests:

WriteTxnMarkers Request (Version: 0) => [transaction_markers] 
  transaction_markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch 
    producer_id => INT64
    producer_epoch => INT16
    transaction_result => BOOLEAN
    topics => topic [partitions] 
      topic => STRING
      partitions => INT32
    coordinator_epoch => INT32
FieldDescription
transaction_markersThe transaction markers to be written.
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
transaction_resultThe result of the transaction to write to the partitions (false = ABORT, true = COMMIT).
topicsThe partitions to write markers for.
topicName of topic
partitionsnull
coordinator_epochEpoch associated with the transaction state partition hosted by this transaction coordinator

Responses:

WriteTxnMarkers Response (Version: 0) => [transaction_markers] 
  transaction_markers => producer_id [topics] 
    producer_id => INT64
    topics => topic [partitions] 
      topic => STRING
      partitions => partition error_code 
        partition => INT32
        error_code => INT16
FieldDescription
transaction_markersErrors per partition from writing markers.
producer_idCurrent producer id in use by the transactional id.
topicsErrors per partition from writing markers.
topicName of topic
partitionsnull
partitionTopic partition id
error_codeResponse error code

TxnOffsetCommit API (Key: 28):
Requests:

TxnOffsetCommit Request (Version: 0) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
group_idThe unique group identifier
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

TxnOffsetCommit Request (Version: 1) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
group_idThe unique group identifier
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
metadataAny associated metadata the client wants to keep.

TxnOffsetCommit Request (Version: 2) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset leader_epoch metadata 
      partition => INT32
      offset => INT64
      leader_epoch => INT32
      metadata => NULLABLE_STRING
FieldDescription
transactional_idThe transactional id corresponding to the transaction.
group_idThe unique group identifier
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsTopics to commit offsets
topicName of topic
partitionsPartitions to commit offsets
partitionTopic partition id
offsetMessage offset to be committed
leader_epochThe leader epoch, if provided is derived from the last consumed record. This is used by the consumer to check for log truncation and to ensure partition metadata is up to date following a group rebalance.
metadataAny associated metadata the client wants to keep.

Responses:

TxnOffsetCommit Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsResponses by topic for committed offsets
topicName of topic
partitionsResponses by partition for committed offsets
partitionTopic partition id
error_codeResponse error code

TxnOffsetCommit Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsResponses by topic for committed offsets
topicName of topic
partitionsResponses by partition for committed offsets
partitionTopic partition id
error_codeResponse error code

TxnOffsetCommit Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsResponses by topic for committed offsets
topicName of topic
partitionsResponses by partition for committed offsets
partitionTopic partition id
error_codeResponse error code

DescribeAcls API (Key: 29):
Requests:

DescribeAcls Request (Version: 0) => resource_type resource_name principal host operation permission_type 
  resource_type => INT8
  resource_name => NULLABLE_STRING
  principal => NULLABLE_STRING
  host => NULLABLE_STRING
  operation => INT8
  permission_type => INT8
FieldDescription
resource_typeThe resource type
resource_nameThe resource name filter
principalThe ACL principal filter
hostThe ACL host filter
operationThe ACL operation
permission_typeThe ACL permission type

DescribeAcls Request (Version: 1) => resource_type resource_name resource_pattern_type_filter principal host operation permission_type 
  resource_type => INT8
  resource_name => NULLABLE_STRING
  resource_pattern_type_filter => INT8
  principal => NULLABLE_STRING
  host => NULLABLE_STRING
  operation => INT8
  permission_type => INT8
FieldDescription
resource_typeThe resource type
resource_nameThe resource name filter
resource_pattern_type_filterThe resource pattern type filter
principalThe ACL principal filter
hostThe ACL host filter
operationThe ACL operation
permission_typeThe ACL permission type

Responses:

DescribeAcls Response (Version: 0) => throttle_time_ms error_code error_message [resources] 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  resources => resource_type resource_name [acls] 
    resource_type => INT8
    resource_name => STRING
    acls => principal host operation permission_type 
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
error_messageResponse error message
resourcesThe resources and their associated ACLs.
resource_typeThe resource type
resource_nameThe resource name
aclsnull
principalThe ACL principal
hostThe ACL host
operationThe ACL operation
permission_typeThe ACL permission type

DescribeAcls Response (Version: 1) => throttle_time_ms error_code error_message [resources] 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  resources => resource_type resource_name resource_pattten_type [acls] 
    resource_type => INT8
    resource_name => STRING
    resource_pattten_type => INT8
    acls => principal host operation permission_type 
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
error_codeResponse error code
error_messageResponse error message
resourcesThe resources and their associated ACLs.
resource_typeThe resource type
resource_nameThe resource name
resource_pattten_typeThe resource pattern type
aclsnull
principalThe ACL principal
hostThe ACL host
operationThe ACL operation
permission_typeThe ACL permission type

CreateAcls API (Key: 30):
Requests:

CreateAcls Request (Version: 0) => [creations] 
  creations => resource_type resource_name principal host operation permission_type 
    resource_type => INT8
    resource_name => STRING
    principal => STRING
    host => STRING
    operation => INT8
    permission_type => INT8
FieldDescription
creationsnull
resource_typeThe resource type
resource_nameThe resource name
principalThe ACL principal
hostThe ACL host
operationThe ACL operation
permission_typeThe ACL permission type

CreateAcls Request (Version: 1) => [creations] 
  creations => resource_type resource_name resource_pattten_type principal host operation permission_type 
    resource_type => INT8
    resource_name => STRING
    resource_pattten_type => INT8
    principal => STRING
    host => STRING
    operation => INT8
    permission_type => INT8
FieldDescription
creationsnull
resource_typeThe resource type
resource_nameThe resource name
resource_pattten_typeThe resource pattern type
principalThe ACL principal
hostThe ACL host
operationThe ACL operation
permission_typeThe ACL permission type

Responses:

CreateAcls Response (Version: 0) => throttle_time_ms [creation_responses] 
  throttle_time_ms => INT32
  creation_responses => error_code error_message 
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
creation_responsesnull
error_codeResponse error code
error_messageResponse error message

CreateAcls Response (Version: 1) => throttle_time_ms [creation_responses] 
  throttle_time_ms => INT32
  creation_responses => error_code error_message 
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
creation_responsesnull
error_codeResponse error code
error_messageResponse error message

DeleteAcls API (Key: 31):
Requests:

DeleteAcls Request (Version: 0) => [filters] 
  filters => resource_type resource_name principal host operation permission_type 
    resource_type => INT8
    resource_name => NULLABLE_STRING
    principal => NULLABLE_STRING
    host => NULLABLE_STRING
    operation => INT8
    permission_type => INT8
FieldDescription
filtersnull
resource_typeThe resource type
resource_nameThe resource name filter
principalThe ACL principal filter
hostThe ACL host filter
operationThe ACL operation
permission_typeThe ACL permission type

DeleteAcls Request (Version: 1) => [filters] 
  filters => resource_type resource_name resource_pattern_type_filter principal host operation permission_type 
    resource_type => INT8
    resource_name => NULLABLE_STRING
    resource_pattern_type_filter => INT8
    principal => NULLABLE_STRING
    host => NULLABLE_STRING
    operation => INT8
    permission_type => INT8
FieldDescription
filtersnull
resource_typeThe resource type
resource_nameThe resource name filter
resource_pattern_type_filterThe resource pattern type filter
principalThe ACL principal filter
hostThe ACL host filter
operationThe ACL operation
permission_typeThe ACL permission type

Responses:

DeleteAcls Response (Version: 0) => throttle_time_ms [filter_responses] 
  throttle_time_ms => INT32
  filter_responses => error_code error_message [matching_acls] 
    error_code => INT16
    error_message => NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name principal host operation permission_type 
      error_code => INT16
      error_message => NULLABLE_STRING
      resource_type => INT8
      resource_name => STRING
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
filter_responsesnull
error_codeResponse error code
error_messageResponse error message
matching_aclsThe matching ACLs
error_codeResponse error code
error_messageResponse error message
resource_typeThe resource type
resource_nameThe resource name
principalThe ACL principal
hostThe ACL host
operationThe ACL operation
permission_typeThe ACL permission type

DeleteAcls Response (Version: 1) => throttle_time_ms [filter_responses] 
  throttle_time_ms => INT32
  filter_responses => error_code error_message [matching_acls] 
    error_code => INT16
    error_message => NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name resource_pattten_type principal host operation permission_type 
      error_code => INT16
      error_message => NULLABLE_STRING
      resource_type => INT8
      resource_name => STRING
      resource_pattten_type => INT8
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
filter_responsesnull
error_codeResponse error code
error_messageResponse error message
matching_aclsThe matching ACLs
error_codeResponse error code
error_messageResponse error message
resource_typeThe resource type
resource_nameThe resource name
resource_pattten_typeThe resource pattern type
principalThe ACL principal
hostThe ACL host
operationThe ACL operation
permission_typeThe ACL permission type

DescribeConfigs API (Key: 32):
Requests:

DescribeConfigs Request (Version: 0) => [resources] 
  resources => resource_type resource_name [config_names] 
    resource_type => INT8
    resource_name => STRING
    config_names => STRING
FieldDescription
resourcesAn array of config resources to be returned.
resource_typenull
resource_namenull
config_namesnull

DescribeConfigs Request (Version: 1) => [resources] include_synonyms 
  resources => resource_type resource_name [config_names] 
    resource_type => INT8
    resource_name => STRING
    config_names => STRING
  include_synonyms => BOOLEAN
FieldDescription
resourcesAn array of config resources to be returned.
resource_typenull
resource_namenull
config_namesnull
include_synonymsnull

DescribeConfigs Request (Version: 2) => [resources] include_synonyms 
  resources => resource_type resource_name [config_names] 
    resource_type => INT8
    resource_name => STRING
    config_names => STRING
  include_synonyms => BOOLEAN
FieldDescription
resourcesAn array of config resources to be returned.
resource_typenull
resource_namenull
config_namesnull
include_synonymsnull

Responses:

DescribeConfigs Response (Version: 0) => throttle_time_ms [resources] 
  throttle_time_ms => INT32
  resources => error_code error_message resource_type resource_name [config_entries] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    config_entries => config_name config_value read_only is_default is_sensitive 
      config_name => STRING
      config_value => NULLABLE_STRING
      read_only => BOOLEAN
      is_default => BOOLEAN
      is_sensitive => BOOLEAN
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
resourcesnull
error_codeResponse error code
error_messageResponse error message
resource_typenull
resource_namenull
config_entriesnull
config_namenull
config_valuenull
read_onlynull
is_defaultnull
is_sensitivenull

DescribeConfigs Response (Version: 1) => throttle_time_ms [resources] 
  throttle_time_ms => INT32
  resources => error_code error_message resource_type resource_name [config_entries] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    config_entries => config_name config_value read_only config_source is_sensitive [config_synonyms] 
      config_name => STRING
      config_value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      config_synonyms => config_name config_value config_source 
        config_name => STRING
        config_value => NULLABLE_STRING
        config_source => INT8
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
resourcesnull
error_codeResponse error code
error_messageResponse error message
resource_typenull
resource_namenull
config_entriesnull
config_namenull
config_valuenull
read_onlynull
config_sourcenull
is_sensitivenull
config_synonymsnull
config_namenull
config_valuenull
config_sourcenull

DescribeConfigs Response (Version: 2) => throttle_time_ms [resources] 
  throttle_time_ms => INT32
  resources => error_code error_message resource_type resource_name [config_entries] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    config_entries => config_name config_value read_only config_source is_sensitive [config_synonyms] 
      config_name => STRING
      config_value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      config_synonyms => config_name config_value config_source 
        config_name => STRING
        config_value => NULLABLE_STRING
        config_source => INT8
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
resourcesnull
error_codeResponse error code
error_messageResponse error message
resource_typenull
resource_namenull
config_entriesnull
config_namenull
config_valuenull
read_onlynull
config_sourcenull
is_sensitivenull
config_synonymsnull
config_namenull
config_valuenull
config_sourcenull

AlterConfigs API (Key: 33):
Requests:

AlterConfigs Request (Version: 0) => [resources] validate_only 
  resources => resource_type resource_name [config_entries] 
    resource_type => INT8
    resource_name => STRING
    config_entries => config_name config_value 
      config_name => STRING
      config_value => NULLABLE_STRING
  validate_only => BOOLEAN
FieldDescription
resourcesAn array of resources to update with the provided configs.
resource_typenull
resource_namenull
config_entriesnull
config_nameConfiguration name
config_valueConfiguration value
validate_onlynull

AlterConfigs Request (Version: 1) => [resources] validate_only 
  resources => resource_type resource_name [config_entries] 
    resource_type => INT8
    resource_name => STRING
    config_entries => config_name config_value 
      config_name => STRING
      config_value => NULLABLE_STRING
  validate_only => BOOLEAN
FieldDescription
resourcesAn array of resources to update with the provided configs.
resource_typenull
resource_namenull
config_entriesnull
config_nameConfiguration name
config_valueConfiguration value
validate_onlynull

Responses:

AlterConfigs Response (Version: 0) => throttle_time_ms [resources] 
  throttle_time_ms => INT32
  resources => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
resourcesnull
error_codeResponse error code
error_messageResponse error message
resource_typenull
resource_namenull

AlterConfigs Response (Version: 1) => throttle_time_ms [resources] 
  throttle_time_ms => INT32
  resources => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
resourcesnull
error_codeResponse error code
error_messageResponse error message
resource_typenull
resource_namenull

AlterReplicaLogDirs API (Key: 34):
Requests:

AlterReplicaLogDirs Request (Version: 0) => [log_dirs] 
  log_dirs => log_dir [topics] 
    log_dir => STRING
    topics => topic [partitions] 
      topic => STRING
      partitions => INT32
FieldDescription
log_dirsnull
log_dirThe absolute log directory path.
topicsnull
topicName of topic
partitionsList of partition ids of the topic.

AlterReplicaLogDirs Request (Version: 1) => [log_dirs] 
  log_dirs => log_dir [topics] 
    log_dir => STRING
    topics => topic [partitions] 
      topic => STRING
      partitions => INT32
FieldDescription
log_dirsnull
log_dirThe absolute log directory path.
topicsnull
topicName of topic
partitionsList of partition ids of the topic.

Responses:

AlterReplicaLogDirs Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
error_codeResponse error code

AlterReplicaLogDirs Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
error_codeResponse error code

DescribeLogDirs API (Key: 35):
Requests:

DescribeLogDirs Request (Version: 0) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
topicsnull
topicName of topic
partitionsList of partition ids of the topic.

DescribeLogDirs Request (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => INT32
FieldDescription
topicsnull
topicName of topic
partitionsList of partition ids of the topic.

Responses:

DescribeLogDirs Response (Version: 0) => throttle_time_ms [log_dirs] 
  throttle_time_ms => INT32
  log_dirs => error_code log_dir [topics] 
    error_code => INT16
    log_dir => STRING
    topics => topic [partitions] 
      topic => STRING
      partitions => partition size offset_lag is_future 
        partition => INT32
        size => INT64
        offset_lag => INT64
        is_future => BOOLEAN
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
log_dirsnull
error_codeResponse error code
log_dirThe absolute log directory path.
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
sizeThe size of the log segments of the partition in bytes.
offset_lagThe lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or current replica's LEO (if it is the future log for the partition)
is_futureTrue if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.

DescribeLogDirs Response (Version: 1) => throttle_time_ms [log_dirs] 
  throttle_time_ms => INT32
  log_dirs => error_code log_dir [topics] 
    error_code => INT16
    log_dir => STRING
    topics => topic [partitions] 
      topic => STRING
      partitions => partition size offset_lag is_future 
        partition => INT32
        size => INT64
        offset_lag => INT64
        is_future => BOOLEAN
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
log_dirsnull
error_codeResponse error code
log_dirThe absolute log directory path.
topicsnull
topicName of topic
partitionsnull
partitionTopic partition id
sizeThe size of the log segments of the partition in bytes.
offset_lagThe lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or current replica's LEO (if it is the future log for the partition)
is_futureTrue if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.

SaslAuthenticate API (Key: 36):
Requests:

SaslAuthenticate Request (Version: 0) => sasl_auth_bytes 
  sasl_auth_bytes => BYTES
FieldDescription
sasl_auth_bytesSASL authentication bytes from client as defined by the SASL mechanism.

SaslAuthenticate Request (Version: 1) => sasl_auth_bytes 
  sasl_auth_bytes => BYTES
FieldDescription
sasl_auth_bytesSASL authentication bytes from client as defined by the SASL mechanism.

Responses:

SaslAuthenticate Response (Version: 0) => error_code error_message sasl_auth_bytes 
  error_code => INT16
  error_message => NULLABLE_STRING
  sasl_auth_bytes => BYTES
FieldDescription
error_codeResponse error code
error_messageResponse error message
sasl_auth_bytesSASL authentication bytes from server as defined by the SASL mechanism.

SaslAuthenticate Response (Version: 1) => error_code error_message sasl_auth_bytes session_lifetime_ms 
  error_code => INT16
  error_message => NULLABLE_STRING
  sasl_auth_bytes => BYTES
  session_lifetime_ms => INT64
FieldDescription
error_codeResponse error code
error_messageResponse error message
sasl_auth_bytesSASL authentication bytes from server as defined by the SASL mechanism.
session_lifetime_msNumber of milliseconds after which only re-authentication over the existing connection to create a new session can occur.

CreatePartitions API (Key: 37):
Requests:

CreatePartitions Request (Version: 0) => [topic_partitions] timeout validate_only 
  topic_partitions => topic new_partitions 
    topic => STRING
    new_partitions => count [assignment] 
      count => INT32
      assignment => ARRAY(INT32)
  timeout => INT32
  validate_only => BOOLEAN
FieldDescription
topic_partitionsList of topic and the corresponding new partitions.
topicName of topic
new_partitionsnull
countThe new partition count.
assignmentThe assigned brokers.
timeoutThe time in ms to wait for the partitions to be created.
validate_onlyIf true then validate the request, but don't actually increase the number of partitions.

CreatePartitions Request (Version: 1) => [topic_partitions] timeout validate_only 
  topic_partitions => topic new_partitions 
    topic => STRING
    new_partitions => count [assignment] 
      count => INT32
      assignment => ARRAY(INT32)
  timeout => INT32
  validate_only => BOOLEAN
FieldDescription
topic_partitionsList of topic and the corresponding new partitions.
topicName of topic
new_partitionsnull
countThe new partition count.
assignmentThe assigned brokers.
timeoutThe time in ms to wait for the partitions to be created.
validate_onlyIf true then validate the request, but don't actually increase the number of partitions.

Responses:

CreatePartitions Response (Version: 0) => throttle_time_ms [topic_errors] 
  throttle_time_ms => INT32
  topic_errors => topic error_code error_message 
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_errorsPer topic results for the create partitions request
topicName of topic
error_codeResponse error code
error_messageResponse error message

CreatePartitions Response (Version: 1) => throttle_time_ms [topic_errors] 
  throttle_time_ms => INT32
  topic_errors => topic error_code error_message 
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topic_errorsPer topic results for the create partitions request
topicName of topic
error_codeResponse error code
error_messageResponse error message

CreateDelegationToken API (Key: 38):
Requests:

CreateDelegationToken Request (Version: 0) => [renewers] max_life_time 
  renewers => principal_type name 
    principal_type => STRING
    name => STRING
  max_life_time => INT64
FieldDescription
renewersAn array of token renewers. Renewer is an Kafka PrincipalType and name string, who is allowed to renew this token before the max lifetime expires.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
max_life_timeMax lifetime period for token in milli seconds. if value is -1, then max lifetime will default to a server side config value.

CreateDelegationToken Request (Version: 1) => [renewers] max_life_time 
  renewers => principal_type name 
    principal_type => STRING
    name => STRING
  max_life_time => INT64
FieldDescription
renewersAn array of token renewers. Renewer is an Kafka PrincipalType and name string, who is allowed to renew this token before the max lifetime expires.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
max_life_timeMax lifetime period for token in milli seconds. if value is -1, then max lifetime will default to a server side config value.

Responses:

CreateDelegationToken Response (Version: 0) => error_code owner issue_timestamp expiry_timestamp max_timestamp token_id hmac throttle_time_ms 
  error_code => INT16
  owner => principal_type name 
    principal_type => STRING
    name => STRING
  issue_timestamp => INT64
  expiry_timestamp => INT64
  max_timestamp => INT64
  token_id => STRING
  hmac => BYTES
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
ownertoken owner.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
issue_timestamptimestamp (in msec) when this token was generated.
expiry_timestamptimestamp (in msec) at which this token expires.
max_timestampmax life time of this token.
token_idUUID to ensure uniqueness.
hmacHMAC of the delegation token.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

CreateDelegationToken Response (Version: 1) => error_code owner issue_timestamp expiry_timestamp max_timestamp token_id hmac throttle_time_ms 
  error_code => INT16
  owner => principal_type name 
    principal_type => STRING
    name => STRING
  issue_timestamp => INT64
  expiry_timestamp => INT64
  max_timestamp => INT64
  token_id => STRING
  hmac => BYTES
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
ownertoken owner.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
issue_timestamptimestamp (in msec) when this token was generated.
expiry_timestamptimestamp (in msec) at which this token expires.
max_timestampmax life time of this token.
token_idUUID to ensure uniqueness.
hmacHMAC of the delegation token.
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

RenewDelegationToken API (Key: 39):
Requests:

RenewDelegationToken Request (Version: 0) => hmac renew_time_period 
  hmac => BYTES
  renew_time_period => INT64
FieldDescription
hmacHMAC of the delegation token to be renewed.
renew_time_periodRenew time period in milli seconds.

RenewDelegationToken Request (Version: 1) => hmac renew_time_period 
  hmac => BYTES
  renew_time_period => INT64
FieldDescription
hmacHMAC of the delegation token to be renewed.
renew_time_periodRenew time period in milli seconds.

Responses:

RenewDelegationToken Response (Version: 0) => error_code expiry_timestamp throttle_time_ms 
  error_code => INT16
  expiry_timestamp => INT64
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
expiry_timestamptimestamp (in msec) at which this token expires..
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

RenewDelegationToken Response (Version: 1) => error_code expiry_timestamp throttle_time_ms 
  error_code => INT16
  expiry_timestamp => INT64
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
expiry_timestamptimestamp (in msec) at which this token expires..
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

ExpireDelegationToken API (Key: 40):
Requests:

ExpireDelegationToken Request (Version: 0) => hmac expiry_time_period 
  hmac => BYTES
  expiry_time_period => INT64
FieldDescription
hmacHMAC of the delegation token to be expired.
expiry_time_periodexpiry time period in milli seconds.

ExpireDelegationToken Request (Version: 1) => hmac expiry_time_period 
  hmac => BYTES
  expiry_time_period => INT64
FieldDescription
hmacHMAC of the delegation token to be expired.
expiry_time_periodexpiry time period in milli seconds.

Responses:

ExpireDelegationToken Response (Version: 0) => error_code expiry_timestamp throttle_time_ms 
  error_code => INT16
  expiry_timestamp => INT64
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
expiry_timestamptimestamp (in msec) at which this token expires..
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

ExpireDelegationToken Response (Version: 1) => error_code expiry_timestamp throttle_time_ms 
  error_code => INT16
  expiry_timestamp => INT64
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
expiry_timestamptimestamp (in msec) at which this token expires..
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

DescribeDelegationToken API (Key: 41):
Requests:

DescribeDelegationToken Request (Version: 0) => [owners] 
  owners => principal_type name 
    principal_type => STRING
    name => STRING
FieldDescription
ownersAn array of token owners.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal

DescribeDelegationToken Request (Version: 1) => [owners] 
  owners => principal_type name 
    principal_type => STRING
    name => STRING
FieldDescription
ownersAn array of token owners.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal

Responses:

DescribeDelegationToken Response (Version: 0) => error_code [token_details] throttle_time_ms 
  error_code => INT16
  token_details => owner issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] 
    owner => principal_type name 
      principal_type => STRING
      name => STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => STRING
    hmac => BYTES
    renewers => principal_type name 
      principal_type => STRING
      name => STRING
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
token_detailsnull
ownertoken owner.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
issue_timestamptimestamp (in msec) when this token was generated.
expiry_timestamptimestamp (in msec) at which this token expires.
max_timestampmax life time of this token.
token_idUUID to ensure uniqueness.
hmacHMAC of the delegation token to be expired.
renewersAn array of token renewers. Renewer is an Kafka PrincipalType and name string, who is allowed to renew this token before the max lifetime expires.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

DescribeDelegationToken Response (Version: 1) => error_code [token_details] throttle_time_ms 
  error_code => INT16
  token_details => owner issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] 
    owner => principal_type name 
      principal_type => STRING
      name => STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => STRING
    hmac => BYTES
    renewers => principal_type name 
      principal_type => STRING
      name => STRING
  throttle_time_ms => INT32
FieldDescription
error_codeResponse error code
token_detailsnull
ownertoken owner.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
issue_timestamptimestamp (in msec) when this token was generated.
expiry_timestamptimestamp (in msec) at which this token expires.
max_timestampmax life time of this token.
token_idUUID to ensure uniqueness.
hmacHMAC of the delegation token to be expired.
renewersAn array of token renewers. Renewer is an Kafka PrincipalType and name string, who is allowed to renew this token before the max lifetime expires.
principal_typeprincipalType of the Kafka principal
namename of the Kafka principal
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)

DeleteGroups API (Key: 42):
Requests:

DeleteGroups Request (Version: 0) => [groups] 
  groups => STRING
FieldDescription
groupsAn array of groups to be deleted.

DeleteGroups Request (Version: 1) => [groups] 
  groups => STRING
FieldDescription
groupsAn array of groups to be deleted.

Responses:

DeleteGroups Response (Version: 0) => throttle_time_ms [group_error_codes] 
  throttle_time_ms => INT32
  group_error_codes => group_id error_code 
    group_id => STRING
    error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
group_error_codesAn array of per group error codes.
group_idThe unique group identifier
error_codeResponse error code

DeleteGroups Response (Version: 1) => throttle_time_ms [group_error_codes] 
  throttle_time_ms => INT32
  group_error_codes => group_id error_code 
    group_id => STRING
    error_code => INT16
FieldDescription
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
group_error_codesAn array of per group error codes.
group_idThe unique group identifier
error_codeResponse error code

ElectPreferredLeaders API (Key: 43):
Requests:

ElectPreferredLeaders Request (Version: 0) => [topic_partitions] timeout_ms 
  topic_partitions => topic [partition_id] 
    topic => STRING
    partition_id => INT32
  timeout_ms => INT32
FieldDescription
topic_partitionsThe topic partitions to elect the preferred leader of.
topicThe name of a topic.
partition_idThe partitions of this topic whose preferred leader should be elected
timeout_msThe time in ms to wait for the election to complete.

Responses:

ElectPreferredLeaders Response (Version: 0) => throttle_time_ms [replica_election_results] 
  throttle_time_ms => INT32
  replica_election_results => topic [partition_result] 
    topic => STRING
    partition_result => partition_id error_code error_message 
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING
FieldDescription
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
replica_election_resultsThe error code, or 0 if there was no error.
topicThe topic name
partition_resultThe results for each partition
partition_idThe partition id
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.

Some Common Philosophical Questions

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.

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.

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.

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.