This is the multi-page printable view of this section. Click here to print.
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 disk 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:
- The memory overhead of objects is very high, often doubling the size of the data stored (or worse).
- 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 then 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 deleting messages as soon as they are consumed, we can retain messages for a relative 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:
- The operating system reads data from the disk into pagecache in kernel space
- The application reads the data from kernel space into a user-space buffer
- The application writes the data back into kernel space into a socket buffer
- 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 kernel 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 by allowing recursive message sets. 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 and Snappy 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 appropriate 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 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 structure 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 one consumer at any given time. This means that the position of 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, or 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 alive 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.
These are not the strongest possible semantics for publishers. Although we cannot be sure of what happened in the case of a network error, it is possible to allow the producer to generate a sort of “primary key” that makes retrying the produce request idempotent. This feature is not trivial for a replicated system because of course it must work even (or especially) in the case of a server failure. With this feature it would suffice for the producer to retry until it receives acknowledgement of a successfully committed message at which point we would guarantee the message had been published exactly once. We hope to add this in a future Kafka version.
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.
- 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.
- 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)? The limitation here is not actually a feature of the messaging system but rather the need to co-ordinate 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 for the consumer position and the storage of the consumers output. But this can be handled more simply and generally by simply 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, our Hadoop ETL that populates data in HDFS stores its offsets in HDFS with 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 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 its offset prior to processing a batch of messages. Exactly-once delivery requires co-operation with the destination storage system but Kafka provides the offset which makes implementing this straight-forward.
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: slaves 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
- A node must be able to maintain its session with ZooKeeper (via ZooKeeper’s heartbeat mechanism)
- If it is a slave 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).
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 request.required.acks setting that the producer uses.
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 slave 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:
- 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).
- 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. In our current release we choose the second strategy and favor choosing a potentially inconsistent replica when all replicas in the ISR are dead. In the future, we would like to make this configurable to better support use cases where downtime is preferable to inconsistency.
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 request.required.acks=-1, 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 request.required.acks=-1 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:
- 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.
- 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 required.acks=-1 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.
- 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.
- 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.
- Journaling for high-availability. A process that does local computation can be made fault-tolerant by logging out changes that it makes to it’s 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 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:
- 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.
- Ordering of messages is always maintained. Compaction will never re-order messages, just remove some.
- The offset for a message never changes. It is the permanent identifier for a position in the log.
- 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. 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). This is important as delete marker removal happens concurrently with read, and thus it is important that we do not remove any delete marker prior to the consumer seeing it.
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:
- It chooses the log that has the highest ratio of log head to log tail
- It creates a succinct summary of the last offset for each key in the head of the log
- 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).
- 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 disabled by default. To enable it set the server config
log.cleaner.enable=true
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.
Further cleaner configurations are described here.
Log Compaction Limitations
- You cannot configure yet how much log is retained without compaction (the “head” of the log). Currently all segments are eligible except for the last segment, i.e. the one currently being written to.
- Log compaction is not yet compatible with compressed topics.
Quotas
Starting in 0.9, the Kafka cluster has the ability to enforce quotas on produce and fetch requests. Quotas are basically byte-rate thresholds defined per client-id. A client-id logically identifies an application making a request. Hence a single client-id can span multiple producer and consumer instances and the quota will apply for all of them as a single entity i.e. if client-id=“test-client” has a produce quota of 10MB/sec, this is shared across all instances with that same id.
Why are quotas necessary?
It is possible for producers and consumers to produce/consume very high volumes of data 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 tbe 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.
Enforcement
By default, each unique client-id receives a fixed quota in bytes/sec as configured by the cluster (quota.producer.default, quota.consumer.default). This quota is defined on a per-broker basis. Each client can publish/fetch a maximum of X bytes/sec 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 does not return an error rather it attempts to slow down a client exceeding its quota. It computes the amount of delay needed to bring a guilty client under it’s quota and delays the response for that time. This approach keeps the quota violation transparent to clients (outside of client side metrics). This also keeps them from having to implement any special backoff and retry behavior which can get tricky. In fact, bad client behavior (retry without backoff) can exacerbate the very problem quotas are trying to solve.
Client byte rate is measured over multiple small windows (for 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.
Quota overrides
It is possible to override the default quota for client-ids that need a higher (or even lower) quota. The mechanism is similar to the per-topic log config overrides. Client-id overrides are written to ZooKeeper 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.
2 - Protocol
Kafka Wire Protocol
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
- Message Sets
- 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.
- Cycle through a list of “bootstrap” kafka urls until we find one we can connect to. Fetch cluster metadata.
- Process fetch or produce requests, directing them to the appropriate broker based on the topic/partitions they send to or fetch from.
- 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:
- It balances data and request load over brokers
- 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 would implement a particular version of the protocol, and indicate this version in their requests. Our goal is primarily to allow API evolution in an environment where downtime is not allowed and clients and servers cannot all be changed at once.
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.
Currently all versions are baselined at 0, as we evolve these APIs we will indicate the format for each version individually.
The Protocol
Protocol Primitive Types
The protocol is built out of the following primitive types.
Fixed Width Primitives
int8, int16, int32, int64 - Signed integers with the given precision (in bits) stored in big endian order.
Variable Length Primitives
bytes, string - These types consist of a signed integer giving a length N followed by N bytes of content. A length of -1 indicates null. string uses an int16 for its size, and bytes uses an int32.
Arrays
This is a notation for handling repeated structures. These will always be encoded as an int32 size containing the length N followed by N repetitions of the structure which can itself be made up of other primitive types. In the BNF grammars below we will show an array of a structure foo as [foo].
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
Field | Description |
---|---|
message_size | The message_size field gives the size of the subsequent request or response message in bytes. The client can read requests by first reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes of the request. |
Message Sets
A description of the message set format can be found here. (KAFKA-3368)
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:
Error | Code | Retriable | Description |
---|---|---|---|
UNKNOWN | -1 | False | The server experienced an unexpected error when processing the request |
NONE | 0 | False | |
OFFSET_OUT_OF_RANGE | 1 | False | The requested offset is not within the range of offsets maintained by the server. |
CORRUPT_MESSAGE | 2 | True | This message has failed its CRC checksum, exceeds the valid size, or is otherwise corrupt. |
UNKNOWN_TOPIC_OR_PARTITION | 3 | True | This server does not host this topic-partition. |
INVALID_FETCH_SIZE | 4 | False | The requested fetch size is invalid. |
LEADER_NOT_AVAILABLE | 5 | True | There is no leader for this topic-partition as we are in the middle of a leadership election. |
NOT_LEADER_FOR_PARTITION | 6 | True | This server is not the leader for that topic-partition. |
REQUEST_TIMED_OUT | 7 | True | The request timed out. |
BROKER_NOT_AVAILABLE | 8 | False | The broker is not available. |
REPLICA_NOT_AVAILABLE | 9 | False | The replica is not available for the requested topic-partition |
MESSAGE_TOO_LARGE | 10 | False | The request included a message larger than the max message size the server will accept. |
STALE_CONTROLLER_EPOCH | 11 | False | The controller moved to another broker. |
OFFSET_METADATA_TOO_LARGE | 12 | False | The metadata field of the offset request was too large. |
NETWORK_EXCEPTION | 13 | True | The server disconnected before a response was received. |
COORDINATOR_LOAD_IN_PROGRESS | 14 | True | The coordinator is loading and hence can't process requests. |
COORDINATOR_NOT_AVAILABLE | 15 | True | The coordinator is not available. |
NOT_COORDINATOR | 16 | True | This is not the correct coordinator. |
INVALID_TOPIC_EXCEPTION | 17 | False | The request attempted to perform an operation on an invalid topic. |
RECORD_LIST_TOO_LARGE | 18 | False | The request included message batch larger than the configured segment size on the server. |
NOT_ENOUGH_REPLICAS | 19 | True | Messages are rejected since there are fewer in-sync replicas than required. |
NOT_ENOUGH_REPLICAS_AFTER_APPEND | 20 | True | Messages are written to the log, but to fewer in-sync replicas than required. |
INVALID_REQUIRED_ACKS | 21 | False | Produce request specified an invalid value for required acks. |
ILLEGAL_GENERATION | 22 | False | Specified group generation id is not valid. |
INCONSISTENT_GROUP_PROTOCOL | 23 | False | The group member's supported protocols are incompatible with those of existing members. |
INVALID_GROUP_ID | 24 | False | The configured groupId is invalid |
UNKNOWN_MEMBER_ID | 25 | False | The coordinator is not aware of this member. |
INVALID_SESSION_TIMEOUT | 26 | False | The 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_PROGRESS | 27 | False | The group is rebalancing, so a rejoin is needed. |
INVALID_COMMIT_OFFSET_SIZE | 28 | False | The committing offset data size is not valid |
TOPIC_AUTHORIZATION_FAILED | 29 | False | Not authorized to access topics: [Topic authorization failed.] |
GROUP_AUTHORIZATION_FAILED | 30 | False | Not authorized to access group: Group authorization failed. |
CLUSTER_AUTHORIZATION_FAILED | 31 | False | Cluster authorization failed. |
INVALID_TIMESTAMP | 32 | False | The timestamp of the message is out of acceptable range. |
UNSUPPORTED_SASL_MECHANISM | 33 | False | The broker does not support the requested SASL mechanism. |
ILLEGAL_SASL_STATE | 34 | False | Request is not valid given the current SASL state. |
UNSUPPORTED_VERSION | 35 | False | The version of API is not supported. |
TOPIC_ALREADY_EXISTS | 36 | False | Topic with this name already exists. |
INVALID_PARTITIONS | 37 | False | Number of partitions is invalid. |
INVALID_REPLICATION_FACTOR | 38 | False | Replication-factor is invalid. |
INVALID_REPLICA_ASSIGNMENT | 39 | False | Replica assignment is invalid. |
INVALID_CONFIG | 40 | False | Configuration is invalid. |
NOT_CONTROLLER | 41 | True | This is not the correct controller for this cluster. |
INVALID_REQUEST | 42 | False | This 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_FORMAT | 43 | False | The message format version on the broker does not support the request. |
POLICY_VIOLATION | 44 | False | Request parameters do not satisfy the configured policy. |
OUT_OF_ORDER_SEQUENCE_NUMBER | 45 | False | The broker received an out of order sequence number |
DUPLICATE_SEQUENCE_NUMBER | 46 | True | The broker received a duplicate sequence number |
INVALID_PRODUCER_EPOCH | 47 | False | Producer 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_STATE | 48 | False | The producer attempted a transactional operation in an invalid state |
INVALID_PRODUCER_ID_MAPPING | 49 | False | The producer attempted to use a producer id which is not currently assigned to its transactional id |
INVALID_TRANSACTION_TIMEOUT | 50 | False | The transaction timeout is larger than the maximum value allowed by the broker (as configured by max.transaction.timeout.ms). |
CONCURRENT_TRANSACTIONS | 51 | False | The producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing |
TRANSACTION_COORDINATOR_FENCED | 52 | False | Indicates that the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer |
TRANSACTIONAL_ID_AUTHORIZATION_FAILED | 53 | False | Transactional Id authorization failed |
SECURITY_DISABLED | 54 | False | Security features are disabled. |
OPERATION_NOT_ATTEMPTED | 55 | False | The 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. |
Api Keys
The following are the numeric codes that the ApiKey in the request can take for each of the below request types.
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
Field | Description |
---|---|
api_key | The id of the request type. |
api_version | The version of the API. |
correlation_id | A user-supplied integer value that will be passed back with the response |
client_id | A user specified identifier for the client making the request. |
Response Header => correlation_id
correlation_id => INT32
Field | Description |
---|---|
correlation_id | The 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
Field | Description |
---|---|
acks | The 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. |
timeout | The time to await a response in ms. |
topic_data | |
topic | |
data | |
partition | |
record_set |
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
Field | Description |
---|---|
acks | The 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. |
timeout | The time to await a response in ms. |
topic_data | |
topic | |
data | |
partition | |
record_set |
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
Field | Description |
---|---|
acks | The 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. |
timeout | The time to await a response in ms. |
topic_data | |
topic | |
data | |
partition | |
record_set |
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
Field | Description |
---|---|
transactional_id | The transactional ID of the producer. This is used to authorize transaction produce requests. This can be null for non-transactional producers. |
acks | The 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. |
timeout | The time to await a response in ms. |
topic_data | |
topic | |
data | |
partition | |
record_set |
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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | |
error_code | |
base_offset |
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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | |
error_code | |
base_offset | |
throttle_time_ms | Duration 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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | |
error_code | |
base_offset | |
log_append_time | The 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_ms | Duration 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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | |
error_code | |
base_offset | |
log_append_time | The 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_ms | Duration 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 max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
topics | Topics to fetch. |
topic | Topic to fetch. |
partitions | Partitions to fetch. |
partition | Topic partition id. |
fetch_offset | Message offset. |
max_bytes | Maximum 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 max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
topics | Topics to fetch. |
topic | Topic to fetch. |
partitions | Partitions to fetch. |
partition | Topic partition id. |
fetch_offset | Message offset. |
max_bytes | Maximum 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 max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
topics | Topics to fetch. |
topic | Topic to fetch. |
partitions | Partitions to fetch. |
partition | Topic partition id. |
fetch_offset | Message offset. |
max_bytes | Maximum 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 max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum 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. |
topics | Topics to fetch in the order provided. |
topic | Topic to fetch. |
partitions | Partitions to fetch. |
partition | Topic partition id. |
fetch_offset | Message offset. |
max_bytes | Maximum 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 max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum 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_level | This 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 |
topics | Topics to fetch in the order provided. |
topic | Topic to fetch. |
partitions | Partitions to fetch. |
partition | Topic partition id. |
fetch_offset | Message offset. |
max_bytes | Maximum 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 max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
max_bytes => INT32
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum 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_level | This 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 |
topics | Topics to fetch in the order provided. |
topic | Topic to fetch. |
partitions | Partitions to fetch. |
partition | Topic partition id. |
fetch_offset | Message offset. |
log_start_offset | Earliest available offset of the follower replica. The field is only used when request is sent by follower. |
max_bytes | Maximum bytes to fetch. |
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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition_header | |
partition | Topic partition id. |
error_code | |
high_watermark | Last committed offset. |
record_set |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition_header | |
partition | Topic partition id. |
error_code | |
high_watermark | Last committed offset. |
record_set |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition_header | |
partition | Topic partition id. |
error_code | |
high_watermark | Last committed offset. |
record_set |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition_header | |
partition | Topic partition id. |
error_code | |
high_watermark | Last committed offset. |
record_set |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition_header | |
partition | Topic partition id. |
error_code | |
high_watermark | Last committed offset. |
last_stable_offset | The 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_transactions | |
producer_id | The producer id associated with the aborted transactions |
first_offset | The first offset in the aborted transaction |
record_set |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition_header | |
partition | Topic partition id. |
error_code | |
high_watermark | Last committed offset. |
last_stable_offset | The 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_offset | Earliest available offset. |
aborted_transactions | |
producer_id | The producer id associated with the aborted transactions |
first_offset | The first offset in the aborted transaction |
record_set |
Offsets API (Key: 2):
Requests:Offsets 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
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
topics | Topics to list offsets. |
topic | Topic to list offset. |
partitions | Partitions to list offset. |
partition | Topic partition id. |
timestamp | Timestamp. |
max_num_offsets | Maximum offsets to return. |
Offsets Request (Version: 1) => replica_id [topics]
replica_id => INT32
topics => topic [partitions]
topic => STRING
partitions => partition timestamp
partition => INT32
timestamp => INT64
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
topics | Topics to list offsets. |
topic | Topic to list offset. |
partitions | Partitions to list offset. |
partition | Topic partition id. |
timestamp | The target timestamp for the partition. |
Offsets 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
Field | Description |
---|---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
isolation_level | This 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 |
topics | Topics to list offsets. |
topic | Topic to list offset. |
partitions | Partitions to list offset. |
partition | Topic partition id. |
timestamp | The target timestamp for the partition. |
Offsets Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code [offsets]
partition => INT32
error_code => INT16
offsets => INT64
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
error_code | |
offsets | A list of offsets. |
Offsets 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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
error_code | |
timestamp | The timestamp associated with the returned offset |
offset | offset found |
Offsets 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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
error_code | |
timestamp | The timestamp associated with the returned offset |
offset | offset found |
Metadata API (Key: 3):
Requests:Metadata Request (Version: 0) => [topics]
topics => STRING
Field | Description |
---|---|
topics | An array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics. |
Metadata Request (Version: 1) => [topics]
topics => STRING
Field | Description |
---|---|
topics | An 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
Field | Description |
---|---|
topics | An 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
Field | Description |
---|---|
topics | An 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
Field | Description |
---|---|
topics | An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics. |
allow_auto_topic_creation | If 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 Response (Version: 0) => [brokers] [topic_metadata]
brokers => node_id host port
node_id => INT32
host => STRING
port => INT32
topic_metadata => topic_error_code topic [partition_metadata]
topic_error_code => INT16
topic => STRING
partition_metadata => partition_error_code partition_id leader [replicas] [isr]
partition_error_code => INT16
partition_id => INT32
leader => INT32
replicas => INT32
isr => INT32
Field | Description |
---|---|
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
topic_metadata | |
topic_error_code | The error code for the given topic. |
topic | The name of the topic |
partition_metadata | Metadata for each partition of the topic. |
partition_error_code | The error code for the partition, if any. |
partition_id | The id of the partition. |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The 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 => topic_error_code topic is_internal [partition_metadata]
topic_error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => partition_error_code partition_id leader [replicas] [isr]
partition_error_code => INT16
partition_id => INT32
leader => INT32
replicas => INT32
isr => INT32
Field | Description |
---|---|
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
controller_id | The broker id of the controller broker. |
topic_metadata | |
topic_error_code | The error code for the given topic. |
topic | The name of the topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
partition_error_code | The error code for the partition, if any. |
partition_id | The id of the partition. |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The 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 => topic_error_code topic is_internal [partition_metadata]
topic_error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => partition_error_code partition_id leader [replicas] [isr]
partition_error_code => INT16
partition_id => INT32
leader => INT32
replicas => INT32
isr => INT32
Field | Description |
---|---|
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | |
topic_error_code | The error code for the given topic. |
topic | The name of the topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
partition_error_code | The error code for the partition, if any. |
partition_id | The id of the partition. |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The 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 => topic_error_code topic is_internal [partition_metadata]
topic_error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => partition_error_code partition_id leader [replicas] [isr]
partition_error_code => INT16
partition_id => INT32
leader => INT32
replicas => INT32
isr => INT32
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | |
topic_error_code | The error code for the given topic. |
topic | The name of the topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
partition_error_code | The error code for the partition, if any. |
partition_id | The id of the partition. |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The 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 => topic_error_code topic is_internal [partition_metadata]
topic_error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => partition_error_code partition_id leader [replicas] [isr]
partition_error_code => INT16
partition_id => INT32
leader => INT32
replicas => INT32
isr => INT32
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | |
topic_error_code | The error code for the given topic. |
topic | The name of the topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
partition_error_code | The error code for the partition, if any. |
partition_id | The id of the partition. |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for 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
Field | Description |
---|---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | |
topic | Topic name. |
partition | Topic partition id. |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_leaders | |
id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
LeaderAndIsr Response (Version: 0) => error_code [partitions]
error_code => INT16
partitions => topic partition error_code
topic => STRING
partition => INT32
error_code => INT16
Field | Description |
---|---|
error_code | Error code. |
partitions | |
topic | Topic name. |
partition | Topic partition id. |
error_code | 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
Field | Description |
---|---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
delete_partitions | Boolean which indicates if replica's partitions must be deleted. |
partitions | |
topic | Topic name. |
partition | Topic partition id. |
StopReplica Response (Version: 0) => error_code [partitions]
error_code => INT16
partitions => topic partition error_code
topic => STRING
partition => INT32
error_code => INT16
Field | Description |
---|---|
error_code | Error code. |
partitions | |
topic | Topic name. |
partition | Topic partition id. |
error_code | 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
Field | Description |
---|---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | |
topic | Topic name. |
partition | Topic partition id. |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | |
id | The broker id. |
host | The hostname of the broker. |
port | The 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
Field | Description |
---|---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | |
topic | Topic name. |
partition | Topic partition id. |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | |
id | The broker id. |
end_points | |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
security_protocol_type | The 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
Field | Description |
---|---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | |
topic | Topic name. |
partition | Topic partition id. |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | |
id | The broker id. |
end_points | |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
security_protocol_type | The security protocol type. |
rack | The 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
Field | Description |
---|---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | |
topic | Topic name. |
partition | Topic partition id. |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | |
id | The broker id. |
end_points | |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
listener_name | The listener name. |
security_protocol_type | The security protocol type. |
rack | The rack |
UpdateMetadata Response (Version: 0) => error_code
error_code => INT16
Field | Description |
---|---|
error_code | Error code. |
UpdateMetadata Response (Version: 1) => error_code
error_code => INT16
Field | Description |
---|---|
error_code | Error code. |
UpdateMetadata Response (Version: 2) => error_code
error_code => INT16
Field | Description |
---|---|
error_code | Error code. |
UpdateMetadata Response (Version: 3) => error_code
error_code => INT16
Field | Description |
---|---|
error_code | Error code. |
ControlledShutdown API (Key: 7):
Requests:ControlledShutdown Request (Version: 1) => broker_id
broker_id => INT32
Field | Description |
---|---|
broker_id | The id of the broker for which controlled shutdown has been requested. |
ControlledShutdown Response (Version: 1) => error_code [partitions_remaining]
error_code => INT16
partitions_remaining => topic partition
topic => STRING
partition => INT32
Field | Description |
---|---|
error_code | |
partitions_remaining | The partitions that the broker still leads. |
topic | |
partition | Topic 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
Field | Description |
---|---|
group_id | The group id. |
topics | Topics to commit offsets. |
topic | Topic to commit. |
partitions | Partitions to commit offsets. |
partition | Topic partition id. |
offset | Message offset to be committed. |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 1) => group_id group_generation_id member_id [topics]
group_id => STRING
group_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
Field | Description |
---|---|
group_id | The group id. |
group_generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator. |
topics | Topics to commit offsets. |
topic | Topic to commit. |
partitions | Partitions to commit offsets. |
partition | Topic partition id. |
offset | Message offset to be committed. |
timestamp | Timestamp of the commit |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 2) => group_id group_generation_id member_id retention_time [topics]
group_id => STRING
group_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
Field | Description |
---|---|
group_id | The group id. |
group_generation_id | The generation of the consumer group. |
member_id | The consumer id assigned by the group coordinator. |
retention_time | Time period in ms to retain the offset. |
topics | Topics to commit offsets. |
topic | Topic to commit. |
partitions | Partitions to commit offsets. |
partition | Topic partition id. |
offset | Message offset to be committed. |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 3) => group_id group_generation_id member_id retention_time [topics]
group_id => STRING
group_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
Field | Description |
---|---|
group_id | The group id. |
group_generation_id | The generation of the consumer group. |
member_id | The consumer id assigned by the group coordinator. |
retention_time | Time period in ms to retain the offset. |
topics | Topics to commit offsets. |
topic | Topic to commit. |
partitions | Partitions to commit offsets. |
partition | Topic partition id. |
offset | Message offset to be committed. |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
error_code |
OffsetCommit Response (Version: 1) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
error_code |
OffsetCommit Response (Version: 2) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
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
Field | Description |
---|---|
group_id | The consumer group id. |
topics | Topics to fetch offsets. |
topic | Topic to fetch offset. |
partitions | Partitions to fetch offsets. |
partition | Topic partition id. |
OffsetFetch Request (Version: 1) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field | Description |
---|---|
group_id | The consumer group id. |
topics | Topics to fetch offsets. |
topic | Topic to fetch offset. |
partitions | Partitions to fetch offsets. |
partition | Topic partition id. |
OffsetFetch Request (Version: 2) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field | Description |
---|---|
group_id | The consumer group id. |
topics | Topics to fetch offsets. If the topic array is null fetch offsets for all topics. |
topic | Topic to fetch offset. |
partitions | Partitions to fetch offsets. |
partition | Topic partition id. |
OffsetFetch Request (Version: 3) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field | Description |
---|---|
group_id | The consumer group id. |
topics | Topics to fetch offsets. If the topic array is null fetch offsets for all topics. |
topic | Topic to fetch offset. |
partitions | Partitions to fetch offsets. |
partition | Topic partition id. |
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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
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
Field | Description |
---|---|
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
error_code | |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
responses | |
topic | |
partition_responses | |
partition | Topic partition id. |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
error_code | |
error_code |
FindCoordinator API (Key: 10):
Requests:FindCoordinator Request (Version: 0) => group_id
group_id => STRING
Field | Description |
---|---|
group_id | The unique group id. |
FindCoordinator Request (Version: 1) => coordinator_key coordinator_type
coordinator_key => STRING
coordinator_type => INT8
Field | Description |
---|---|
coordinator_key | Id to use for finding the coordinator (for groups, this is the groupId, for transactional producers, this is the transactional id) |
coordinator_type | The type of coordinator to find (0 = group, 1 = transaction) |
FindCoordinator Response (Version: 0) => error_code coordinator
error_code => INT16
coordinator => node_id host port
node_id => INT32
host => STRING
port => INT32
Field | Description |
---|---|
error_code | |
coordinator | Host and port information for the coordinator for a consumer group. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The 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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | |
error_message | |
coordinator | Host and port information for the coordinator for a consumer group. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The 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
Field | Description |
---|---|
group_id | The group id. |
session_timeout | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms. |
member_id | The assigned consumer id or an empty string for a new consumer. |
protocol_type | Unique name for class of protocols implemented by group |
group_protocols | List of protocols that the member supports |
protocol_name | |
protocol_metadata |
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
Field | Description |
---|---|
group_id | The group id. |
session_timeout | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms. |
rebalance_timeout | The maximum time that the coordinator will wait for each member to rejoin when rebalancing the group |
member_id | The assigned consumer id or an empty string for a new consumer. |
protocol_type | Unique name for class of protocols implemented by group |
group_protocols | List of protocols that the member supports |
protocol_name | |
protocol_metadata |
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
Field | Description |
---|---|
group_id | The group id. |
session_timeout | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms. |
rebalance_timeout | The maximum time that the coordinator will wait for each member to rejoin when rebalancing the group |
member_id | The assigned consumer id or an empty string for a new consumer. |
protocol_type | Unique name for class of protocols implemented by group |
group_protocols | List of protocols that the member supports |
protocol_name | |
protocol_metadata |
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
Field | Description |
---|---|
error_code | |
generation_id | The generation of the consumer group. |
group_protocol | The group protocol selected by the coordinator |
leader_id | The leader of the group |
member_id | The consumer id assigned by the group coordinator. |
members | |
member_id | |
member_metadata |
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
Field | Description |
---|---|
error_code | |
generation_id | The generation of the consumer group. |
group_protocol | The group protocol selected by the coordinator |
leader_id | The leader of the group |
member_id | The consumer id assigned by the group coordinator. |
members | |
member_id | |
member_metadata |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | |
generation_id | The generation of the consumer group. |
group_protocol | The group protocol selected by the coordinator |
leader_id | The leader of the group |
member_id | The consumer id assigned by the group coordinator. |
members | |
member_id | |
member_metadata |
Heartbeat API (Key: 12):
Requests:Heartbeat Request (Version: 0) => group_id group_generation_id member_id
group_id => STRING
group_generation_id => INT32
member_id => STRING
Field | Description |
---|---|
group_id | The group id. |
group_generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator. |
Heartbeat Request (Version: 1) => group_id group_generation_id member_id
group_id => STRING
group_generation_id => INT32
member_id => STRING
Field | Description |
---|---|
group_id | The group id. |
group_generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator. |
Heartbeat Response (Version: 0) => error_code
error_code => INT16
Field | Description |
---|---|
error_code |
Heartbeat Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code |
LeaveGroup API (Key: 13):
Requests:LeaveGroup Request (Version: 0) => group_id member_id
group_id => STRING
member_id => STRING
Field | Description |
---|---|
group_id | The group id. |
member_id | The member id assigned by the group coordinator. |
LeaveGroup Request (Version: 1) => group_id member_id
group_id => STRING
member_id => STRING
Field | Description |
---|---|
group_id | The group id. |
member_id | The member id assigned by the group coordinator. |
LeaveGroup Response (Version: 0) => error_code
error_code => INT16
Field | Description |
---|---|
error_code |
LeaveGroup Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code |
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
Field | Description |
---|---|
group_id | |
generation_id | |
member_id | |
group_assignment | |
member_id | |
member_assignment |
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
Field | Description |
---|---|
group_id | |
generation_id | |
member_id | |
group_assignment | |
member_id | |
member_assignment |
SyncGroup Response (Version: 0) => error_code member_assignment
error_code => INT16
member_assignment => BYTES
Field | Description |
---|---|
error_code | |
member_assignment |
SyncGroup Response (Version: 1) => throttle_time_ms error_code member_assignment
throttle_time_ms => INT32
error_code => INT16
member_assignment => BYTES
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | |
member_assignment |
DescribeGroups API (Key: 15):
Requests:DescribeGroups Request (Version: 0) => [group_ids]
group_ids => STRING
Field | Description |
---|---|
group_ids | List of groupIds to request metadata for (an empty groupId array will return empty group metadata). |
DescribeGroups Request (Version: 1) => [group_ids]
group_ids => STRING
Field | Description |
---|---|
group_ids | List of groupIds to request metadata for (an empty groupId array will return empty group metadata). |
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
Field | Description |
---|---|
groups | |
error_code | |
group_id | |
state | The current state of the group (one of: Dead, Stable, AwaitingSync, PreparingRebalance, or empty if there is no active group) |
protocol_type | The current group protocol type (will be empty if there is no active group) |
protocol | The current group protocol (only provided if the group is Stable) |
members | Current group members (only provided if the group is not Dead) |
member_id | The memberId assigned by the coordinator |
client_id | The client id used in the member's latest join group request |
client_host | The client host used in the request session corresponding to the member's join group. |
member_metadata | The metadata corresponding to the current group protocol in use (will only be present if the group is stable). |
member_assignment | The 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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
groups | |
error_code | |
group_id | |
state | The current state of the group (one of: Dead, Stable, AwaitingSync, PreparingRebalance, or empty if there is no active group) |
protocol_type | The current group protocol type (will be empty if there is no active group) |
protocol | The current group protocol (only provided if the group is Stable) |
members | Current group members (only provided if the group is not Dead) |
member_id | The memberId assigned by the coordinator |
client_id | The client id used in the member's latest join group request |
client_host | The client host used in the request session corresponding to the member's join group. |
member_metadata | The metadata corresponding to the current group protocol in use (will only be present if the group is stable). |
member_assignment | The 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) =>
Field | Description |
---|
ListGroups Request (Version: 1) =>
Field | Description |
---|
ListGroups Response (Version: 0) => error_code [groups]
error_code => INT16
groups => group_id protocol_type
group_id => STRING
protocol_type => STRING
Field | Description |
---|---|
error_code | |
groups | |
group_id | |
protocol_type |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | |
groups | |
group_id | |
protocol_type |
SaslHandshake API (Key: 17):
Requests:SaslHandshake Request (Version: 0) => mechanism
mechanism => STRING
Field | Description |
---|---|
mechanism | SASL Mechanism chosen by the client. |
SaslHandshake Response (Version: 0) => error_code [enabled_mechanisms]
error_code => INT16
enabled_mechanisms => STRING
Field | Description |
---|---|
error_code | |
enabled_mechanisms | Array of mechanisms enabled in the server. |
ApiVersions API (Key: 18):
Requests:ApiVersions Request (Version: 0) =>
Field | Description |
---|
ApiVersions Request (Version: 1) =>
Field | Description |
---|
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
Field | Description |
---|---|
error_code | Error code. |
api_versions | API versions supported by the broker. |
api_key | API key. |
min_version | Minimum supported version. |
max_version | Maximum 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
Field | Description |
---|---|
error_code | Error code. |
api_versions | API versions supported by the broker. |
api_key | API key. |
min_version | Minimum supported version. |
max_version | Maximum supported version. |
throttle_time_ms | Duration 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_id [replicas]
partition_id => INT32
replicas => INT32
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
timeout => INT32
Field | Description |
---|---|
create_topic_requests | An array of single topic creation requests. Can not have multiple entries for the same topic. |
topic | Name for newly created topic. |
num_partitions | Number of partitions to be created. -1 indicates unset. |
replication_factor | Replication factor for the topic. -1 indicates unset. |
replica_assignment | Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset. |
partition_id | |
replicas | The set of all nodes that should host this partition. The first replica in the list is the preferred leader. |
config_entries | Topic level configuration for topic to be set. |
config_name | Configuration name |
config_value | Configuration value |
timeout | The 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_id [replicas]
partition_id => INT32
replicas => INT32
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
timeout => INT32
validate_only => BOOLEAN
Field | Description |
---|---|
create_topic_requests | An array of single topic creation requests. Can not have multiple entries for the same topic. |
topic | Name for newly created topic. |
num_partitions | Number of partitions to be created. -1 indicates unset. |
replication_factor | Replication factor for the topic. -1 indicates unset. |
replica_assignment | Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset. |
partition_id | |
replicas | The set of all nodes that should host this partition. The first replica in the list is the preferred leader. |
config_entries | Topic level configuration for topic to be set. |
config_name | Configuration name |
config_value | Configuration value |
timeout | The 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_only | If 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_id [replicas]
partition_id => INT32
replicas => INT32
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
timeout => INT32
validate_only => BOOLEAN
Field | Description |
---|---|
create_topic_requests | An array of single topic creation requests. Can not have multiple entries for the same topic. |
topic | Name for newly created topic. |
num_partitions | Number of partitions to be created. -1 indicates unset. |
replication_factor | Replication factor for the topic. -1 indicates unset. |
replica_assignment | Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset. |
partition_id | |
replicas | The set of all nodes that should host this partition. The first replica in the list is the preferred leader. |
config_entries | Topic level configuration for topic to be set. |
config_name | Configuration name |
config_value | Configuration value |
timeout | The 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_only | If this is true, the request will be validated, but the topic won't be created. |
CreateTopics Response (Version: 0) => [topic_errors]
topic_errors => topic error_code
topic => STRING
error_code => INT16
Field | Description |
---|---|
topic_errors | An array of per topic error codes. |
topic | |
error_code |
CreateTopics Response (Version: 1) => [topic_errors]
topic_errors => topic error_code error_message
topic => STRING
error_code => INT16
error_message => NULLABLE_STRING
Field | Description |
---|---|
topic_errors | An array of per topic errors. |
topic | |
error_code | |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
topic_errors | An array of per topic errors. |
topic | |
error_code | |
error_message |
DeleteTopics API (Key: 20):
Requests:DeleteTopics Request (Version: 0) => [topics] timeout
topics => STRING
timeout => INT32
Field | Description |
---|---|
topics | An array of topics to be deleted. |
timeout | The 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
Field | Description |
---|---|
topics | An array of topics to be deleted. |
timeout | The 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 Response (Version: 0) => [topic_error_codes]
topic_error_codes => topic error_code
topic => STRING
error_code => INT16
Field | Description |
---|---|
topic_error_codes | An array of per topic error codes. |
topic | |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
topic_error_codes | An array of per topic error codes. |
topic | |
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
Field | Description |
---|---|
topics | |
topic | Topic name. |
partitions | |
partition | Topic partition id. |
offset | The offset before which the messages will be deleted. |
timeout | The maximum time to await a response in ms. |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
topics | |
topic | Topic name. |
partitions | |
partition | Topic partition id. |
low_watermark | Smallest available offset of all live replicas |
error_code | The error code for the given partition. |
InitProducerId API (Key: 22):
Requests:InitProducerId Request (Version: 0) => transactional_id transaction_timeout_ms
transactional_id => NULLABLE_STRING
transaction_timeout_ms => INT32
Field | Description |
---|---|
transactional_id | The transactional id whose producer id we want to retrieve or generate. |
transaction_timeout_ms | The time in ms to wait for before aborting idle transactions sent by this producer. |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | An integer error code. |
producer_id | The producer id for the input transactional id. If the input id was empty, then this is used only for ensuring idempotence of messages. |
producer_epoch | The epoch for the producer id. Will always be 0 if no transactional id was specified in the request. |
OffsetForLeaderEpoch API (Key: 23):
Requests:OffsetForLeaderEpoch Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition_id leader_epoch
partition_id => INT32
leader_epoch => INT32
Field | Description |
---|---|
topics | An array of topics to get epochs for |
topic | The topic |
partitions | The partition |
partition_id | The partition_id |
leader_epoch | The epoch |
OffsetForLeaderEpoch Response (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => error_code partition_id end_offset
error_code => INT16
partition_id => INT32
end_offset => INT64
Field | Description |
---|---|
topics | An array of topics for which we have leader offsets for some requested Partition Leader Epoch |
topic | The topic |
partitions | The partition |
error_code | The error code |
partition_id | The partition id |
end_offset | The 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
Field | Description |
---|---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to add to the transaction. |
topic | |
partitions |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
errors | |
topic | |
partition_errors | |
partition | |
error_code |
AddOffsetsToTxn API (Key: 25):
Requests:AddOffsetsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch consumer_group_id
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
consumer_group_id => STRING
Field | Description |
---|---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
consumer_group_id | Consumer group id whose offsets should be included in the transaction. |
AddOffsetsToTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | An integer 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
Field | Description |
---|---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
transaction_result | The result of the transaction (0 = ABORT, 1 = COMMIT) |
EndTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | An integer 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
Field | Description |
---|---|
transaction_markers | The transaction markers to be written. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
transaction_result | The result of the transaction to write to the partitions (false = ABORT, true = COMMIT). |
topics | The partitions to write markers for. |
topic | |
partitions | |
coordinator_epoch | Epoch associated with the transaction state partition hosted by this transaction coordinator |
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
Field | Description |
---|---|
transaction_markers | Errors per partition from writing markers. |
producer_id | Current producer id in use by the transactional id. |
topics | Errors per partition from writing markers. |
topic | |
partitions | |
partition | |
error_code |
TxnOffsetCommit API (Key: 28):
Requests:TxnOffsetCommit Request (Version: 0) => transactional_id consumer_group_id producer_id producer_epoch [topics]
transactional_id => STRING
consumer_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
Field | Description |
---|---|
transactional_id | The transactional id corresponding to the transaction. |
consumer_group_id | Id of the associated consumer group to commit offsets for. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to write markers for. |
topic | |
partitions | |
partition | |
offset | |
metadata |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
topics | Errors per partition from writing markers. |
topic | |
partitions | |
partition | |
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
Field | Description |
---|---|
resource_type | The filter resource type. |
resource_name | The filter resource name. |
principal | The filter principal name. |
host | The filter ip address. |
operation | The filter operation type. |
permission_type | The filter permission type. |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
error_code | The error code. |
error_message | The error message. |
resources | The resources and their associated ACLs. |
resource_type | The resource type |
resource_name | The resource name |
acls | |
principal | The ACL principal |
host | The ACL host |
operation | The ACL operation |
permission_type | The 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
Field | Description |
---|---|
creations | |
resource_type | The resource type. |
resource_name | The resource name. |
principal | The principal. |
host | The ip address. |
operation | The ACL operation |
permission_type | The ACL permission type |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
creation_responses | |
error_code | The error code. |
error_message | The 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
Field | Description |
---|---|
filters | |
resource_type | The resource type filter. |
resource_name | The resource name filter. |
principal | The principal filter. |
host | The ip address filter. |
operation | The ACL operation filter. |
permission_type | The ACL permission type filter. |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
filter_responses | |
error_code | The error code. |
error_message | The error message. |
matching_acls | The matching ACLs |
error_code | The error code. |
error_message | The error message. |
resource_type | The resource type. |
resource_name | The resource name. |
principal | The principal. |
host | The ip address. |
operation | The ACL operation |
permission_type | The 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
Field | Description |
---|---|
resources | An array of config resources to be returned. |
resource_type | |
resource_name | |
config_names |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
resources | |
error_code | |
error_message | |
resource_type | |
resource_name | |
config_entries | |
config_name | |
config_value | |
read_only | |
is_default | |
is_sensitive |
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
Field | Description |
---|---|
resources | An array of resources to update with the provided configs. |
resource_type | |
resource_name | |
config_entries | |
config_name | Configuration name |
config_value | Configuration value |
validate_only |
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
Field | Description |
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation. (Zero if the request did not violate any quota.) |
resources | |
error_code | |
error_message | |
resource_type | |
resource_name |
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.