Kafka protocol guide

This document covers the wire protocol implemented in Kafka. It is meant to give a readable guide to the protocol that covers the available requests, their binary format, and the proper way to make use of them to implement a client. This document assumes you understand the basic design and terminology described here

Preliminaries

Network

Kafka uses a binary protocol over TCP. The protocol defines all apis as request response message pairs. All messages are size delimited and are made up of the following primitive types.

The client initiates a socket connection and then writes a sequence of request messages and reads back the corresponding response message. No handshake is required on connection or disconnection. TCP is happier if you maintain persistent connections used for many requests to amortize the cost of the TCP handshake, but beyond this penalty connecting is pretty cheap.

The client will likely need to maintain a connection to multiple brokers, as data is partitioned and the clients will need to talk to the server that has their data. However it should not generally be necessary to maintain multiple connections to a single broker from a single client instance (i.e. connection pooling).

The server guarantees that on a single TCP connection, requests will be processed in the order they are sent and responses will return in that order as well. The broker's request processing allows only a single in-flight request per connection in order to guarantee this ordering. Note that clients can (and ideally should) use non-blocking IO to implement request pipelining and achieve higher throughput. i.e., clients can send requests even while awaiting responses for preceding requests since the outstanding requests will be buffered in the underlying OS socket buffer. All requests are initiated by the client, and result in a corresponding response message from the server except where noted.

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

Partitioning and bootstrapping

Kafka is a partitioned system so not all servers have the complete data set. Instead recall that topics are split into a pre-defined number of partitions, P, and each partition is replicated with some replication factor, N. Topic partitions themselves are just ordered "commit logs" numbered 0, 1, ..., P.

All systems of this nature have the question of how a particular piece of data is assigned to a particular partition. Kafka clients directly control this assignment, the brokers themselves enforce no particular semantics of which messages should be published to a particular partition. Rather, to publish messages the client directly addresses messages to a particular partition, and when fetching messages, fetches from a particular partition. If two clients want to use the same partitioning scheme they must use the same method to compute the mapping of key to partition.

These requests to publish or fetch data must be sent to the broker that is currently acting as the leader for a given partition. This condition is enforced by the broker, so a request for a particular partition to the wrong broker will result in an the NotLeaderForPartition error code (described below).

How can the client find out which topics exist, what partitions they have, and which brokers currently host those partitions so that it can direct its requests to the right hosts? This information is dynamic, so you can't just configure each client with some static mapping file. Instead all Kafka brokers can answer a metadata request that describes the current state of the cluster: what topics there are, which partitions those topics have, which broker is the leader for those partitions, and the host and port information for these brokers.

In other words, the client needs to somehow find one broker and that broker will tell the client about all the other brokers that exist and what partitions they host. This first broker may itself go down so the best practice for a client implementation is to take a list of two or three urls to bootstrap from. The user can then choose to use a load balancer or just statically configure two or three of their kafka hosts in the clients.

The client does not need to keep polling to see if the cluster has changed; it can fetch metadata once when it is instantiated cache that metadata until it receives an error indicating that the metadata is out of date. This error can come in two forms: (1) a socket error indicating the client cannot communicate with a particular broker, (2) an error code in the response to a request indicating that this broker no longer hosts the partition for which data was requested.

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

As mentioned above the assignment of messages to partitions is something the producing client controls. That said, how should this functionality be exposed to the end-user?

Partitioning really serves two purposes in Kafka:

  1. It balances data and request load over brokers
  2. It serves as a way to divvy up processing among consumer processes while allowing local state and preserving order within the partition. We call this semantic partitioning.

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

To accomplish simple load balancing a simple approach would be for the client to just round robin requests over all brokers. Another alternative, in an environment where there are many more producers than brokers, would be to have each client chose a single partition at random and publish to that. This later strategy will result in far fewer TCP connections.

Semantic partitioning means using some key in the message to assign messages to partitions. For example if you were processing a click message stream you might want to partition the stream by the user id so that all data for a particular user would go to a single consumer. To accomplish this the client can take a key associated with the message and use some hash of this key to choose the partition to which to deliver the message.

Batching

Our apis encourage batching small things together for efficiency. We have found this is a very significant performance win. Both our API to send messages and our API to fetch messages always work with a sequence of messages not a single message to encourage this. A clever client can make use of this and support an "asynchronous" mode in which it batches together messages sent individually and sends them in larger clumps. We go even further with this and allow the batching across multiple topics and partitions, so a produce request may contain data to append to many partitions and a fetch request may pull data from many partitions all at once.

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

Versioning and Compatibility

The protocol is designed to enable incremental evolution in a backward compatible fashion. Our versioning is on a per API basis, each version consisting of a request and response pair. Each request contains an API key that identifies the API being invoked and a version number that indicates the format of the request and the expected format of the response.

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

The server will reject requests with a version it does not support, and will always respond to the client with exactly the protocol format it expects based on the version it included in its request. The intended upgrade path is that new features would first be rolled out on the server (with the older clients not making use of them) and then as newer clients are deployed these new features would gradually be taken advantage of.

Our goal is primarily to allow API evolution in an environment where downtime is not allowed and clients and servers cannot all be changed at once.

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

Retrieving Supported API versions

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

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

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

The following sequence is used for SASL authentication:

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

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

The Protocol

Protocol Primitive Types

The protocol is built out of the following primitive types.

Type Description
BOOLEANRepresents a boolean value in a byte. Values 0 and 1 are used to represent false and true respectively. When reading a boolean value, any non-zero value is considered true.
INT8Represents an integer between -27 and 27-1 inclusive.
INT16Represents an integer between -215 and 215-1 inclusive. The values are encoded using two bytes in network byte order (big-endian).
INT32Represents an integer between -231 and 231-1 inclusive. The values are encoded using four bytes in network byte order (big-endian).
INT64Represents an integer between -263 and 263-1 inclusive. The values are encoded using eight bytes in network byte order (big-endian).
UINT32Represents an integer between 0 and 232-1 inclusive. The values are encoded using four bytes in network byte order (big-endian).
VARINTRepresents an integer between -231 and 231-1 inclusive. Encoding follows the variable-length zig-zag encoding from Google Protocol Buffers.
VARLONGRepresents an integer between -263 and 263-1 inclusive. Encoding follows the variable-length zig-zag encoding from Google Protocol Buffers.
STRINGRepresents a sequence of characters. First the length N is given as an INT16. Then N bytes follow which are the UTF-8 encoding of the character sequence. Length must not be negative.
NULLABLE_STRINGRepresents a sequence of characters or null. For non-null strings, first the length N is given as an INT16. Then N bytes follow which are the UTF-8 encoding of the character sequence. A null value is encoded with length of -1 and there are no following bytes.
BYTESRepresents a raw sequence of bytes. First the length N is given as an INT32. Then N bytes follow.
NULLABLE_BYTESRepresents a raw sequence of bytes or null. For non-null values, first the length N is given as an INT32. Then N bytes follow. A null value is encoded with length of -1 and there are no following bytes.
RECORDSRepresents a sequence of Kafka records as NULLABLE_BYTES. For a detailed description of records see Message Sets.
ARRAYRepresents a sequence of objects of a given type T. Type T can be either a primitive type (e.g. STRING) or a structure. First, the length N is given as an INT32. Then N instances of type T follow. A null array is represented with a length of -1. In protocol documentation an array of T instances is referred to as [T].
Notes on reading the request format grammars

The BNFs below give an exact context free grammar for the request and response binary format. The BNF is intentionally not compact in order to give human-readable name. As always in a BNF a sequence of productions indicates concatenation. When there are multiple possible productions these are separated with '|' and may be enclosed in parenthesis for grouping. The top-level definition is always given first and subsequent sub-parts are indented.

Common Request and Response Structure

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


RequestOrResponse => Size (RequestMessage | ResponseMessage)
  Size => int32
FieldDescription
message_sizeThe message_size field gives the size of the subsequent request or response message in bytes. The client can read requests by first reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes of the request.
Record Batch

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

Constants

Error Codes

We use numeric codes to indicate what problem occurred on the server. These can be translated by the client into exceptions or whatever the appropriate error handling mechanism in the client language. Here is a table of the error codes currently in use:

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

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

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

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_keyThe id of the request type.
api_versionThe version of the API.
correlation_idA user-supplied integer value that will be passed back with the response
client_idA user specified identifier for the client making the request.
Response Header => correlation_id 
  correlation_id => INT32
Field Description
correlation_idThe user-supplied value passed in with the request
Produce API (Key: 0):
Requests:

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

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

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

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

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

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

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

Responses:

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

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

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

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

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

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

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

Fetch API (Key: 1):
Requests:

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

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

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

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

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

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

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

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

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

Responses:

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

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

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

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

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

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

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

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

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

ListOffsets API (Key: 2):
Requests:

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

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

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

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

Responses:

ListOffsets Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code [offsets] 
      partition => INT32
      error_code => INT16
      offsets => INT64
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
offsetsA list of offsets.

ListOffsets Response (Version: 1) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code timestamp offset 
      partition => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code
timestampThe timestamp associated with the returned offset
offsetoffset found

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

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

Metadata API (Key: 3):
Requests:

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

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

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

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

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

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

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

Responses:

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

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

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

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

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

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

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

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_idThe controller id.
controller_epochThe controller epoch.
partition_statesnull
topicName of topic
partitionTopic partition id
controller_epochThe controller epoch.
leaderThe broker id for the leader.
leader_epochThe leader epoch.
isrThe in sync replica ids.
zk_versionThe ZK version.
replicasThe replica ids.
live_leadersnull
idThe broker id.
hostThe hostname of the broker.
portThe port on which the broker accepts requests.

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

Responses:

LeaderAndIsr Response (Version: 0) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
Field Description
error_codeResponse error code
partitionsnull
topicName of topic
partitionTopic partition id
error_codeResponse error code

LeaderAndIsr Response (Version: 1) => error_code [partitions] 
  error_code => INT16
  partitions => topic partition error_code 
    topic => STRING
    partition => INT32
    error_code => INT16
Field Description
error_codeResponse error code
partitionsnull
topicName of topic
partitionTopic partition id
error_codeResponse error code

StopReplica API (Key: 5):
Requests:

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

Responses:

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_codeResponse error code
partitionsnull
topicName of topic
partitionTopic partition id
error_codeResponse error code

UpdateMetadata API (Key: 6):
Requests:

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

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

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

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

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

Responses:

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

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

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

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

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

ControlledShutdown API (Key: 7):
Requests:

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

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

Responses:

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

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

OffsetCommit API (Key: 8):
Requests:

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

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

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

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

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

Responses:

OffsetCommit Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 1) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code

OffsetCommit Response (Version: 2) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
error_codeResponse error code

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

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

OffsetFetch API (Key: 9):
Requests:

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

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

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

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

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

Responses:

OffsetFetch Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
offsetLast committed message offset.
metadataAny associated metadata the client wants to keep.
error_codeResponse error code

OffsetFetch Response (Version: 1) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
offsetLast committed message offset.
metadataAny associated metadata the client wants to keep.
error_codeResponse error code

OffsetFetch Response (Version: 2) => [responses] error_code 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition offset metadata error_code 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
Field Description
responsesnull
topicName of topic
partition_responsesnull
partitionTopic partition id
offsetLast committed message offset.
metadataAny associated metadata the client wants to keep.
error_codeResponse error code
error_codeResponse error code

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

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

FindCoordinator API (Key: 10):
Requests:

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

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

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

Responses:

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

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

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

JoinGroup API (Key: 11):
Requests:

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

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

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

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

Responses:

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

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

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

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

Heartbeat API (Key: 12):
Requests:

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

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

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

Responses:

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

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

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

LeaveGroup API (Key: 13):
Requests:

LeaveGroup Request (Version: 0) => group_id member_id 
  group_id => STRING
  member_id => STRING
Field Description
group_idThe unique group identifier
member_idThe member id assigned by the group coordinator or null if joining for the first time.

LeaveGroup Request (Version: 1) => group_id member_id 
  group_id => STRING
  member_id => STRING
Field Description
group_idThe unique group identifier
member_idThe member id assigned by the group coordinator or null if joining for the first time.

LeaveGroup Request (Version: 2) => group_id member_id 
  group_id => STRING
  member_id => STRING
Field Description
group_idThe unique group identifier
member_idThe member id assigned by the group coordinator or null if joining for the first time.

Responses:

LeaveGroup Response (Version: 0) => error_code 
  error_code => INT16
Field Description
error_codeResponse error code

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

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

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_idThe unique group identifier
generation_idThe generation of the group.
member_idThe member id assigned by the group coordinator or null if joining for the first time.
group_assignmentnull
member_idThe member id assigned by the group coordinator or null if joining for the first time.
member_assignmentnull

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

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

Responses:

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

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

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

DescribeGroups API (Key: 15):
Requests:

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

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

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

Responses:

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

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

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

ListGroups API (Key: 16):
Requests:

ListGroups Request (Version: 0) => 
Field Description

ListGroups Request (Version: 1) => 
Field Description

ListGroups Request (Version: 2) => 
Field Description

Responses:

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

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

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

SaslHandshake API (Key: 17):
Requests:

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

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

Responses:

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

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

ApiVersions API (Key: 18):
Requests:

ApiVersions Request (Version: 0) => 
Field Description

ApiVersions Request (Version: 1) => 
Field Description

ApiVersions Request (Version: 2) => 
Field Description

Responses:

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

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

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

CreateTopics API (Key: 19):
Requests:

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

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

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

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

Responses:

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

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

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

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

DeleteTopics API (Key: 20):
Requests:

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

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

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

Responses:

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

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

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

DeleteRecords API (Key: 21):
Requests:

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

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

Responses:

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

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

InitProducerId API (Key: 22):
Requests:

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

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

Responses:

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

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

OffsetForLeaderEpoch API (Key: 23):
Requests:

OffsetForLeaderEpoch Request (Version: 0) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition leader_epoch 
      partition => INT32
      leader_epoch => INT32
Field Description
topicsAn array of topics to get epochs for
topicName of topic
partitionsnull
partitionTopic partition id
leader_epochThe epoch

OffsetForLeaderEpoch Request (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition leader_epoch 
      partition => INT32
      leader_epoch => INT32
Field Description
topicsAn array of topics to get epochs for
topicName of topic
partitionsnull
partitionTopic partition id
leader_epochThe epoch

Responses:

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

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

AddPartitionsToTxn API (Key: 24):
Requests:

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

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

Responses:

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

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

AddOffsetsToTxn API (Key: 25):
Requests:

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

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

Responses:

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

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

EndTxn API (Key: 26):
Requests:

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

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

Responses:

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

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

WriteTxnMarkers API (Key: 27):
Requests:

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

Responses:

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

TxnOffsetCommit API (Key: 28):
Requests:

TxnOffsetCommit Request (Version: 0) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
Field Description
transactional_idThe transactional id corresponding to the transaction.
group_idThe unique group identifier
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsThe partitions to write markers for.
topicName of topic
partitionsnull
partitionTopic partition id
offsetnull
metadatanull

TxnOffsetCommit Request (Version: 1) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING
Field Description
transactional_idThe transactional id corresponding to the transaction.
group_idThe unique group identifier
producer_idCurrent producer id in use by the transactional id.
producer_epochCurrent epoch associated with the producer id.
topicsThe partitions to write markers for.
topicName of topic
partitionsnull
partitionTopic partition id
offsetnull
metadatanull

Responses:

TxnOffsetCommit Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsErrors per partition from writing markers.
topicName of topic
partitionsnull
partitionTopic partition id
error_codeResponse error code

TxnOffsetCommit Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition error_code 
      partition => INT32
      error_code => INT16
Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)
topicsErrors per partition from writing markers.
topicName of topic
partitionsnull
partitionTopic partition id
error_codeResponse error code

DescribeAcls API (Key: 29):
Requests:

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

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

Responses:

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

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

CreateAcls API (Key: 30):
Requests:

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

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

Responses:

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

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

DeleteAcls API (Key: 31):
Requests:

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

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

Responses:

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

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

DescribeConfigs API (Key: 32):
Requests:

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

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

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

Responses:

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

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

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

AlterConfigs API (Key: 33):
Requests:

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

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

Responses:

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

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

AlterReplicaLogDirs API (Key: 34):
Requests:

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

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

Responses:

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

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

DescribeLogDirs API (Key: 35):
Requests:

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

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

Responses:

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

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

SaslAuthenticate API (Key: 36):
Requests:

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

Responses:

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

CreatePartitions API (Key: 37):
Requests:

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

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

Responses:

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

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

CreateDelegationToken API (Key: 38):
Requests:

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

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

Responses:

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

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

RenewDelegationToken API (Key: 39):
Requests:

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

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

Responses:

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

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

ExpireDelegationToken API (Key: 40):
Requests:

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

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

Responses:

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

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

DescribeDelegationToken API (Key: 41):
Requests:

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

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

Responses:

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

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

DeleteGroups API (Key: 42):
Requests:

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

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

Responses:

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

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

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.