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

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.

Compatibility

Kafka has a "bidirectional" client compatibility policy. In other words, new clients can talk to old servers, and old clients can talk to new servers. This allows users to upgrade either clients or servers without experiencing any downtime.

Since the Kafka protocol has changed over time, clients and servers need to agree on the schema of the message that they are sending over the wire. This is done through API versioning.

Before each request is sent, the client sends the API key and the API version. These two 16-bit numbers, when taken together, uniquely identify the schema of the message to follow.

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. Note there is an exceptional case while retrieving supported API versions where the server can respond with a different version.

Note that KIP-482 tagged fields can be added to a request without incrementing the version number. This offers an additional way of evolving the message schema without breaking compatibility. Tagged fields do not take up any space when the field is not set. Therefore, if a field is rarely used, it is more efficient to make it a tagged field than to put it in the mandatory schema. However, tagged fields are ignored by recipients that don't know about them, which could pose a challenge if this is not the behavior that the sender wants. In such cases, a version bump may be more appropriate.

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 an 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. Also note that if the client ApiVersionsRequest version is unsupported by the broker (client is ahead), and the broker version is 2.4.0 or greater, then the broker will respond with a version 0 ApiVersionsResponse with the error code set to UNSUPPORTED_VERSION and the api_versions field populated with the supported version of the ApiVersionsRequest. It is then up to the client to retry, making another ApiVersionsRequest using the highest version supported by the client and broker. See KIP-511: Collect and Expose Client's Name and Version in the Brokers
  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).
UINT16Represents an integer between 0 and 65535 inclusive. The values are encoded using two 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.
UUIDRepresents a type 4 immutable universally unique identifier (Uuid). The values are encoded using sixteen bytes in network byte order (big-endian).
FLOAT64Represents a double-precision 64-bit format IEEE 754 value. The values are encoded using eight bytes in network byte order (big-endian).
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.
COMPACT_STRINGRepresents a sequence of characters. First the length N + 1 is given as an UNSIGNED_VARINT . Then N bytes follow which are the UTF-8 encoding of the character sequence.
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.
COMPACT_NULLABLE_STRINGRepresents a sequence of characters. First the length N + 1 is given as an UNSIGNED_VARINT . Then N bytes follow which are the UTF-8 encoding of the character sequence. A null string is represented with a length of 0.
BYTESRepresents a raw sequence of bytes. First the length N is given as an INT32. Then N bytes follow.
COMPACT_BYTESRepresents a raw sequence of bytes. First the length N+1 is given as an UNSIGNED_VARINT.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.
COMPACT_NULLABLE_BYTESRepresents a raw sequence of bytes. First the length N+1 is given as an UNSIGNED_VARINT.Then N bytes follow. A null object is represented with a length of 0.
RECORDSRepresents a sequence of Kafka records as NULLABLE_BYTES. For a detailed description of records see Message Sets.
COMPACT_RECORDSRepresents a sequence of Kafka records as COMPACT_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].
COMPACT_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 + 1 is given as an UNSIGNED_VARINT. Then N instances of type T follow. A null array is represented with a length of 0. 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.
Request and Response Headers

Different request and response versions require different versions of the corresponding headers. These header versions are specified below together with API message descriptions.

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, has a null key for a compacted topic, or is otherwise corrupt.
UNKNOWN_TOPIC_OR_PARTITION3TrueThis server does not host this topic-partition.
INVALID_FETCH_SIZE4FalseThe requested fetch size is invalid.
LEADER_NOT_AVAILABLE5TrueThere is no leader for this topic-partition as we are in the middle of a leadership election.
NOT_LEADER_OR_FOLLOWER6TrueFor requests intended only for the leader, this error indicates that the broker is not the current leader. For requests intended for any replica, this error indicates that the broker is not a replica of the topic partition.
REQUEST_TIMED_OUT7TrueThe request timed out.
BROKER_NOT_AVAILABLE8FalseThe broker is not available.
REPLICA_NOT_AVAILABLE9TrueThe replica is not available for the requested topic-partition. Produce/Fetch requests and other requests intended only for the leader or follower return NOT_LEADER_OR_FOLLOWER if the broker is not a replica of the 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_FAILED29FalseTopic authorization failed.
GROUP_AUTHORIZATION_FAILED30FalseGroup 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 to produce with an old epoch.
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_TRANSACTIONS51TrueThe producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing.
TRANSACTION_COORDINATOR_FENCED52FalseIndicates that the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer.
TRANSACTIONAL_ID_AUTHORIZATION_FAILED53FalseTransactional Id authorization failed.
SECURITY_DISABLED54FalseSecurity features are disabled.
OPERATION_NOT_ATTEMPTED55FalseThe broker did not attempt to execute this operation. This may happen for batched RPCs where some operations in the batch failed, causing the broker to respond without trying the rest.
KAFKA_STORAGE_ERROR56TrueDisk error when trying to access log file on the disk.
LOG_DIR_NOT_FOUND57FalseThe user-specified log directory is not found in the broker config.
SASL_AUTHENTICATION_FAILED58FalseSASL Authentication failed.
UNKNOWN_PRODUCER_ID59FalseThis exception is raised by the broker if it could not locate the producer metadata associated with the producerId in question. This could happen if, for instance, the producer's records were deleted because their retention time had elapsed. Once the last records of the producerId are removed, the producer's metadata is removed from the broker, and future appends by the producer will return this exception.
REASSIGNMENT_IN_PROGRESS60FalseA partition reassignment is in progress.
DELEGATION_TOKEN_AUTH_DISABLED61FalseDelegation Token feature is not enabled.
DELEGATION_TOKEN_NOT_FOUND62FalseDelegation Token is not found on server.
DELEGATION_TOKEN_OWNER_MISMATCH63FalseSpecified Principal is not valid Owner/Renewer.
DELEGATION_TOKEN_REQUEST_NOT_ALLOWED64FalseDelegation Token requests are not allowed on PLAINTEXT/1-way SSL channels and on delegation token authenticated channels.
DELEGATION_TOKEN_AUTHORIZATION_FAILED65FalseDelegation Token authorization failed.
DELEGATION_TOKEN_EXPIRED66FalseDelegation Token is expired.
INVALID_PRINCIPAL_TYPE67FalseSupplied principalType is not supported.
NON_EMPTY_GROUP68FalseThe group is not empty.
GROUP_ID_NOT_FOUND69FalseThe group id does not exist.
FETCH_SESSION_ID_NOT_FOUND70TrueThe fetch session ID was not found.
INVALID_FETCH_SESSION_EPOCH71TrueThe fetch session epoch is invalid.
LISTENER_NOT_FOUND72TrueThere is no listener on the leader broker that matches the listener on which metadata request was processed.
TOPIC_DELETION_DISABLED73FalseTopic deletion is disabled.
FENCED_LEADER_EPOCH74TrueThe leader epoch in the request is older than the epoch on the broker.
UNKNOWN_LEADER_EPOCH75TrueThe leader epoch in the request is newer than the epoch on the broker.
UNSUPPORTED_COMPRESSION_TYPE76FalseThe requesting client does not support the compression type of given partition.
STALE_BROKER_EPOCH77FalseBroker epoch has changed.
OFFSET_NOT_AVAILABLE78TrueThe leader high watermark has not caught up from a recent leader election so the offsets cannot be guaranteed to be monotonically increasing.
MEMBER_ID_REQUIRED79FalseThe group member needs to have a valid member id before actually entering a consumer group.
PREFERRED_LEADER_NOT_AVAILABLE80TrueThe preferred leader was not available.
GROUP_MAX_SIZE_REACHED81FalseThe consumer group has reached its max size.
FENCED_INSTANCE_ID82FalseThe broker rejected this static consumer since another consumer with the same group.instance.id has registered with a different member.id.
ELIGIBLE_LEADERS_NOT_AVAILABLE83TrueEligible topic partition leaders are not available.
ELECTION_NOT_NEEDED84TrueLeader election not needed for topic partition.
NO_REASSIGNMENT_IN_PROGRESS85FalseNo partition reassignment is in progress.
GROUP_SUBSCRIBED_TO_TOPIC86FalseDeleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.
INVALID_RECORD87FalseThis record has failed the validation on broker and hence will be rejected.
UNSTABLE_OFFSET_COMMIT88TrueThere are unstable offsets that need to be cleared.
THROTTLING_QUOTA_EXCEEDED89TrueThe throttling quota has been exceeded.
PRODUCER_FENCED90FalseThere is a newer producer with the same transactionalId which fences the current one.
RESOURCE_NOT_FOUND91FalseA request illegally referred to a resource that does not exist.
DUPLICATE_RESOURCE92FalseA request illegally referred to the same resource twice.
UNACCEPTABLE_CREDENTIAL93FalseRequested credential would not meet criteria for acceptability.
INCONSISTENT_VOTER_SET94FalseIndicates that the either the sender or recipient of a voter-only request is not one of the expected voters.
INVALID_UPDATE_VERSION95FalseThe given update version was invalid.
FEATURE_UPDATE_FAILED96FalseUnable to update finalized features due to an unexpected server error.
PRINCIPAL_DESERIALIZATION_FAILURE97FalseRequest principal deserialization failed during forwarding. This indicates an internal error on the broker cluster security setup.
SNAPSHOT_NOT_FOUND98FalseRequested snapshot was not found.
POSITION_OUT_OF_RANGE99FalseRequested position is not greater than or equal to zero, and less than the size of the snapshot.
UNKNOWN_TOPIC_ID100TrueThis server does not host this topic ID.
DUPLICATE_BROKER_REGISTRATION101FalseThis broker ID is already in use.
BROKER_ID_NOT_REGISTERED102FalseThe given broker ID was not registered.
INCONSISTENT_TOPIC_ID103TrueThe log's topic ID did not match the topic ID in the request.
INCONSISTENT_CLUSTER_ID104FalseThe clusterId in the request does not match that found on the server.
TRANSACTIONAL_ID_NOT_FOUND105FalseThe transactionalId could not be found.
FETCH_SESSION_TOPIC_ID_ERROR106TrueThe fetch session encountered inconsistent topic ID usage.
INELIGIBLE_REPLICA107FalseThe new ISR contains at least one ineligible replica.
NEW_LEADER_ELECTED108FalseThe AlterPartition request successfully updated the partition state but the leader has changed.
OFFSET_MOVED_TO_TIERED_STORAGE109FalseThe requested offset is moved to tiered storage.
FENCED_MEMBER_EPOCH110FalseThe member epoch is fenced by the group coordinator. The member must abandon all its partitions and rejoin.
UNRELEASED_INSTANCE_ID111FalseThe instance ID is still used by another member in the consumer group. That member must leave first.
UNSUPPORTED_ASSIGNOR112FalseThe assignor or its version range is not supported by the consumer group.
STALE_MEMBER_EPOCH113FalseThe member epoch is stale. The member must retry after receiving its updated member epoch via the ConsumerGroupHeartbeat API.
MISMATCHED_ENDPOINT_TYPE114FalseThe request was sent to an endpoint of the wrong type.
UNSUPPORTED_ENDPOINT_TYPE115FalseThis endpoint type is not supported yet.
UNKNOWN_CONTROLLER_ID116FalseThis controller ID is not known.
UNKNOWN_SUBSCRIPTION_ID117FalseClient sent a push telemetry request with an invalid or outdated subscription ID.
TELEMETRY_TOO_LARGE118FalseClient sent a push telemetry request larger than the maximum size the broker will accept.
INVALID_REGISTRATION119FalseThe controller has considered the broker registration to be invalid.
TRANSACTION_ABORTABLE120FalseThe server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.
INVALID_RECORD_STATE121FalseThe record state is invalid. The acknowledgement of delivery could not be completed.
SHARE_SESSION_NOT_FOUND122TrueThe share session was not found.
INVALID_SHARE_SESSION_EPOCH123TrueThe share session epoch is invalid.
FENCED_STATE_EPOCH124FalseThe share coordinator rejected the request because the share-group state epoch did not match.
INVALID_VOTER_KEY125FalseThe voter key doesn't match the receiving replica's key.
DUPLICATE_VOTER126FalseThe voter is already part of the set of voters.
VOTER_NOT_FOUND127FalseThe voter is not part of the set of voters.
INVALID_REGULAR_EXPRESSION128FalseThe regular expression is not valid.
REBOOTSTRAP_REQUIRED129FalseClient metadata is stale, client should rebootstrap to obtain new metadata.
Api Keys

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

Name Key
Produce0
Fetch1
ListOffsets2
Metadata3
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
ElectLeaders43
IncrementalAlterConfigs44
AlterPartitionReassignments45
ListPartitionReassignments46
OffsetDelete47
DescribeClientQuotas48
AlterClientQuotas49
DescribeUserScramCredentials50
AlterUserScramCredentials51
DescribeQuorum55
UpdateFeatures57
DescribeCluster60
DescribeProducers61
UnregisterBroker64
DescribeTransactions65
ListTransactions66
ConsumerGroupHeartbeat68
ConsumerGroupDescribe69
GetTelemetrySubscriptions71
PushTelemetry72
ListClientMetricsResources74
DescribeTopicPartitions75
AddRaftVoter80
RemoveRaftVoter81

The Messages

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

The message consists of the header and body:

Message => RequestOrResponseHeader Body

RequestOrResponseHeader is the versioned request or response header. Body is the message-specific body.

Headers:
Request Header v1 => request_api_key request_api_version correlation_id client_id 
  request_api_key => INT16
  request_api_version => INT16
  correlation_id => INT32
  client_id => NULLABLE_STRING
Field Description
request_api_keyThe API key of this request.
request_api_versionThe API version of this request.
correlation_idThe correlation ID of this request.
client_idThe client ID string.
Request Header v2 => request_api_key request_api_version correlation_id client_id _tagged_fields 
  request_api_key => INT16
  request_api_version => INT16
  correlation_id => INT32
  client_id => NULLABLE_STRING
Field Description
request_api_keyThe API key of this request.
request_api_versionThe API version of this request.
correlation_idThe correlation ID of this request.
client_idThe client ID string.
_tagged_fieldsThe tagged fields
Response Header v0 => correlation_id 
  correlation_id => INT32
Field Description
correlation_idThe correlation ID of this response.
Response Header v1 => correlation_id _tagged_fields 
  correlation_id => INT32
Field Description
correlation_idThe correlation ID of this response.
_tagged_fieldsThe tagged fields
Produce API (Key: 0):
Requests:
Produce Request (Version: 3) => transactional_id acks timeout_ms [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] 
    name => STRING
    partition_data => index records 
      index => INT32
      records => RECORDS

Request header version: 1

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
Produce Request (Version: 4) => transactional_id acks timeout_ms [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] 
    name => STRING
    partition_data => index records 
      index => INT32
      records => RECORDS

Request header version: 1

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
Produce Request (Version: 5) => transactional_id acks timeout_ms [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] 
    name => STRING
    partition_data => index records 
      index => INT32
      records => RECORDS

Request header version: 1

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
Produce Request (Version: 6) => transactional_id acks timeout_ms [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] 
    name => STRING
    partition_data => index records 
      index => INT32
      records => RECORDS

Request header version: 1

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
Produce Request (Version: 7) => transactional_id acks timeout_ms [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] 
    name => STRING
    partition_data => index records 
      index => INT32
      records => RECORDS

Request header version: 1

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
Produce Request (Version: 8) => transactional_id acks timeout_ms [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] 
    name => STRING
    partition_data => index records 
      index => INT32
      records => RECORDS

Request header version: 1

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
Produce Request (Version: 9) => transactional_id acks timeout_ms [topic_data] _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] _tagged_fields 
    name => COMPACT_STRING
    partition_data => index records _tagged_fields 
      index => INT32
      records => COMPACT_RECORDS

Request header version: 2

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Produce Request (Version: 10) => transactional_id acks timeout_ms [topic_data] _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] _tagged_fields 
    name => COMPACT_STRING
    partition_data => index records _tagged_fields 
      index => INT32
      records => COMPACT_RECORDS

Request header version: 2

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Produce Request (Version: 11) => transactional_id acks timeout_ms [topic_data] _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] _tagged_fields 
    name => COMPACT_STRING
    partition_data => index records _tagged_fields 
      index => INT32
      records => COMPACT_RECORDS

Request header version: 2

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Produce Request (Version: 12) => transactional_id acks timeout_ms [topic_data] _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  acks => INT16
  timeout_ms => INT32
  topic_data => name [partition_data] _tagged_fields 
    name => COMPACT_STRING
    partition_data => index records _tagged_fields 
      index => INT32
      records => COMPACT_RECORDS

Request header version: 2

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.
timeout_msThe timeout to await a response in milliseconds.
topic_dataEach topic to produce to.
nameThe topic name.
partition_dataEach partition to produce to.
indexThe partition index.
recordsThe record data to be produced.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
Produce Response (Version: 3) => [responses] throttle_time_ms 
  responses => name [partition_responses] 
    name => STRING
    partition_responses => index error_code base_offset log_append_time_ms 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
Produce Response (Version: 4) => [responses] throttle_time_ms 
  responses => name [partition_responses] 
    name => STRING
    partition_responses => index error_code base_offset log_append_time_ms 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
Produce Response (Version: 5) => [responses] throttle_time_ms 
  responses => name [partition_responses] 
    name => STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
Produce Response (Version: 6) => [responses] throttle_time_ms 
  responses => name [partition_responses] 
    name => STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
Produce Response (Version: 7) => [responses] throttle_time_ms 
  responses => name [partition_responses] 
    name => STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
Produce Response (Version: 8) => [responses] throttle_time_ms 
  responses => name [partition_responses] 
    name => STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
      record_errors => batch_index batch_index_error_message 
        batch_index => INT32
        batch_index_error_message => NULLABLE_STRING
      error_message => NULLABLE_STRING
  throttle_time_ms => INT32

Response header version: 0

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
record_errorsThe batch indices of records that caused the batch to be dropped.
batch_indexThe batch index of the record that caused the batch to be dropped.
batch_index_error_messageThe error message of the record that caused the batch to be dropped.
error_messageThe global error message summarizing the common root cause of the records that caused the batch to be dropped.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
Produce Response (Version: 9) => [responses] throttle_time_ms _tagged_fields 
  responses => name [partition_responses] _tagged_fields 
    name => COMPACT_STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message _tagged_fields 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
      record_errors => batch_index batch_index_error_message _tagged_fields 
        batch_index => INT32
        batch_index_error_message => COMPACT_NULLABLE_STRING
      error_message => COMPACT_NULLABLE_STRING
  throttle_time_ms => INT32

Response header version: 1

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
record_errorsThe batch indices of records that caused the batch to be dropped.
batch_indexThe batch index of the record that caused the batch to be dropped.
batch_index_error_messageThe error message of the record that caused the batch to be dropped.
_tagged_fieldsThe tagged fields
error_messageThe global error message summarizing the common root cause of the records that caused the batch to be dropped.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
_tagged_fieldsThe tagged fields
Produce Response (Version: 10) => [responses] throttle_time_ms _tagged_fields 
  responses => name [partition_responses] _tagged_fields 
    name => COMPACT_STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message _tagged_fields 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
      record_errors => batch_index batch_index_error_message _tagged_fields 
        batch_index => INT32
        batch_index_error_message => COMPACT_NULLABLE_STRING
      error_message => COMPACT_NULLABLE_STRING
  throttle_time_ms => INT32

Response header version: 1

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
record_errorsThe batch indices of records that caused the batch to be dropped.
batch_indexThe batch index of the record that caused the batch to be dropped.
batch_index_error_messageThe error message of the record that caused the batch to be dropped.
_tagged_fieldsThe tagged fields
error_messageThe global error message summarizing the common root cause of the records that caused the batch to be dropped.
_tagged_fields
Tag Tagged field Description
0current_leaderThe leader broker that the producer should use for future requests.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
_tagged_fields
Tag Tagged field Description
0node_endpointsEndpoints for all current-leaders enumerated in PartitionProduceResponses, with errors NOT_LEADER_OR_FOLLOWER.
Field Description
node_idThe ID of the associated node.
hostThe node's hostname.
portThe node's port.
rackThe rack of the node, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
Produce Response (Version: 11) => [responses] throttle_time_ms _tagged_fields 
  responses => name [partition_responses] _tagged_fields 
    name => COMPACT_STRING
    partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message _tagged_fields 
      index => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time_ms => INT64
      log_start_offset => INT64
      record_errors => batch_index batch_index_error_message _tagged_fields 
        batch_index => INT32
        batch_index_error_message => COMPACT_NULLABLE_STRING
      error_message => COMPACT_NULLABLE_STRING
  throttle_time_ms => INT32

Response header version: 1

Field Description
responsesEach produce response.
nameThe topic name.
partition_responsesEach partition that we produced to within the topic.
indexThe partition index.
error_codeThe error code, or 0 if there was no error.
base_offsetThe base offset.
log_append_time_msThe 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 log start offset.
record_errorsThe batch indices of records that caused the batch to be dropped.
batch_indexThe batch index of the record that caused the batch to be dropped.
batch_index_error_messageThe error message of the record that caused the batch to be dropped.
_tagged_fieldsThe tagged fields
error_messageThe global error message summarizing the common root cause of the records that caused the batch to be dropped.
_tagged_fields
Tag Tagged field Description
0current_leaderThe leader broker that the producer should use for future requests.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
_tagged_fields
Tag Tagged field Description
0node_endpointsEndpoints for all current-leaders enumerated in PartitionProduceResponses, with errors NOT_LEADER_OR_FOLLOWER.
Field Description
node_idThe ID of the associated node.
hostThe node's hostname.
portThe node's port.
rackThe rack of the node, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
Fetch API (Key: 1):
Requests:
Fetch Request (Version: 4) => replica_id max_wait_ms min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
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.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
fetch_offsetThe message offset.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
Fetch Request (Version: 5) => replica_id max_wait_ms min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
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.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
Fetch Request (Version: 6) => replica_id max_wait_ms min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
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.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
Fetch Request (Version: 7) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topicThe topic name.
partitionsThe partitions indexes to forget.
Fetch Request (Version: 8) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topicThe topic name.
partitionsThe partitions indexes to forget.
Fetch Request (Version: 9) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topicThe topic name.
partitionsThe partitions indexes to forget.
Fetch Request (Version: 10) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topicThe topic name.
partitionsThe partitions indexes to forget.
Fetch Request (Version: 11) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
  rack_id => STRING

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topicThe topic name.
partitionsThe partitions indexes to forget.
rack_idRack ID of the consumer making this request.
Fetch Request (Version: 12) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id _tagged_fields 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => INT32
  rack_id => COMPACT_STRING

Request header version: 2

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topicThe name of the topic to fetch.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topicThe topic name.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
rack_idRack ID of the consumer making this request.
_tagged_fields
Tag Tagged field Description
0cluster_idThe clusterId if known. This is used to validate metadata fetches prior to broker registration.
Fetch Request (Version: 13) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id _tagged_fields 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32
  rack_id => COMPACT_STRING

Request header version: 2

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topic_idThe unique topic ID.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topic_idThe unique topic ID.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
rack_idRack ID of the consumer making this request.
_tagged_fields
Tag Tagged field Description
0cluster_idThe clusterId if known. This is used to validate metadata fetches prior to broker registration.
Fetch Request (Version: 14) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id _tagged_fields 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32
  rack_id => COMPACT_STRING

Request header version: 2

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topic_idThe unique topic ID.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topic_idThe unique topic ID.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
rack_idRack ID of the consumer making this request.
_tagged_fields
Tag Tagged field Description
0cluster_idThe clusterId if known. This is used to validate metadata fetches prior to broker registration.
Fetch Request (Version: 15) => max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id _tagged_fields 
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32
  rack_id => COMPACT_STRING

Request header version: 2

Field Description
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topic_idThe unique topic ID.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topic_idThe unique topic ID.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
rack_idRack ID of the consumer making this request.
_tagged_fields
Tag Tagged field Description
0cluster_idThe clusterId if known. This is used to validate metadata fetches prior to broker registration.
1replica_stateThe state of the replica in the follower.
Field Description
replica_idThe replica ID of the follower, or -1 if this request is from a consumer.
replica_epochThe epoch of this follower, or -1 if not available.
_tagged_fieldsThe tagged fields
Fetch Request (Version: 16) => max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id _tagged_fields 
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32
  rack_id => COMPACT_STRING

Request header version: 2

Field Description
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topic_idThe unique topic ID.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topic_idThe unique topic ID.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
rack_idRack ID of the consumer making this request.
_tagged_fields
Tag Tagged field Description
0cluster_idThe clusterId if known. This is used to validate metadata fetches prior to broker registration.
1replica_stateThe state of the replica in the follower.
Field Description
replica_idThe replica ID of the follower, or -1 if this request is from a consumer.
replica_epochThe epoch of this follower, or -1 if not available.
_tagged_fieldsThe tagged fields
Fetch Request (Version: 17) => max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id _tagged_fields 
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32
  rack_id => COMPACT_STRING

Request header version: 2

Field Description
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelThis setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records.
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
topic_idThe unique topic ID.
partitionsThe partitions to fetch.
partitionThe partition index.
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fields
Tag Tagged field Description
0replica_directory_idThe directory id of the follower fetching.
_tagged_fieldsThe tagged fields
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
topic_idThe unique topic ID.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
rack_idRack ID of the consumer making this request.
_tagged_fields
Tag Tagged field Description
0cluster_idThe clusterId if known. This is used to validate metadata fetches prior to broker registration.
1replica_stateThe state of the replica in the follower.
Field Description
replica_idThe replica ID of the follower, or -1 if this request is from a consumer.
replica_epochThe epoch of this follower, or -1 if not available.
_tagged_fieldsThe tagged fields
Responses:
Fetch Response (Version: 4) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset [aborted_transactions] records 
      partition_index => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 5) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] records 
      partition_index => 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
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 6) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] records 
      partition_index => 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
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 7) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] records 
      partition_index => 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
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 8) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] records 
      partition_index => 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
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 9) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] records 
      partition_index => 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
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 10) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] records 
      partition_index => 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
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
recordsThe record data.
Fetch Response (Version: 11) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partitions] 
    topic => STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records 
      partition_index => 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
      preferred_read_replica => INT32
      records => RECORDS

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request.
recordsThe record data.
Fetch Response (Version: 12) => throttle_time_ms error_code session_id [responses] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records _tagged_fields 
      partition_index => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset _tagged_fields 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      records => COMPACT_RECORDS

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topicThe topic name.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
_tagged_fieldsThe tagged fields
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request.
recordsThe record data.
_tagged_fields
Tag Tagged field Description
0diverging_epochIn case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge.
Field Description
epochThe largest epoch.
end_offsetThe end offset of the epoch.
_tagged_fieldsThe tagged fields
1current_leaderThe current leader of the partition.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
2snapshot_idIn the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.
Field Description
end_offsetThe end offset of the epoch.
epochThe largest epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Fetch Response (Version: 13) => throttle_time_ms error_code session_id [responses] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records _tagged_fields 
      partition_index => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset _tagged_fields 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      records => COMPACT_RECORDS

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topic_idThe unique topic ID.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
_tagged_fieldsThe tagged fields
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request.
recordsThe record data.
_tagged_fields
Tag Tagged field Description
0diverging_epochIn case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge.
Field Description
epochThe largest epoch.
end_offsetThe end offset of the epoch.
_tagged_fieldsThe tagged fields
1current_leaderThe current leader of the partition.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
2snapshot_idIn the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.
Field Description
end_offsetThe end offset of the epoch.
epochThe largest epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Fetch Response (Version: 14) => throttle_time_ms error_code session_id [responses] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records _tagged_fields 
      partition_index => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset _tagged_fields 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      records => COMPACT_RECORDS

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topic_idThe unique topic ID.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
_tagged_fieldsThe tagged fields
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request.
recordsThe record data.
_tagged_fields
Tag Tagged field Description
0diverging_epochIn case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge.
Field Description
epochThe largest epoch.
end_offsetThe end offset of the epoch.
_tagged_fieldsThe tagged fields
1current_leaderThe current leader of the partition.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
2snapshot_idIn the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.
Field Description
end_offsetThe end offset of the epoch.
epochThe largest epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Fetch Response (Version: 15) => throttle_time_ms error_code session_id [responses] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records _tagged_fields 
      partition_index => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset _tagged_fields 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      records => COMPACT_RECORDS

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topic_idThe unique topic ID.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
_tagged_fieldsThe tagged fields
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request.
recordsThe record data.
_tagged_fields
Tag Tagged field Description
0diverging_epochIn case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge.
Field Description
epochThe largest epoch.
end_offsetThe end offset of the epoch.
_tagged_fieldsThe tagged fields
1current_leaderThe current leader of the partition.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
2snapshot_idIn the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.
Field Description
end_offsetThe end offset of the epoch.
epochThe largest epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Fetch Response (Version: 16) => throttle_time_ms error_code session_id [responses] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records _tagged_fields 
      partition_index => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset _tagged_fields 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      records => COMPACT_RECORDS

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
topic_idThe unique topic ID.
partitionsThe topic partitions.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
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_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
_tagged_fieldsThe tagged fields
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request.
recordsThe record data.
_tagged_fields
Tag Tagged field Description
0diverging_epochIn case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge.
Field Description
epochThe largest epoch.
end_offsetThe end offset of the epoch.
_tagged_fieldsThe tagged fields
1current_leaderThe current leader of the partition.
Field Description
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
_tagged_fieldsThe tagged fields
2snapshot_idIn the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.
Field Description
end_offsetThe end offset of the epoch.
epochThe largest epoch.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fields
Tag Tagged field Description
0node_endpointsEndpoints for all current-leaders enumerated in PartitionData, with errors NOT_LEADER_OR_FOLLOWER & FENCED_LEADER_EPOCH.
Field Description
node_idThe ID of the associated node.
hostThe node's hostname.
portThe node's port.
rackThe rack of the node, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
ListOffsets API (Key: 2):
Requests:
ListOffsets Request (Version: 1) => replica_id [topics] 
  replica_id => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp 
      partition_index => INT32
      timestamp => INT64

Request header version: 1

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
timestampThe current timestamp.
ListOffsets Request (Version: 2) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp 
      partition_index => INT32
      timestamp => INT64

Request header version: 1

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
timestampThe current timestamp.
ListOffsets Request (Version: 3) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp 
      partition_index => INT32
      timestamp => INT64

Request header version: 1

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
timestampThe current timestamp.
ListOffsets Request (Version: 4) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index current_leader_epoch timestamp 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64

Request header version: 1

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
ListOffsets Request (Version: 5) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index current_leader_epoch timestamp 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64

Request header version: 1

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
ListOffsets Request (Version: 6) => replica_id isolation_level [topics] _tagged_fields 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index current_leader_epoch timestamp _tagged_fields 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64

Request header version: 2

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Request (Version: 7) => replica_id isolation_level [topics] _tagged_fields 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index current_leader_epoch timestamp _tagged_fields 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64

Request header version: 2

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Request (Version: 8) => replica_id isolation_level [topics] _tagged_fields 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index current_leader_epoch timestamp _tagged_fields 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64

Request header version: 2

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Request (Version: 9) => replica_id isolation_level [topics] _tagged_fields 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index current_leader_epoch timestamp _tagged_fields 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64

Request header version: 2

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Request (Version: 10) => replica_id isolation_level [topics] timeout_ms _tagged_fields 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index current_leader_epoch timestamp _tagged_fields 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64
  timeout_ms => INT32

Request header version: 2

Field Description
replica_idThe broker ID of the requester, or -1 if this request is being made by a normal consumer.
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.
topicsEach topic in the request.
nameThe topic name.
partitionsEach partition in the request.
partition_indexThe partition index.
current_leader_epochThe current leader epoch.
timestampThe current timestamp.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msThe timeout to await a response in milliseconds for requests that require reading from remote storage for topics enabled with tiered storage.
_tagged_fieldsThe tagged fields
Responses:
ListOffsets Response (Version: 1) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64

Response header version: 0

Field Description
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
ListOffsets Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
ListOffsets Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
ListOffsets Response (Version: 4) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset leader_epoch 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
leader_epochThe leader epoch associated with the returned offset.
ListOffsets Response (Version: 5) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset leader_epoch 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
leader_epochThe leader epoch associated with the returned offset.
ListOffsets Response (Version: 6) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code timestamp offset leader_epoch _tagged_fields 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
leader_epochThe leader epoch associated with the returned offset.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Response (Version: 7) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code timestamp offset leader_epoch _tagged_fields 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
leader_epochThe leader epoch associated with the returned offset.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Response (Version: 8) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code timestamp offset leader_epoch _tagged_fields 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
leader_epochThe leader epoch associated with the returned offset.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListOffsets Response (Version: 9) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code timestamp offset leader_epoch _tagged_fields 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic in the response.
nameThe topic name.
partitionsEach partition in the response.
partition_indexThe partition index.
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
offsetThe returned offset.
leader_epochThe leader epoch associated with the returned offset.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Metadata API (Key: 3):
Requests:
Metadata Request (Version: 0) => [topics] 
  topics => name 
    name => STRING

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
Metadata Request (Version: 1) => [topics] 
  topics => name 
    name => STRING

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
Metadata Request (Version: 2) => [topics] 
  topics => name 
    name => STRING

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
Metadata Request (Version: 3) => [topics] 
  topics => name 
    name => STRING

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
Metadata Request (Version: 4) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
Metadata Request (Version: 5) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
Metadata Request (Version: 6) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
Metadata Request (Version: 7) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
Metadata Request (Version: 8) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN
  include_cluster_authorized_operations => BOOLEAN
  include_topic_authorized_operations => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
include_cluster_authorized_operationsWhether to include cluster authorized operations.
include_topic_authorized_operationsWhether to include topic authorized operations.
Metadata Request (Version: 9) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations _tagged_fields 
  topics => name _tagged_fields 
    name => COMPACT_STRING
  allow_auto_topic_creation => BOOLEAN
  include_cluster_authorized_operations => BOOLEAN
  include_topic_authorized_operations => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to fetch metadata for.
nameThe topic name.
_tagged_fieldsThe tagged fields
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
include_cluster_authorized_operationsWhether to include cluster authorized operations.
include_topic_authorized_operationsWhether to include topic authorized operations.
_tagged_fieldsThe tagged fields
Metadata Request (Version: 10) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations _tagged_fields 
  topics => topic_id name _tagged_fields 
    topic_id => UUID
    name => COMPACT_NULLABLE_STRING
  allow_auto_topic_creation => BOOLEAN
  include_cluster_authorized_operations => BOOLEAN
  include_topic_authorized_operations => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to fetch metadata for.
topic_idThe topic id.
nameThe topic name.
_tagged_fieldsThe tagged fields
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
include_cluster_authorized_operationsWhether to include cluster authorized operations.
include_topic_authorized_operationsWhether to include topic authorized operations.
_tagged_fieldsThe tagged fields
Metadata Request (Version: 11) => [topics] allow_auto_topic_creation include_topic_authorized_operations _tagged_fields 
  topics => topic_id name _tagged_fields 
    topic_id => UUID
    name => COMPACT_NULLABLE_STRING
  allow_auto_topic_creation => BOOLEAN
  include_topic_authorized_operations => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to fetch metadata for.
topic_idThe topic id.
nameThe topic name.
_tagged_fieldsThe tagged fields
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
include_topic_authorized_operationsWhether to include topic authorized operations.
_tagged_fieldsThe tagged fields
Metadata Request (Version: 12) => [topics] allow_auto_topic_creation include_topic_authorized_operations _tagged_fields 
  topics => topic_id name _tagged_fields 
    topic_id => UUID
    name => COMPACT_NULLABLE_STRING
  allow_auto_topic_creation => BOOLEAN
  include_topic_authorized_operations => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to fetch metadata for.
topic_idThe topic id.
nameThe topic name.
_tagged_fieldsThe tagged fields
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
include_topic_authorized_operationsWhether to include topic authorized operations.
_tagged_fieldsThe tagged fields
Metadata Request (Version: 13) => [topics] allow_auto_topic_creation include_topic_authorized_operations _tagged_fields 
  topics => topic_id name _tagged_fields 
    topic_id => UUID
    name => COMPACT_NULLABLE_STRING
  allow_auto_topic_creation => BOOLEAN
  include_topic_authorized_operations => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to fetch metadata for.
topic_idThe topic id.
nameThe topic name.
_tagged_fieldsThe tagged fields
allow_auto_topic_creationIf this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.
include_topic_authorized_operationsWhether to include topic authorized operations.
_tagged_fieldsThe tagged fields
Responses:
Metadata Response (Version: 0) => [brokers] [topics] 
  brokers => node_id host port 
    node_id => INT32
    host => STRING
    port => INT32
  topics => error_code name [partitions] 
    error_code => INT16
    name => STRING
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32

Response header version: 0

Field Description
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
Metadata Response (Version: 1) => [brokers] controller_id [topics] 
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32

Response header version: 0

Field Description
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
Metadata Response (Version: 2) => [brokers] cluster_id controller_id [topics] 
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32

Response header version: 0

Field Description
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe 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 [topics] 
  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
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe 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 [topics] 
  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
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe 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 [topics] 
  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
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe 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 [topics] 
  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
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.
Metadata Response (Version: 7) => throttle_time_ms [brokers] cluster_id controller_id [topics] 
  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
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
leader_epochThe leader epoch of this partition.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe 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: 8) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations 
  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
  topics => error_code name is_internal [partitions] topic_authorized_operations 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32
  cluster_authorized_operations => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
leader_epochThe leader epoch of this partition.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.
topic_authorized_operations32-bit bitfield to represent authorized operations for this topic.
cluster_authorized_operations32-bit bitfield to represent authorized operations for this cluster.
Metadata Response (Version: 9) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations _tagged_fields 
  throttle_time_ms => INT32
  brokers => node_id host port rack _tagged_fields 
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_id => COMPACT_NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] topic_authorized_operations _tagged_fields 
    error_code => INT16
    name => COMPACT_STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] _tagged_fields 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32
  cluster_authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
leader_epochThe leader epoch of this partition.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.
_tagged_fieldsThe tagged fields
topic_authorized_operations32-bit bitfield to represent authorized operations for this topic.
_tagged_fieldsThe tagged fields
cluster_authorized_operations32-bit bitfield to represent authorized operations for this cluster.
_tagged_fieldsThe tagged fields
Metadata Response (Version: 10) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations _tagged_fields 
  throttle_time_ms => INT32
  brokers => node_id host port rack _tagged_fields 
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_id => COMPACT_NULLABLE_STRING
  controller_id => INT32
  topics => error_code name topic_id is_internal [partitions] topic_authorized_operations _tagged_fields 
    error_code => INT16
    name => COMPACT_STRING
    topic_id => UUID
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] _tagged_fields 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32
  cluster_authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
topic_idThe topic id. Zero for non-existing topics queried by name. This is never zero when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
leader_epochThe leader epoch of this partition.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.
_tagged_fieldsThe tagged fields
topic_authorized_operations32-bit bitfield to represent authorized operations for this topic.
_tagged_fieldsThe tagged fields
cluster_authorized_operations32-bit bitfield to represent authorized operations for this cluster.
_tagged_fieldsThe tagged fields
Metadata Response (Version: 11) => throttle_time_ms [brokers] cluster_id controller_id [topics] _tagged_fields 
  throttle_time_ms => INT32
  brokers => node_id host port rack _tagged_fields 
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_id => COMPACT_NULLABLE_STRING
  controller_id => INT32
  topics => error_code name topic_id is_internal [partitions] topic_authorized_operations _tagged_fields 
    error_code => INT16
    name => COMPACT_STRING
    topic_id => UUID
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] _tagged_fields 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
topic_idThe topic id. Zero for non-existing topics queried by name. This is never zero when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
leader_epochThe leader epoch of this partition.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.
_tagged_fieldsThe tagged fields
topic_authorized_operations32-bit bitfield to represent authorized operations for this topic.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Metadata Response (Version: 12) => throttle_time_ms [brokers] cluster_id controller_id [topics] _tagged_fields 
  throttle_time_ms => INT32
  brokers => node_id host port rack _tagged_fields 
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_id => COMPACT_NULLABLE_STRING
  controller_id => INT32
  topics => error_code name topic_id is_internal [partitions] topic_authorized_operations _tagged_fields 
    error_code => INT16
    name => COMPACT_NULLABLE_STRING
    topic_id => UUID
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] _tagged_fields 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
brokersA list of brokers present in the cluster.
node_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
topicsEach topic in the response.
error_codeThe topic error, or 0 if there was no error.
nameThe topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated.
topic_idThe topic id. Zero for non-existing topics queried by name. This is never zero when ErrorCode is zero. One of Name and TopicId is always populated.
is_internalTrue if the topic is internal.
partitionsEach partition in the topic.
error_codeThe partition error, or 0 if there was no error.
partition_indexThe partition index.
leader_idThe ID of the leader broker.
leader_epochThe leader epoch of this partition.
replica_nodesThe set of all nodes that host this partition.
isr_nodesThe set of nodes that are in sync with the leader for this partition.
offline_replicasThe set of offline replicas of this partition.
_tagged_fieldsThe tagged fields
topic_authorized_operations32-bit bitfield to represent authorized operations for this topic.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
OffsetCommit API (Key: 8):
Requests:
OffsetCommit Request (Version: 2) => group_id generation_id_or_member_epoch member_id retention_time_ms [topics] 
  group_id => STRING
  generation_id_or_member_epoch => INT32
  member_id => STRING
  retention_time_ms => INT64
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
retention_time_msThe time period in ms to retain the offset.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_metadataAny associated metadata the client wants to keep.
OffsetCommit Request (Version: 3) => group_id generation_id_or_member_epoch member_id retention_time_ms [topics] 
  group_id => STRING
  generation_id_or_member_epoch => INT32
  member_id => STRING
  retention_time_ms => INT64
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
retention_time_msThe time period in ms to retain the offset.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_metadataAny associated metadata the client wants to keep.
OffsetCommit Request (Version: 4) => group_id generation_id_or_member_epoch member_id retention_time_ms [topics] 
  group_id => STRING
  generation_id_or_member_epoch => INT32
  member_id => STRING
  retention_time_ms => INT64
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
retention_time_msThe time period in ms to retain the offset.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_metadataAny associated metadata the client wants to keep.
OffsetCommit Request (Version: 5) => group_id generation_id_or_member_epoch member_id [topics] 
  group_id => STRING
  generation_id_or_member_epoch => INT32
  member_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_metadataAny associated metadata the client wants to keep.
OffsetCommit Request (Version: 6) => group_id generation_id_or_member_epoch member_id [topics] 
  group_id => STRING
  generation_id_or_member_epoch => INT32
  member_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of this partition.
committed_metadataAny associated metadata the client wants to keep.
OffsetCommit Request (Version: 7) => group_id generation_id_or_member_epoch member_id group_instance_id [topics] 
  group_id => STRING
  generation_id_or_member_epoch => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of this partition.
committed_metadataAny associated metadata the client wants to keep.
OffsetCommit Request (Version: 8) => group_id generation_id_or_member_epoch member_id group_instance_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  generation_id_or_member_epoch => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of this partition.
committed_metadataAny associated metadata the client wants to keep.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
OffsetCommit Request (Version: 9) => group_id generation_id_or_member_epoch member_id group_instance_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  generation_id_or_member_epoch => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe unique group identifier.
generation_id_or_member_epochThe generation of the group if using the classic group protocol or the member epoch if using the consumer protocol.
member_idThe member ID assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
topicsThe topics to commit offsets for.
nameThe topic name.
partitionsEach partition to commit offsets for.
partition_indexThe partition index.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of this partition.
committed_metadataAny associated metadata the client wants to keep.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
OffsetCommit Response (Version: 2) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
OffsetCommit Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
OffsetCommit Response (Version: 4) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
OffsetCommit Response (Version: 5) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
OffsetCommit Response (Version: 6) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
OffsetCommit Response (Version: 7) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
OffsetCommit Response (Version: 8) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code _tagged_fields 
      partition_index => INT32
      error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
OffsetFetch API (Key: 9):
Requests:
OffsetFetch Request (Version: 1) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32

Request header version: 1

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
OffsetFetch Request (Version: 2) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32

Request header version: 1

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
OffsetFetch Request (Version: 3) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32

Request header version: 1

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
OffsetFetch Request (Version: 4) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32

Request header version: 1

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
OffsetFetch Request (Version: 5) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32

Request header version: 1

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
OffsetFetch Request (Version: 6) => group_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  topics => name [partition_indexes] _tagged_fields 
    name => COMPACT_STRING
    partition_indexes => INT32

Request header version: 2

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
OffsetFetch Request (Version: 7) => group_id [topics] require_stable _tagged_fields 
  group_id => COMPACT_STRING
  topics => name [partition_indexes] _tagged_fields 
    name => COMPACT_STRING
    partition_indexes => INT32
  require_stable => BOOLEAN

Request header version: 2

Field Description
group_idThe group to fetch offsets for.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
_tagged_fieldsThe tagged fields
require_stableWhether broker should hold on returning unstable offsets but set a retriable error code for the partitions.
_tagged_fieldsThe tagged fields
OffsetFetch Request (Version: 8) => [groups] require_stable _tagged_fields 
  groups => group_id [topics] _tagged_fields 
    group_id => COMPACT_STRING
    topics => name [partition_indexes] _tagged_fields 
      name => COMPACT_STRING
      partition_indexes => INT32
  require_stable => BOOLEAN

Request header version: 2

Field Description
groupsEach group we would like to fetch offsets for.
group_idThe group ID.
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
require_stableWhether broker should hold on returning unstable offsets but set a retriable error code for the partitions.
_tagged_fieldsThe tagged fields
OffsetFetch Request (Version: 9) => [groups] require_stable _tagged_fields 
  groups => group_id member_id member_epoch [topics] _tagged_fields 
    group_id => COMPACT_STRING
    member_id => COMPACT_NULLABLE_STRING
    member_epoch => INT32
    topics => name [partition_indexes] _tagged_fields 
      name => COMPACT_STRING
      partition_indexes => INT32
  require_stable => BOOLEAN

Request header version: 2

Field Description
groupsEach group we would like to fetch offsets for.
group_idThe group ID.
member_idThe member id.
member_epochThe member epoch if using the new consumer protocol (KIP-848).
topicsEach topic we would like to fetch offsets for, or null to fetch offsets for all topics.
nameThe topic name.
partition_indexesThe partition indexes we would like to fetch offsets for.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
require_stableWhether broker should hold on returning unstable offsets but set a retriable error code for the partitions.
_tagged_fieldsThe tagged fields
Responses:
OffsetFetch Response (Version: 1) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16

Response header version: 0

Field Description
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
OffsetFetch Response (Version: 2) => [topics] error_code 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16

Response header version: 0

Field Description
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
error_codeThe top-level error code, or 0 if there was no error.
OffsetFetch Response (Version: 3) => throttle_time_ms [topics] error_code 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
error_codeThe top-level error code, or 0 if there was no error.
OffsetFetch Response (Version: 4) => throttle_time_ms [topics] error_code 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
error_codeThe top-level error code, or 0 if there was no error.
OffsetFetch Response (Version: 5) => throttle_time_ms [topics] error_code 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
committed_leader_epochThe leader epoch.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
error_codeThe top-level error code, or 0 if there was no error.
OffsetFetch Response (Version: 6) => throttle_time_ms [topics] error_code _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch metadata error_code _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      metadata => COMPACT_NULLABLE_STRING
      error_code => INT16
  error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
committed_leader_epochThe leader epoch.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
error_codeThe top-level error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
OffsetFetch Response (Version: 7) => throttle_time_ms [topics] error_code _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch metadata error_code _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      metadata => COMPACT_NULLABLE_STRING
      error_code => INT16
  error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
committed_leader_epochThe leader epoch.
metadataThe partition metadata.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
error_codeThe top-level error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
OffsetFetch Response (Version: 8) => throttle_time_ms [groups] _tagged_fields 
  throttle_time_ms => INT32
  groups => group_id [topics] error_code _tagged_fields 
    group_id => COMPACT_STRING
    topics => name [partitions] _tagged_fields 
      name => COMPACT_STRING
      partitions => partition_index committed_offset committed_leader_epoch metadata error_code _tagged_fields 
        partition_index => INT32
        committed_offset => INT64
        committed_leader_epoch => INT32
        metadata => COMPACT_NULLABLE_STRING
        error_code => INT16
    error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsThe responses per group id.
group_idThe group ID.
topicsThe responses per topic.
nameThe topic name.
partitionsThe responses per partition.
partition_indexThe partition index.
committed_offsetThe committed message offset.
committed_leader_epochThe leader epoch.
metadataThe partition metadata.
error_codeThe partition-level error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
error_codeThe group-level error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
FindCoordinator API (Key: 10):
Requests:
FindCoordinator Request (Version: 0) => key 
  key => STRING

Request header version: 1

Field Description
keyThe coordinator key.
FindCoordinator Request (Version: 1) => key key_type 
  key => STRING
  key_type => INT8

Request header version: 1

Field Description
keyThe coordinator key.
key_typeThe coordinator key type. (group, transaction, share).
FindCoordinator Request (Version: 2) => key key_type 
  key => STRING
  key_type => INT8

Request header version: 1

Field Description
keyThe coordinator key.
key_typeThe coordinator key type. (group, transaction, share).
FindCoordinator Request (Version: 3) => key key_type _tagged_fields 
  key => COMPACT_STRING
  key_type => INT8

Request header version: 2

Field Description
keyThe coordinator key.
key_typeThe coordinator key type. (group, transaction, share).
_tagged_fieldsThe tagged fields
FindCoordinator Request (Version: 4) => key_type [coordinator_keys] _tagged_fields 
  key_type => INT8
  coordinator_keys => COMPACT_STRING

Request header version: 2

Field Description
key_typeThe coordinator key type. (group, transaction, share).
coordinator_keysThe coordinator keys.
_tagged_fieldsThe tagged fields
FindCoordinator Request (Version: 5) => key_type [coordinator_keys] _tagged_fields 
  key_type => INT8
  coordinator_keys => COMPACT_STRING

Request header version: 2

Field Description
key_typeThe coordinator key type. (group, transaction, share).
coordinator_keysThe coordinator keys.
_tagged_fieldsThe tagged fields
FindCoordinator Request (Version: 6) => key_type [coordinator_keys] _tagged_fields 
  key_type => INT8
  coordinator_keys => COMPACT_STRING

Request header version: 2

Field Description
key_typeThe coordinator key type. (group, transaction, share).
coordinator_keysThe coordinator keys.
_tagged_fieldsThe tagged fields
Responses:
FindCoordinator Response (Version: 0) => error_code node_id host port 
  error_code => INT16
  node_id => INT32
  host => STRING
  port => INT32

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
node_idThe node id.
hostThe host name.
portThe port.
FindCoordinator Response (Version: 1) => throttle_time_ms error_code error_message node_id host port 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  node_id => INT32
  host => STRING
  port => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
node_idThe node id.
hostThe host name.
portThe port.
FindCoordinator Response (Version: 2) => throttle_time_ms error_code error_message node_id host port 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  node_id => INT32
  host => STRING
  port => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
node_idThe node id.
hostThe host name.
portThe port.
FindCoordinator Response (Version: 3) => throttle_time_ms error_code error_message node_id host port _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  node_id => INT32
  host => COMPACT_STRING
  port => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
node_idThe node id.
hostThe host name.
portThe port.
_tagged_fieldsThe tagged fields
FindCoordinator Response (Version: 4) => throttle_time_ms [coordinators] _tagged_fields 
  throttle_time_ms => INT32
  coordinators => key node_id host port error_code error_message _tagged_fields 
    key => COMPACT_STRING
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
coordinatorsEach coordinator result in the response.
keyThe coordinator key.
node_idThe node id.
hostThe host name.
portThe port.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
FindCoordinator Response (Version: 5) => throttle_time_ms [coordinators] _tagged_fields 
  throttle_time_ms => INT32
  coordinators => key node_id host port error_code error_message _tagged_fields 
    key => COMPACT_STRING
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
coordinatorsEach coordinator result in the response.
keyThe coordinator key.
node_idThe node id.
hostThe host name.
portThe port.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
JoinGroup API (Key: 11):
Requests:
JoinGroup Request (Version: 2) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES

Request header version: 1

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
JoinGroup Request (Version: 3) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES

Request header version: 1

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
JoinGroup Request (Version: 4) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES

Request header version: 1

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
JoinGroup Request (Version: 5) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES

Request header version: 1

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
JoinGroup Request (Version: 6) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] _tagged_fields 
  group_id => COMPACT_STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_STRING
  protocols => name metadata _tagged_fields 
    name => COMPACT_STRING
    metadata => COMPACT_BYTES

Request header version: 2

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
JoinGroup Request (Version: 7) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] _tagged_fields 
  group_id => COMPACT_STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_STRING
  protocols => name metadata _tagged_fields 
    name => COMPACT_STRING
    metadata => COMPACT_BYTES

Request header version: 2

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
JoinGroup Request (Version: 8) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] reason _tagged_fields 
  group_id => COMPACT_STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_STRING
  protocols => name metadata _tagged_fields 
    name => COMPACT_STRING
    metadata => COMPACT_BYTES
  reason => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
_tagged_fieldsThe tagged fields
reasonThe reason why the member (re-)joins the group.
_tagged_fieldsThe tagged fields
JoinGroup Request (Version: 9) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] reason _tagged_fields 
  group_id => COMPACT_STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_STRING
  protocols => name metadata _tagged_fields 
    name => COMPACT_STRING
    metadata => COMPACT_BYTES
  reason => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe group identifier.
session_timeout_msThe coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.
rebalance_timeout_msThe maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.
member_idThe member id assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
protocol_typeThe unique name the for class of protocols implemented by the group we want to join.
protocolsThe list of protocols that the member supports.
nameThe protocol name.
metadataThe protocol metadata.
_tagged_fieldsThe tagged fields
reasonThe reason why the member (re-)joins the group.
_tagged_fieldsThe tagged fields
Responses:
JoinGroup Response (Version: 2) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
metadataThe group member metadata.
JoinGroup Response (Version: 3) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
metadataThe group member metadata.
JoinGroup Response (Version: 4) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
metadataThe group member metadata.
JoinGroup Response (Version: 5) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id group_instance_id metadata 
    member_id => STRING
    group_instance_id => NULLABLE_STRING
    metadata => BYTES

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
group_instance_idThe unique identifier of the consumer instance provided by end user.
metadataThe group member metadata.
JoinGroup Response (Version: 6) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => COMPACT_STRING
  leader => COMPACT_STRING
  member_id => COMPACT_STRING
  members => member_id group_instance_id metadata _tagged_fields 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    metadata => COMPACT_BYTES

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
group_instance_idThe unique identifier of the consumer instance provided by end user.
metadataThe group member metadata.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
JoinGroup Response (Version: 7) => throttle_time_ms error_code generation_id protocol_type protocol_name leader member_id [members] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_type => COMPACT_NULLABLE_STRING
  protocol_name => COMPACT_NULLABLE_STRING
  leader => COMPACT_STRING
  member_id => COMPACT_STRING
  members => member_id group_instance_id metadata _tagged_fields 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    metadata => COMPACT_BYTES

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_typeThe group protocol name.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
group_instance_idThe unique identifier of the consumer instance provided by end user.
metadataThe group member metadata.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
JoinGroup Response (Version: 8) => throttle_time_ms error_code generation_id protocol_type protocol_name leader member_id [members] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_type => COMPACT_NULLABLE_STRING
  protocol_name => COMPACT_NULLABLE_STRING
  leader => COMPACT_STRING
  member_id => COMPACT_STRING
  members => member_id group_instance_id metadata _tagged_fields 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    metadata => COMPACT_BYTES

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
generation_idThe generation ID of the group.
protocol_typeThe group protocol name.
protocol_nameThe group protocol selected by the coordinator.
leaderThe leader of the group.
member_idThe member ID assigned by the group coordinator.
membersThe group members.
member_idThe group member ID.
group_instance_idThe unique identifier of the consumer instance provided by end user.
metadataThe group member metadata.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Heartbeat API (Key: 12):
Requests:
Heartbeat Request (Version: 0) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING

Request header version: 1

Field Description
group_idThe group id.
generation_idThe generation of the group.
member_idThe member ID.
Heartbeat Request (Version: 1) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING

Request header version: 1

Field Description
group_idThe group id.
generation_idThe generation of the group.
member_idThe member ID.
Heartbeat Request (Version: 2) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING

Request header version: 1

Field Description
group_idThe group id.
generation_idThe generation of the group.
member_idThe member ID.
Heartbeat Request (Version: 3) => group_id generation_id member_id group_instance_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe group id.
generation_idThe generation of the group.
member_idThe member ID.
group_instance_idThe unique identifier of the consumer instance provided by end user.
Heartbeat Request (Version: 4) => group_id generation_id member_id group_instance_id _tagged_fields 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe group id.
generation_idThe generation of the group.
member_idThe member ID.
group_instance_idThe unique identifier of the consumer instance provided by end user.
_tagged_fieldsThe tagged fields
Responses:
Heartbeat Response (Version: 0) => error_code 
  error_code => INT16

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
Heartbeat Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16

Response header version: 0

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

Response header version: 0

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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
LeaveGroup API (Key: 13):
Requests:
LeaveGroup Request (Version: 0) => group_id member_id 
  group_id => STRING
  member_id => STRING

Request header version: 1

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

Request header version: 1

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

Request header version: 1

Field Description
group_idThe ID of the group to leave.
member_idThe member ID to remove from the group.
LeaveGroup Request (Version: 3) => group_id [members] 
  group_id => STRING
  members => member_id group_instance_id 
    member_id => STRING
    group_instance_id => NULLABLE_STRING

Request header version: 1

Field Description
group_idThe ID of the group to leave.
membersList of leaving member identities.
member_idThe member ID to remove from the group.
group_instance_idThe group instance ID to remove from the group.
LeaveGroup Request (Version: 4) => group_id [members] _tagged_fields 
  group_id => COMPACT_STRING
  members => member_id group_instance_id _tagged_fields 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe ID of the group to leave.
membersList of leaving member identities.
member_idThe member ID to remove from the group.
group_instance_idThe group instance ID to remove from the group.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
LeaveGroup Request (Version: 5) => group_id [members] _tagged_fields 
  group_id => COMPACT_STRING
  members => member_id group_instance_id reason _tagged_fields 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    reason => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
group_idThe ID of the group to leave.
membersList of leaving member identities.
member_idThe member ID to remove from the group.
group_instance_idThe group instance ID to remove from the group.
reasonThe reason why the member left the group.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
LeaveGroup Response (Version: 0) => error_code 
  error_code => INT16

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
LeaveGroup Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16

Response header version: 0

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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
LeaveGroup Response (Version: 3) => throttle_time_ms error_code [members] 
  throttle_time_ms => INT32
  error_code => INT16
  members => member_id group_instance_id error_code 
    member_id => STRING
    group_instance_id => NULLABLE_STRING
    error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
membersList of leaving member responses.
member_idThe member ID to remove from the group.
group_instance_idThe group instance ID to remove from the group.
error_codeThe error code, or 0 if there was no error.
LeaveGroup Response (Version: 4) => throttle_time_ms error_code [members] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  members => member_id group_instance_id error_code _tagged_fields 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
membersList of leaving member responses.
member_idThe member ID to remove from the group.
group_instance_idThe group instance ID to remove from the group.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
SyncGroup API (Key: 14):
Requests:
SyncGroup Request (Version: 0) => group_id generation_id member_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_idThe generation of the group.
member_idThe member ID assigned by the group.
assignmentsEach assignment.
member_idThe ID of the member to assign.
assignmentThe member assignment.
SyncGroup Request (Version: 1) => group_id generation_id member_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_idThe generation of the group.
member_idThe member ID assigned by the group.
assignmentsEach assignment.
member_idThe ID of the member to assign.
assignmentThe member assignment.
SyncGroup Request (Version: 2) => group_id generation_id member_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_idThe generation of the group.
member_idThe member ID assigned by the group.
assignmentsEach assignment.
member_idThe ID of the member to assign.
assignmentThe member assignment.
SyncGroup Request (Version: 3) => group_id generation_id member_id group_instance_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES

Request header version: 1

Field Description
group_idThe unique group identifier.
generation_idThe generation of the group.
member_idThe member ID assigned by the group.
group_instance_idThe unique identifier of the consumer instance provided by end user.
assignmentsEach assignment.
member_idThe ID of the member to assign.
assignmentThe member assignment.
SyncGroup Request (Version: 4) => group_id generation_id member_id group_instance_id [assignments] _tagged_fields 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  assignments => member_id assignment _tagged_fields 
    member_id => COMPACT_STRING
    assignment => COMPACT_BYTES

Request header version: 2

Field Description
group_idThe unique group identifier.
generation_idThe generation of the group.
member_idThe member ID assigned by the group.
group_instance_idThe unique identifier of the consumer instance provided by end user.
assignmentsEach assignment.
member_idThe ID of the member to assign.
assignmentThe member assignment.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
SyncGroup Request (Version: 5) => group_id generation_id member_id group_instance_id protocol_type protocol_name [assignments] _tagged_fields 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_NULLABLE_STRING
  protocol_name => COMPACT_NULLABLE_STRING
  assignments => member_id assignment _tagged_fields 
    member_id => COMPACT_STRING
    assignment => COMPACT_BYTES

Request header version: 2

Field Description
group_idThe unique group identifier.
generation_idThe generation of the group.
member_idThe member ID assigned by the group.
group_instance_idThe unique identifier of the consumer instance provided by end user.
protocol_typeThe group protocol type.
protocol_nameThe group protocol name.
assignmentsEach assignment.
member_idThe ID of the member to assign.
assignmentThe member assignment.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
SyncGroup Response (Version: 0) => error_code assignment 
  error_code => INT16
  assignment => BYTES

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
assignmentThe member assignment.
SyncGroup Response (Version: 1) => throttle_time_ms error_code assignment 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => BYTES

Response header version: 0

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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
assignmentThe member assignment.
SyncGroup Response (Version: 3) => throttle_time_ms error_code assignment 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => BYTES

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
assignmentThe member assignment.
SyncGroup Response (Version: 4) => throttle_time_ms error_code assignment _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => COMPACT_BYTES

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
assignmentThe member assignment.
_tagged_fieldsThe tagged fields
DescribeGroups API (Key: 15):
Requests:
DescribeGroups Request (Version: 0) => [groups] 
  groups => STRING

Request header version: 1

Field Description
groupsThe names of the groups to describe.
DescribeGroups Request (Version: 1) => [groups] 
  groups => STRING

Request header version: 1

Field Description
groupsThe names of the groups to describe.
DescribeGroups Request (Version: 2) => [groups] 
  groups => STRING

Request header version: 1

Field Description
groupsThe names of the groups to describe.
DescribeGroups Request (Version: 3) => [groups] include_authorized_operations 
  groups => STRING
  include_authorized_operations => BOOLEAN

Request header version: 1

Field Description
groupsThe names of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
DescribeGroups Request (Version: 4) => [groups] include_authorized_operations 
  groups => STRING
  include_authorized_operations => BOOLEAN

Request header version: 1

Field Description
groupsThe names of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
DescribeGroups Request (Version: 5) => [groups] include_authorized_operations _tagged_fields 
  groups => COMPACT_STRING
  include_authorized_operations => BOOLEAN

Request header version: 2

Field Description
groupsThe names of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
_tagged_fieldsThe tagged fields
DescribeGroups Request (Version: 6) => [groups] include_authorized_operations _tagged_fields 
  groups => COMPACT_STRING
  include_authorized_operations => BOOLEAN

Request header version: 2

Field Description
groupsThe names of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
_tagged_fieldsThe tagged fields
Responses:
DescribeGroups Response (Version: 0) => [groups] 
  groups => error_code group_id group_state protocol_type protocol_data [members] 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => 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

Response header version: 0

Field Description
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
protocol_typeThe group protocol type, or the empty string.
protocol_dataThe group protocol data, or the empty string.
membersThe group members.
member_idThe member id.
client_idThe client ID used in the member's latest join group request.
client_hostThe client host.
member_metadataThe metadata corresponding to the current group protocol in use.
member_assignmentThe current assignment provided by the group leader.
DescribeGroups Response (Version: 1) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => 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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
protocol_typeThe group protocol type, or the empty string.
protocol_dataThe group protocol data, or the empty string.
membersThe group members.
member_idThe member id.
client_idThe client ID used in the member's latest join group request.
client_hostThe client host.
member_metadataThe metadata corresponding to the current group protocol in use.
member_assignmentThe current assignment provided by the group leader.
DescribeGroups Response (Version: 2) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => 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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
protocol_typeThe group protocol type, or the empty string.
protocol_dataThe group protocol data, or the empty string.
membersThe group members.
member_idThe member id.
client_idThe client ID used in the member's latest join group request.
client_hostThe client host.
member_metadataThe metadata corresponding to the current group protocol in use.
member_assignmentThe current assignment provided by the group leader.
DescribeGroups Response (Version: 3) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => 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
    authorized_operations => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
protocol_typeThe group protocol type, or the empty string.
protocol_dataThe group protocol data, or the empty string.
membersThe group members.
member_idThe member id.
client_idThe client ID used in the member's latest join group request.
client_hostThe client host.
member_metadataThe metadata corresponding to the current group protocol in use.
member_assignmentThe current assignment provided by the group leader.
authorized_operations32-bit bitfield to represent authorized operations for this group.
DescribeGroups Response (Version: 4) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => STRING
    members => member_id group_instance_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      group_instance_id => NULLABLE_STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
    authorized_operations => INT32

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
protocol_typeThe group protocol type, or the empty string.
protocol_dataThe group protocol data, or the empty string.
membersThe group members.
member_idThe member id.
group_instance_idThe unique identifier of the consumer instance provided by end user.
client_idThe client ID used in the member's latest join group request.
client_hostThe client host.
member_metadataThe metadata corresponding to the current group protocol in use.
member_assignmentThe current assignment provided by the group leader.
authorized_operations32-bit bitfield to represent authorized operations for this group.
DescribeGroups Response (Version: 5) => throttle_time_ms [groups] _tagged_fields 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations _tagged_fields 
    error_code => INT16
    group_id => COMPACT_STRING
    group_state => COMPACT_STRING
    protocol_type => COMPACT_STRING
    protocol_data => COMPACT_STRING
    members => member_id group_instance_id client_id client_host member_metadata member_assignment _tagged_fields 
      member_id => COMPACT_STRING
      group_instance_id => COMPACT_NULLABLE_STRING
      client_id => COMPACT_STRING
      client_host => COMPACT_STRING
      member_metadata => COMPACT_BYTES
      member_assignment => COMPACT_BYTES
    authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
protocol_typeThe group protocol type, or the empty string.
protocol_dataThe group protocol data, or the empty string.
membersThe group members.
member_idThe member id.
group_instance_idThe unique identifier of the consumer instance provided by end user.
client_idThe client ID used in the member's latest join group request.
client_hostThe client host.
member_metadataThe metadata corresponding to the current group protocol in use.
member_assignmentThe current assignment provided by the group leader.
_tagged_fieldsThe tagged fields
authorized_operations32-bit bitfield to represent authorized operations for this group.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListGroups API (Key: 16):
Requests:
ListGroups Request (Version: 0) => 

Request header version: 1

Field Description
ListGroups Request (Version: 1) => 

Request header version: 1

Field Description
ListGroups Request (Version: 2) => 

Request header version: 1

Field Description
ListGroups Request (Version: 3) => _tagged_fields 

Request header version: 2

Field Description
_tagged_fieldsThe tagged fields
ListGroups Request (Version: 4) => [states_filter] _tagged_fields 
  states_filter => COMPACT_STRING

Request header version: 2

Field Description
states_filterThe states of the groups we want to list. If empty, all groups are returned with their state.
_tagged_fieldsThe tagged fields
ListGroups Request (Version: 5) => [states_filter] [types_filter] _tagged_fields 
  states_filter => COMPACT_STRING
  types_filter => COMPACT_STRING

Request header version: 2

Field Description
states_filterThe states of the groups we want to list. If empty, all groups are returned with their state.
types_filterThe types of the groups we want to list. If empty, all groups are returned with their type.
_tagged_fieldsThe tagged fields
Responses:
ListGroups Response (Version: 0) => error_code [groups] 
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
groupsEach group in the response.
group_idThe group ID.
protocol_typeThe group protocol type.
ListGroups Response (Version: 1) => throttle_time_ms error_code [groups] 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
groupsEach group in the response.
group_idThe group ID.
protocol_typeThe group protocol type.
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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
groupsEach group in the response.
group_idThe group ID.
protocol_typeThe group protocol type.
ListGroups Response (Version: 3) => throttle_time_ms error_code [groups] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type _tagged_fields 
    group_id => COMPACT_STRING
    protocol_type => COMPACT_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
groupsEach group in the response.
group_idThe group ID.
protocol_typeThe group protocol type.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ListGroups Response (Version: 4) => throttle_time_ms error_code [groups] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type group_state _tagged_fields 
    group_id => COMPACT_STRING
    protocol_type => COMPACT_STRING
    group_state => COMPACT_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
groupsEach group in the response.
group_idThe group ID.
protocol_typeThe group protocol type.
group_stateThe group state name.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
SaslHandshake API (Key: 17):
Requests:
SaslHandshake Request (Version: 0) => mechanism 
  mechanism => STRING

Request header version: 1

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

Request header version: 1

Field Description
mechanismThe SASL mechanism chosen by the client.
Responses:
SaslHandshake Response (Version: 0) => error_code [mechanisms] 
  error_code => INT16
  mechanisms => STRING

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
mechanismsThe mechanisms enabled in the server.
ApiVersions API (Key: 18):
Requests:
ApiVersions Request (Version: 0) => 

Request header version: 1

Field Description
ApiVersions Request (Version: 1) => 

Request header version: 1

Field Description
ApiVersions Request (Version: 2) => 

Request header version: 1

Field Description
ApiVersions Request (Version: 3) => client_software_name client_software_version _tagged_fields 
  client_software_name => COMPACT_STRING
  client_software_version => COMPACT_STRING

Request header version: 2

Field Description
client_software_nameThe name of the client.
client_software_versionThe version of the client.
_tagged_fieldsThe tagged fields
ApiVersions Request (Version: 4) => client_software_name client_software_version _tagged_fields 
  client_software_name => COMPACT_STRING
  client_software_version => COMPACT_STRING

Request header version: 2

Field Description
client_software_nameThe name of the client.
client_software_versionThe version of the client.
_tagged_fieldsThe tagged fields
Responses:
ApiVersions Response (Version: 0) => error_code [api_keys] 
  error_code => INT16
  api_keys => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16

Response header version: 0

Field Description
error_codeThe top-level error code.
api_keysThe APIs supported by the broker.
api_keyThe API index.
min_versionThe minimum supported version, inclusive.
max_versionThe maximum supported version, inclusive.
ApiVersions Response (Version: 1) => error_code [api_keys] throttle_time_ms 
  error_code => INT16
  api_keys => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe top-level error code.
api_keysThe APIs supported by the broker.
api_keyThe API index.
min_versionThe minimum supported version, inclusive.
max_versionThe maximum supported version, inclusive.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
ApiVersions Response (Version: 2) => error_code [api_keys] throttle_time_ms 
  error_code => INT16
  api_keys => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe top-level error code.
api_keysThe APIs supported by the broker.
api_keyThe API index.
min_versionThe minimum supported version, inclusive.
max_versionThe maximum supported version, inclusive.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
ApiVersions Response (Version: 3) => error_code [api_keys] throttle_time_ms _tagged_fields 
  error_code => INT16
  api_keys => api_key min_version max_version _tagged_fields 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe top-level error code.
api_keysThe APIs supported by the broker.
api_keyThe API index.
min_versionThe minimum supported version, inclusive.
max_versionThe maximum supported version, inclusive.
_tagged_fieldsThe tagged fields
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
_tagged_fields
Tag Tagged field Description
0supported_featuresFeatures supported by the broker. Note: in v0-v3, features with MinSupportedVersion = 0 are omitted.
Field Description
nameThe name of the feature.
min_versionThe minimum supported version for the feature.
max_versionThe maximum supported version for the feature.
_tagged_fieldsThe tagged fields
1finalized_features_epochThe monotonically increasing epoch for the finalized features information. Valid values are >= 0. A value of -1 is special and represents unknown epoch.
2finalized_featuresList of cluster-wide finalized features. The information is valid only if FinalizedFeaturesEpoch >= 0.
Field Description
nameThe name of the feature.
max_version_levelThe cluster-wide finalized max version level for the feature.
min_version_levelThe cluster-wide finalized min version level for the feature.
_tagged_fieldsThe tagged fields
3zk_migration_readySet by a KRaft controller if the required configurations for ZK migration are present.
CreateTopics API (Key: 19):
Requests:
CreateTopics Request (Version: 2) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to create.
nameThe topic name.
num_partitionsThe number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.
replication_factorThe number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.
assignmentsThe manual partition assignment, or the empty array if we are using automatic assignment.
partition_indexThe partition index.
broker_idsThe brokers to place the partition on.
configsThe custom topic configurations to set.
nameThe configuration name.
valueThe configuration value.
timeout_msHow long to wait in milliseconds before timing out the request.
validate_onlyIf true, check that the topics can be created as specified, but don't create anything.
CreateTopics Request (Version: 3) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to create.
nameThe topic name.
num_partitionsThe number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.
replication_factorThe number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.
assignmentsThe manual partition assignment, or the empty array if we are using automatic assignment.
partition_indexThe partition index.
broker_idsThe brokers to place the partition on.
configsThe custom topic configurations to set.
nameThe configuration name.
valueThe configuration value.
timeout_msHow long to wait in milliseconds before timing out the request.
validate_onlyIf true, check that the topics can be created as specified, but don't create anything.
CreateTopics Request (Version: 4) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 1

Field Description
topicsThe topics to create.
nameThe topic name.
num_partitionsThe number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.
replication_factorThe number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.
assignmentsThe manual partition assignment, or the empty array if we are using automatic assignment.
partition_indexThe partition index.
broker_idsThe brokers to place the partition on.
configsThe custom topic configurations to set.
nameThe configuration name.
valueThe configuration value.
timeout_msHow long to wait in milliseconds before timing out the request.
validate_onlyIf true, check that the topics can be created as specified, but don't create anything.
CreateTopics Request (Version: 5) => [topics] timeout_ms validate_only _tagged_fields 
  topics => name num_partitions replication_factor [assignments] [configs] _tagged_fields 
    name => COMPACT_STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] _tagged_fields 
      partition_index => INT32
      broker_ids => INT32
    configs => name value _tagged_fields 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to create.
nameThe topic name.
num_partitionsThe number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.
replication_factorThe number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.
assignmentsThe manual partition assignment, or the empty array if we are using automatic assignment.
partition_indexThe partition index.
broker_idsThe brokers to place the partition on.
_tagged_fieldsThe tagged fields
configsThe custom topic configurations to set.
nameThe configuration name.
valueThe configuration value.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msHow long to wait in milliseconds before timing out the request.
validate_onlyIf true, check that the topics can be created as specified, but don't create anything.
_tagged_fieldsThe tagged fields
CreateTopics Request (Version: 6) => [topics] timeout_ms validate_only _tagged_fields 
  topics => name num_partitions replication_factor [assignments] [configs] _tagged_fields 
    name => COMPACT_STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] _tagged_fields 
      partition_index => INT32
      broker_ids => INT32
    configs => name value _tagged_fields 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to create.
nameThe topic name.
num_partitionsThe number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.
replication_factorThe number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.
assignmentsThe manual partition assignment, or the empty array if we are using automatic assignment.
partition_indexThe partition index.
broker_idsThe brokers to place the partition on.
_tagged_fieldsThe tagged fields
configsThe custom topic configurations to set.
nameThe configuration name.
valueThe configuration value.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msHow long to wait in milliseconds before timing out the request.
validate_onlyIf true, check that the topics can be created as specified, but don't create anything.
_tagged_fieldsThe tagged fields
CreateTopics Request (Version: 7) => [topics] timeout_ms validate_only _tagged_fields 
  topics => name num_partitions replication_factor [assignments] [configs] _tagged_fields 
    name => COMPACT_STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] _tagged_fields 
      partition_index => INT32
      broker_ids => INT32
    configs => name value _tagged_fields 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 2

Field Description
topicsThe topics to create.
nameThe topic name.
num_partitionsThe number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.
replication_factorThe number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.
assignmentsThe manual partition assignment, or the empty array if we are using automatic assignment.
partition_indexThe partition index.
broker_idsThe brokers to place the partition on.
_tagged_fieldsThe tagged fields
configsThe custom topic configurations to set.
nameThe configuration name.
valueThe configuration value.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msHow long to wait in milliseconds before timing out the request.
validate_onlyIf true, check that the topics can be created as specified, but don't create anything.
_tagged_fieldsThe tagged fields
Responses:
CreateTopics Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsResults for each topic we tried to create.
nameThe topic name.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
CreateTopics Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsResults for each topic we tried to create.
nameThe topic name.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
CreateTopics Response (Version: 4) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsResults for each topic we tried to create.
nameThe topic name.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
CreateTopics Response (Version: 5) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name error_code error_message num_partitions replication_factor [configs] _tagged_fields 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    num_partitions => INT32
    replication_factor => INT16
    configs => name value read_only config_source is_sensitive _tagged_fields 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsResults for each topic we tried to create.
nameThe topic name.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
num_partitionsNumber of partitions of the topic.
replication_factorReplication factor of the topic.
configsConfiguration of the topic.
nameThe configuration name.
valueThe configuration value.
read_onlyTrue if the configuration is read-only.
config_sourceThe configuration source.
is_sensitiveTrue if this configuration is sensitive.
_tagged_fieldsThe tagged fields
_tagged_fields
Tag Tagged field Description
0topic_config_error_codeOptional topic config error returned if configs are not returned in the response.
_tagged_fieldsThe tagged fields
CreateTopics Response (Version: 6) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name error_code error_message num_partitions replication_factor [configs] _tagged_fields 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    num_partitions => INT32
    replication_factor => INT16
    configs => name value read_only config_source is_sensitive _tagged_fields 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsResults for each topic we tried to create.
nameThe topic name.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
num_partitionsNumber of partitions of the topic.
replication_factorReplication factor of the topic.
configsConfiguration of the topic.
nameThe configuration name.
valueThe configuration value.
read_onlyTrue if the configuration is read-only.
config_sourceThe configuration source.
is_sensitiveTrue if this configuration is sensitive.
_tagged_fieldsThe tagged fields
_tagged_fields
Tag Tagged field Description
0topic_config_error_codeOptional topic config error returned if configs are not returned in the response.
_tagged_fieldsThe tagged fields
DeleteTopics API (Key: 20):
Requests:
DeleteTopics Request (Version: 1) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32

Request header version: 1

Field Description
topic_namesThe names of the topics to delete.
timeout_msThe length of time in milliseconds to wait for the deletions to complete.
DeleteTopics Request (Version: 2) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32

Request header version: 1

Field Description
topic_namesThe names of the topics to delete.
timeout_msThe length of time in milliseconds to wait for the deletions to complete.
DeleteTopics Request (Version: 3) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32

Request header version: 1

Field Description
topic_namesThe names of the topics to delete.
timeout_msThe length of time in milliseconds to wait for the deletions to complete.
DeleteTopics Request (Version: 4) => [topic_names] timeout_ms _tagged_fields 
  topic_names => COMPACT_STRING
  timeout_ms => INT32

Request header version: 2

Field Description
topic_namesThe names of the topics to delete.
timeout_msThe length of time in milliseconds to wait for the deletions to complete.
_tagged_fieldsThe tagged fields
DeleteTopics Request (Version: 5) => [topic_names] timeout_ms _tagged_fields 
  topic_names => COMPACT_STRING
  timeout_ms => INT32

Request header version: 2

Field Description
topic_namesThe names of the topics to delete.
timeout_msThe length of time in milliseconds to wait for the deletions to complete.
_tagged_fieldsThe tagged fields
DeleteTopics Request (Version: 6) => [topics] timeout_ms _tagged_fields 
  topics => name topic_id _tagged_fields 
    name => COMPACT_NULLABLE_STRING
    topic_id => UUID
  timeout_ms => INT32

Request header version: 2

Field Description
topicsThe name or topic ID of the topic.
nameThe topic name.
topic_idThe unique topic ID.
_tagged_fieldsThe tagged fields
timeout_msThe length of time in milliseconds to wait for the deletions to complete.
_tagged_fieldsThe tagged fields
Responses:
DeleteTopics Response (Version: 1) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => name error_code 
    name => STRING
    error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe results for each topic we tried to delete.
nameThe topic name.
error_codeThe deletion error, or 0 if the deletion succeeded.
DeleteTopics Response (Version: 2) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => name error_code 
    name => STRING
    error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe results for each topic we tried to delete.
nameThe topic name.
error_codeThe deletion error, or 0 if the deletion succeeded.
DeleteTopics Response (Version: 3) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => name error_code 
    name => STRING
    error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe results for each topic we tried to delete.
nameThe topic name.
error_codeThe deletion error, or 0 if the deletion succeeded.
DeleteTopics Response (Version: 4) => throttle_time_ms [responses] _tagged_fields 
  throttle_time_ms => INT32
  responses => name error_code _tagged_fields 
    name => COMPACT_STRING
    error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe results for each topic we tried to delete.
nameThe topic name.
error_codeThe deletion error, or 0 if the deletion succeeded.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DeleteTopics Response (Version: 5) => throttle_time_ms [responses] _tagged_fields 
  throttle_time_ms => INT32
  responses => name error_code error_message _tagged_fields 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe results for each topic we tried to delete.
nameThe topic name.
error_codeThe deletion error, or 0 if the deletion succeeded.
error_messageThe error message, or null if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DeleteRecords API (Key: 21):
Requests:
DeleteRecords Request (Version: 0) => [topics] timeout_ms 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index offset 
      partition_index => INT32
      offset => INT64
  timeout_ms => INT32

Request header version: 1

Field Description
topicsEach topic that we want to delete records from.
nameThe topic name.
partitionsEach partition that we want to delete records from.
partition_indexThe partition index.
offsetThe deletion offset.
timeout_msHow long to wait for the deletion to complete, in milliseconds.
DeleteRecords Request (Version: 1) => [topics] timeout_ms 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index offset 
      partition_index => INT32
      offset => INT64
  timeout_ms => INT32

Request header version: 1

Field Description
topicsEach topic that we want to delete records from.
nameThe topic name.
partitionsEach partition that we want to delete records from.
partition_indexThe partition index.
offsetThe deletion offset.
timeout_msHow long to wait for the deletion to complete, in milliseconds.
DeleteRecords Request (Version: 2) => [topics] timeout_ms _tagged_fields 
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index offset _tagged_fields 
      partition_index => INT32
      offset => INT64
  timeout_ms => INT32

Request header version: 2

Field Description
topicsEach topic that we want to delete records from.
nameThe topic name.
partitionsEach partition that we want to delete records from.
partition_indexThe partition index.
offsetThe deletion offset.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msHow long to wait for the deletion to complete, in milliseconds.
_tagged_fieldsThe tagged fields
Responses:
DeleteRecords Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index low_watermark error_code 
      partition_index => INT32
      low_watermark => INT64
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic that we wanted to delete records from.
nameThe topic name.
partitionsEach partition that we wanted to delete records from.
partition_indexThe partition index.
low_watermarkThe partition low water mark.
error_codeThe deletion error code, or 0 if the deletion succeeded.
DeleteRecords Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index low_watermark error_code 
      partition_index => INT32
      low_watermark => INT64
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic that we wanted to delete records from.
nameThe topic name.
partitionsEach partition that we wanted to delete records from.
partition_indexThe partition index.
low_watermarkThe partition low water mark.
error_codeThe deletion error code, or 0 if the deletion succeeded.
InitProducerId API (Key: 22):
Requests:
InitProducerId Request (Version: 0) => transactional_id transaction_timeout_ms 
  transactional_id => NULLABLE_STRING
  transaction_timeout_ms => INT32

Request header version: 1

Field Description
transactional_idThe transactional id, or null if the producer is not transactional.
transaction_timeout_msThe time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.
InitProducerId Request (Version: 1) => transactional_id transaction_timeout_ms 
  transactional_id => NULLABLE_STRING
  transaction_timeout_ms => INT32

Request header version: 1

Field Description
transactional_idThe transactional id, or null if the producer is not transactional.
transaction_timeout_msThe time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.
InitProducerId Request (Version: 2) => transactional_id transaction_timeout_ms _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32

Request header version: 2

Field Description
transactional_idThe transactional id, or null if the producer is not transactional.
transaction_timeout_msThe time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.
_tagged_fieldsThe tagged fields
InitProducerId Request (Version: 3) => transactional_id transaction_timeout_ms producer_id producer_epoch _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32
  producer_id => INT64
  producer_epoch => INT16

Request header version: 2

Field Description
transactional_idThe transactional id, or null if the producer is not transactional.
transaction_timeout_msThe time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.
producer_idThe producer id. This is used to disambiguate requests if a transactional id is reused following its expiration.
producer_epochThe producer's current epoch. This will be checked against the producer epoch on the broker, and the request will return an error if they do not match.
_tagged_fieldsThe tagged fields
InitProducerId Request (Version: 4) => transactional_id transaction_timeout_ms producer_id producer_epoch _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32
  producer_id => INT64
  producer_epoch => INT16

Request header version: 2

Field Description
transactional_idThe transactional id, or null if the producer is not transactional.
transaction_timeout_msThe time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.
producer_idThe producer id. This is used to disambiguate requests if a transactional id is reused following its expiration.
producer_epochThe producer's current epoch. This will be checked against the producer epoch on the broker, and the request will return an error if they do not match.
_tagged_fieldsThe tagged fields
InitProducerId Request (Version: 5) => transactional_id transaction_timeout_ms producer_id producer_epoch _tagged_fields 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32
  producer_id => INT64
  producer_epoch => INT16

Request header version: 2

Field Description
transactional_idThe transactional id, or null if the producer is not transactional.
transaction_timeout_msThe time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.
producer_idThe producer id. This is used to disambiguate requests if a transactional id is reused following its expiration.
producer_epochThe producer's current epoch. This will be checked against the producer epoch on the broker, and the request will return an error if they do not match.
_tagged_fieldsThe tagged fields
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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
producer_idThe current producer id.
producer_epochThe current 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

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
producer_idThe current producer id.
producer_epochThe current epoch associated with the producer id.
InitProducerId Response (Version: 2) => throttle_time_ms error_code producer_id producer_epoch _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
producer_idThe current producer id.
producer_epochThe current epoch associated with the producer id.
_tagged_fieldsThe tagged fields
InitProducerId Response (Version: 3) => throttle_time_ms error_code producer_id producer_epoch _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
producer_idThe current producer id.
producer_epochThe current epoch associated with the producer id.
_tagged_fieldsThe tagged fields
InitProducerId Response (Version: 4) => throttle_time_ms error_code producer_id producer_epoch _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
producer_idThe current producer id.
producer_epochThe current epoch associated with the producer id.
_tagged_fieldsThe tagged fields
OffsetForLeaderEpoch API (Key: 23):
Requests:
OffsetForLeaderEpoch Request (Version: 2) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch leader_epoch 
      partition => INT32
      current_leader_epoch => INT32
      leader_epoch => INT32

Request header version: 1

Field Description
topicsEach topic to get offsets for.
topicThe topic name.
partitionsEach partition to get offsets for.
partitionThe partition index.
current_leader_epochAn epoch used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
leader_epochThe epoch to look up an offset for.
OffsetForLeaderEpoch Request (Version: 3) => replica_id [topics] 
  replica_id => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch leader_epoch 
      partition => INT32
      current_leader_epoch => INT32
      leader_epoch => INT32

Request header version: 1

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
topicsEach topic to get offsets for.
topicThe topic name.
partitionsEach partition to get offsets for.
partitionThe partition index.
current_leader_epochAn epoch used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
leader_epochThe epoch to look up an offset for.
OffsetForLeaderEpoch Request (Version: 4) => replica_id [topics] _tagged_fields 
  replica_id => INT32
  topics => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => partition current_leader_epoch leader_epoch _tagged_fields 
      partition => INT32
      current_leader_epoch => INT32
      leader_epoch => INT32

Request header version: 2

Field Description
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
topicsEach topic to get offsets for.
topicThe topic name.
partitionsEach partition to get offsets for.
partitionThe partition index.
current_leader_epochAn epoch used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.
leader_epochThe epoch to look up an offset for.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
OffsetForLeaderEpoch Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic we fetched offsets for.
topicThe topic name.
partitionsEach partition in the topic we fetched offsets for.
error_codeThe error code 0, or if there was no error.
partitionThe partition index.
leader_epochThe leader epoch of the partition.
end_offsetThe end offset of the epoch.
OffsetForLeaderEpoch Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsEach topic we fetched offsets for.
topicThe topic name.
partitionsEach partition in the topic we fetched offsets for.
error_codeThe error code 0, or if there was no error.
partitionThe partition index.
leader_epochThe leader epoch of the partition.
end_offsetThe end offset of the epoch.
AddPartitionsToTxn API (Key: 24):
Requests:
AddPartitionsToTxn Request (Version: 0) => v3_and_below_transactional_id v3_and_below_producer_id v3_and_below_producer_epoch [v3_and_below_topics] 
  v3_and_below_transactional_id => STRING
  v3_and_below_producer_id => INT64
  v3_and_below_producer_epoch => INT16
  v3_and_below_topics => name [partitions] 
    name => STRING
    partitions => INT32

Request header version: 1

Field Description
v3_and_below_transactional_idThe transactional id corresponding to the transaction.
v3_and_below_producer_idCurrent producer id in use by the transactional id.
v3_and_below_producer_epochCurrent epoch associated with the producer id.
v3_and_below_topicsThe partitions to add to the transaction.
nameThe name of the topic.
partitionsThe partition indexes to add to the transaction.
AddPartitionsToTxn Request (Version: 1) => v3_and_below_transactional_id v3_and_below_producer_id v3_and_below_producer_epoch [v3_and_below_topics] 
  v3_and_below_transactional_id => STRING
  v3_and_below_producer_id => INT64
  v3_and_below_producer_epoch => INT16
  v3_and_below_topics => name [partitions] 
    name => STRING
    partitions => INT32

Request header version: 1

Field Description
v3_and_below_transactional_idThe transactional id corresponding to the transaction.
v3_and_below_producer_idCurrent producer id in use by the transactional id.
v3_and_below_producer_epochCurrent epoch associated with the producer id.
v3_and_below_topicsThe partitions to add to the transaction.
nameThe name of the topic.
partitionsThe partition indexes to add to the transaction.
AddPartitionsToTxn Request (Version: 2) => v3_and_below_transactional_id v3_and_below_producer_id v3_and_below_producer_epoch [v3_and_below_topics] 
  v3_and_below_transactional_id => STRING
  v3_and_below_producer_id => INT64
  v3_and_below_producer_epoch => INT16
  v3_and_below_topics => name [partitions] 
    name => STRING
    partitions => INT32

Request header version: 1

Field Description
v3_and_below_transactional_idThe transactional id corresponding to the transaction.
v3_and_below_producer_idCurrent producer id in use by the transactional id.
v3_and_below_producer_epochCurrent epoch associated with the producer id.
v3_and_below_topicsThe partitions to add to the transaction.
nameThe name of the topic.
partitionsThe partition indexes to add to the transaction.
AddPartitionsToTxn Request (Version: 3) => v3_and_below_transactional_id v3_and_below_producer_id v3_and_below_producer_epoch [v3_and_below_topics] _tagged_fields 
  v3_and_below_transactional_id => COMPACT_STRING
  v3_and_below_producer_id => INT64
  v3_and_below_producer_epoch => INT16
  v3_and_below_topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => INT32

Request header version: 2

Field Description
v3_and_below_transactional_idThe transactional id corresponding to the transaction.
v3_and_below_producer_idCurrent producer id in use by the transactional id.
v3_and_below_producer_epochCurrent epoch associated with the producer id.
v3_and_below_topicsThe partitions to add to the transaction.
nameThe name of the topic.
partitionsThe partition indexes to add to the transaction.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
AddPartitionsToTxn Request (Version: 4) => [transactions] _tagged_fields 
  transactions => transactional_id producer_id producer_epoch verify_only [topics] _tagged_fields 
    transactional_id => COMPACT_STRING
    producer_id => INT64
    producer_epoch => INT16
    verify_only => BOOLEAN
    topics => name [partitions] _tagged_fields 
      name => COMPACT_STRING
      partitions => INT32

Request header version: 2

Field Description
transactionsList of transactions to add partitions to.
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.
verify_onlyBoolean to signify if we want to check if the partition is in the transaction rather than add it.
topicsThe partitions to add to the transaction.
nameThe name of the topic.
partitionsThe partition indexes to add to the transaction.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
AddPartitionsToTxn Request (Version: 5) => [transactions] _tagged_fields 
  transactions => transactional_id producer_id producer_epoch verify_only [topics] _tagged_fields 
    transactional_id => COMPACT_STRING
    producer_id => INT64
    producer_epoch => INT16
    verify_only => BOOLEAN
    topics => name [partitions] _tagged_fields 
      name => COMPACT_STRING
      partitions => INT32

Request header version: 2

Field Description
transactionsList of transactions to add partitions to.
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.
verify_onlyBoolean to signify if we want to check if the partition is in the transaction rather than add it.
topicsThe partitions to add to the transaction.
nameThe name of the topic.
partitionsThe partition indexes to add to the transaction.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
AddPartitionsToTxn Response (Version: 0) => throttle_time_ms [results_by_topic_v3_and_below] 
  throttle_time_ms => INT32
  results_by_topic_v3_and_below => name [results_by_partition] 
    name => STRING
    results_by_partition => partition_index partition_error_code 
      partition_index => INT32
      partition_error_code => INT16

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
results_by_topic_v3_and_belowThe results for each topic.
nameThe topic name.
results_by_partitionThe results for each partition.
partition_indexThe partition indexes.
partition_error_codeThe response error code.
AddPartitionsToTxn Response (Version: 1) => throttle_time_ms [results_by_topic_v3_and_below] 
  throttle_time_ms => INT32
  results_by_topic_v3_and_below => name [results_by_partition] 
    name => STRING
    results_by_partition => partition_index partition_error_code 
      partition_index => INT32
      partition_error_code => INT16

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
results_by_topic_v3_and_belowThe results for each topic.
nameThe topic name.
results_by_partitionThe results for each partition.
partition_indexThe partition indexes.
partition_error_codeThe response error code.
AddPartitionsToTxn Response (Version: 2) => throttle_time_ms [results_by_topic_v3_and_below] 
  throttle_time_ms => INT32
  results_by_topic_v3_and_below => name [results_by_partition] 
    name => STRING
    results_by_partition => partition_index partition_error_code 
      partition_index => INT32
      partition_error_code => INT16

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
results_by_topic_v3_and_belowThe results for each topic.
nameThe topic name.
results_by_partitionThe results for each partition.
partition_indexThe partition indexes.
partition_error_codeThe response error code.
AddPartitionsToTxn Response (Version: 3) => throttle_time_ms [results_by_topic_v3_and_below] _tagged_fields 
  throttle_time_ms => INT32
  results_by_topic_v3_and_below => name [results_by_partition] _tagged_fields 
    name => COMPACT_STRING
    results_by_partition => partition_index partition_error_code _tagged_fields 
      partition_index => INT32
      partition_error_code => INT16

Response header version: 1

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
results_by_topic_v3_and_belowThe results for each topic.
nameThe topic name.
results_by_partitionThe results for each partition.
partition_indexThe partition indexes.
partition_error_codeThe response error code.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
AddPartitionsToTxn Response (Version: 4) => throttle_time_ms error_code [results_by_transaction] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  results_by_transaction => transactional_id [topic_results] _tagged_fields 
    transactional_id => COMPACT_STRING
    topic_results => name [results_by_partition] _tagged_fields 
      name => COMPACT_STRING
      results_by_partition => partition_index partition_error_code _tagged_fields 
        partition_index => INT32
        partition_error_code => INT16

Response header version: 1

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe response top level error code.
results_by_transactionResults categorized by transactional ID.
transactional_idThe transactional id corresponding to the transaction.
topic_resultsThe results for each topic.
nameThe topic name.
results_by_partitionThe results for each partition.
partition_indexThe partition indexes.
partition_error_codeThe response error code.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
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

Request header version: 1

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

Request header version: 1

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: 2) => transactional_id producer_id producer_epoch group_id 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => STRING

Request header version: 1

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: 3) => transactional_id producer_id producer_epoch group_id _tagged_fields 
  transactional_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => COMPACT_STRING

Request header version: 2

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.
_tagged_fieldsThe tagged fields
AddOffsetsToTxn Request (Version: 4) => transactional_id producer_id producer_epoch group_id _tagged_fields 
  transactional_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => COMPACT_STRING

Request header version: 2

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.
_tagged_fieldsThe tagged fields
Responses:
AddOffsetsToTxn Response (Version: 0) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16

Response header version: 0

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

Response header version: 0

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

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe response error code, or 0 if there was no error.
AddOffsetsToTxn Response (Version: 3) => throttle_time_ms error_code _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16

Response header version: 1

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe response error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
EndTxn API (Key: 26):
Requests:
EndTxn Request (Version: 0) => transactional_id producer_id producer_epoch committed 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN

Request header version: 1

Field Description
transactional_idThe ID of the transaction to end.
producer_idThe producer ID.
producer_epochThe current epoch associated with the producer.
committedTrue if the transaction was committed, false if it was aborted.
EndTxn Request (Version: 1) => transactional_id producer_id producer_epoch committed 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN

Request header version: 1

Field Description
transactional_idThe ID of the transaction to end.
producer_idThe producer ID.
producer_epochThe current epoch associated with the producer.
committedTrue if the transaction was committed, false if it was aborted.
EndTxn Request (Version: 2) => transactional_id producer_id producer_epoch committed 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN

Request header version: 1

Field Description
transactional_idThe ID of the transaction to end.
producer_idThe producer ID.
producer_epochThe current epoch associated with the producer.
committedTrue if the transaction was committed, false if it was aborted.
EndTxn Request (Version: 3) => transactional_id producer_id producer_epoch committed _tagged_fields 
  transactional_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN

Request header version: 2

Field Description
transactional_idThe ID of the transaction to end.
producer_idThe producer ID.
producer_epochThe current epoch associated with the producer.
committedTrue if the transaction was committed, false if it was aborted.
_tagged_fieldsThe tagged fields
EndTxn Request (Version: 4) => transactional_id producer_id producer_epoch committed _tagged_fields 
  transactional_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN

Request header version: 2

Field Description
transactional_idThe ID of the transaction to end.
producer_idThe producer ID.
producer_epochThe current epoch associated with the producer.
committedTrue if the transaction was committed, false if it was aborted.
_tagged_fieldsThe tagged fields
EndTxn Request (Version: 5) => transactional_id producer_id producer_epoch committed _tagged_fields 
  transactional_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN

Request header version: 2

Field Description
transactional_idThe ID of the transaction to end.
producer_idThe producer ID.
producer_epochThe current epoch associated with the producer.
committedTrue if the transaction was committed, false if it was aborted.
_tagged_fieldsThe tagged fields
Responses:
EndTxn Response (Version: 0) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16

Response header version: 0

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

Response header version: 0

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

Response header version: 0

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

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
EndTxn Response (Version: 4) => throttle_time_ms error_code _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
WriteTxnMarkers API (Key: 27):
Requests:
WriteTxnMarkers Request (Version: 1) => [markers] _tagged_fields 
  markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch _tagged_fields 
    producer_id => INT64
    producer_epoch => INT16
    transaction_result => BOOLEAN
    topics => name [partition_indexes] _tagged_fields 
      name => COMPACT_STRING
      partition_indexes => INT32
    coordinator_epoch => INT32

Request header version: 2

Field Description
markersThe transaction markers to be written.
producer_idThe current producer ID.
producer_epochThe current epoch associated with the producer ID.
transaction_resultThe result of the transaction to write to the partitions (false = ABORT, true = COMMIT).
topicsEach topic that we want to write transaction marker(s) for.
nameThe topic name.
partition_indexesThe indexes of the partitions to write transaction markers for.
_tagged_fieldsThe tagged fields
coordinator_epochEpoch associated with the transaction state partition hosted by this transaction coordinator.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
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 => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
transactional_idThe ID of the transaction.
group_idThe ID of the group.
producer_idThe current producer ID in use by the transactional ID.
producer_epochThe current epoch associated with the producer ID.
topicsEach topic that we want to commit offsets for.
nameThe topic name.
partitionsThe partitions inside the topic that we want to commit offsets for.
partition_indexThe index of the partition within the topic.
committed_offsetThe message offset to be committed.
committed_metadataAny associated metadata the client wants to keep.
TxnOffsetCommit Request (Version: 1) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
transactional_idThe ID of the transaction.
group_idThe ID of the group.
producer_idThe current producer ID in use by the transactional ID.
producer_epochThe current epoch associated with the producer ID.
topicsEach topic that we want to commit offsets for.
nameThe topic name.
partitionsThe partitions inside the topic that we want to commit offsets for.
partition_indexThe index of the partition within the topic.
committed_offsetThe message offset to be committed.
committed_metadataAny associated metadata the client wants to keep.
TxnOffsetCommit Request (Version: 2) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => NULLABLE_STRING

Request header version: 1

Field Description
transactional_idThe ID of the transaction.
group_idThe ID of the group.
producer_idThe current producer ID in use by the transactional ID.
producer_epochThe current epoch associated with the producer ID.
topicsEach topic that we want to commit offsets for.
nameThe topic name.
partitionsThe partitions inside the topic that we want to commit offsets for.
partition_indexThe index of the partition within the topic.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of the last consumed record.
committed_metadataAny associated metadata the client wants to keep.
TxnOffsetCommit Request (Version: 3) => transactional_id group_id producer_id producer_epoch generation_id member_id group_instance_id [topics] _tagged_fields 
  transactional_id => COMPACT_STRING
  group_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
transactional_idThe ID of the transaction.
group_idThe ID of the group.
producer_idThe current producer ID in use by the transactional ID.
producer_epochThe current epoch associated with the producer ID.
generation_idThe generation of the consumer.
member_idThe member ID assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
topicsEach topic that we want to commit offsets for.
nameThe topic name.
partitionsThe partitions inside the topic that we want to commit offsets for.
partition_indexThe index of the partition within the topic.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of the last consumed record.
committed_metadataAny associated metadata the client wants to keep.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
TxnOffsetCommit Request (Version: 4) => transactional_id group_id producer_id producer_epoch generation_id member_id group_instance_id [topics] _tagged_fields 
  transactional_id => COMPACT_STRING
  group_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
transactional_idThe ID of the transaction.
group_idThe ID of the group.
producer_idThe current producer ID in use by the transactional ID.
producer_epochThe current epoch associated with the producer ID.
generation_idThe generation of the consumer.
member_idThe member ID assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
topicsEach topic that we want to commit offsets for.
nameThe topic name.
partitionsThe partitions inside the topic that we want to commit offsets for.
partition_indexThe index of the partition within the topic.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of the last consumed record.
committed_metadataAny associated metadata the client wants to keep.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
TxnOffsetCommit Request (Version: 5) => transactional_id group_id producer_id producer_epoch generation_id member_id group_instance_id [topics] _tagged_fields 
  transactional_id => COMPACT_STRING
  group_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata _tagged_fields 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING

Request header version: 2

Field Description
transactional_idThe ID of the transaction.
group_idThe ID of the group.
producer_idThe current producer ID in use by the transactional ID.
producer_epochThe current epoch associated with the producer ID.
generation_idThe generation of the consumer.
member_idThe member ID assigned by the group coordinator.
group_instance_idThe unique identifier of the consumer instance provided by end user.
topicsEach topic that we want to commit offsets for.
nameThe topic name.
partitionsThe partitions inside the topic that we want to commit offsets for.
partition_indexThe index of the partition within the topic.
committed_offsetThe message offset to be committed.
committed_leader_epochThe leader epoch of the last consumed record.
committed_metadataAny associated metadata the client wants to keep.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
TxnOffsetCommit Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
TxnOffsetCommit Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
TxnOffsetCommit Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
TxnOffsetCommit Response (Version: 3) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code _tagged_fields 
      partition_index => INT32
      error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
TxnOffsetCommit Response (Version: 4) => throttle_time_ms [topics] _tagged_fields 
  throttle_time_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index error_code _tagged_fields 
      partition_index => INT32
      error_code => INT16

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
topicsThe responses for each topic.
nameThe topic name.
partitionsThe responses for each partition in the topic.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeAcls API (Key: 29):
Requests:
DescribeAcls Request (Version: 1) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type 
  resource_type_filter => INT8
  resource_name_filter => NULLABLE_STRING
  pattern_type_filter => INT8
  principal_filter => NULLABLE_STRING
  host_filter => NULLABLE_STRING
  operation => INT8
  permission_type => INT8

Request header version: 1

Field Description
resource_type_filterThe resource type.
resource_name_filterThe resource name, or null to match any resource name.
pattern_type_filterThe resource pattern to match.
principal_filterThe principal to match, or null to match any principal.
host_filterThe host to match, or null to match any host.
operationThe operation to match.
permission_typeThe permission type to match.
DescribeAcls Request (Version: 2) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type _tagged_fields 
  resource_type_filter => INT8
  resource_name_filter => COMPACT_NULLABLE_STRING
  pattern_type_filter => INT8
  principal_filter => COMPACT_NULLABLE_STRING
  host_filter => COMPACT_NULLABLE_STRING
  operation => INT8
  permission_type => INT8

Request header version: 2

Field Description
resource_type_filterThe resource type.
resource_name_filterThe resource name, or null to match any resource name.
pattern_type_filterThe resource pattern to match.
principal_filterThe principal to match, or null to match any principal.
host_filterThe host to match, or null to match any host.
operationThe operation to match.
permission_typeThe permission type to match.
_tagged_fieldsThe tagged fields
DescribeAcls Request (Version: 3) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type _tagged_fields 
  resource_type_filter => INT8
  resource_name_filter => COMPACT_NULLABLE_STRING
  pattern_type_filter => INT8
  principal_filter => COMPACT_NULLABLE_STRING
  host_filter => COMPACT_NULLABLE_STRING
  operation => INT8
  permission_type => INT8

Request header version: 2

Field Description
resource_type_filterThe resource type.
resource_name_filterThe resource name, or null to match any resource name.
pattern_type_filterThe resource pattern to match.
principal_filterThe principal to match, or null to match any principal.
host_filterThe host to match, or null to match any host.
operationThe operation to match.
permission_typeThe permission type to match.
_tagged_fieldsThe tagged fields
Responses:
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 pattern_type [acls] 
    resource_type => INT8
    resource_name => STRING
    pattern_type => INT8
    acls => principal host operation permission_type 
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
resourcesEach Resource that is referenced in an ACL.
resource_typeThe resource type.
resource_nameThe resource name.
pattern_typeThe resource pattern type.
aclsThe ACLs.
principalThe ACL principal.
hostThe ACL host.
operationThe ACL operation.
permission_typeThe ACL permission type.
DescribeAcls Response (Version: 2) => throttle_time_ms error_code error_message [resources] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  resources => resource_type resource_name pattern_type [acls] _tagged_fields 
    resource_type => INT8
    resource_name => COMPACT_STRING
    pattern_type => INT8
    acls => principal host operation permission_type _tagged_fields 
      principal => COMPACT_STRING
      host => COMPACT_STRING
      operation => INT8
      permission_type => INT8

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
resourcesEach Resource that is referenced in an ACL.
resource_typeThe resource type.
resource_nameThe resource name.
pattern_typeThe resource pattern type.
aclsThe ACLs.
principalThe ACL principal.
hostThe ACL host.
operationThe ACL operation.
permission_typeThe ACL permission type.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
CreateAcls API (Key: 30):
Requests:
CreateAcls Request (Version: 1) => [creations] 
  creations => resource_type resource_name resource_pattern_type principal host operation permission_type 
    resource_type => INT8
    resource_name => STRING
    resource_pattern_type => INT8
    principal => STRING
    host => STRING
    operation => INT8
    permission_type => INT8

Request header version: 1

Field Description
creationsThe ACLs that we want to create.
resource_typeThe type of the resource.
resource_nameThe resource name for the ACL.
resource_pattern_typeThe pattern type for the ACL.
principalThe principal for the ACL.
hostThe host for the ACL.
operationThe operation type for the ACL (read, write, etc.).
permission_typeThe permission type for the ACL (allow, deny, etc.).
CreateAcls Request (Version: 2) => [creations] _tagged_fields 
  creations => resource_type resource_name resource_pattern_type principal host operation permission_type _tagged_fields 
    resource_type => INT8
    resource_name => COMPACT_STRING
    resource_pattern_type => INT8
    principal => COMPACT_STRING
    host => COMPACT_STRING
    operation => INT8
    permission_type => INT8

Request header version: 2

Field Description
creationsThe ACLs that we want to create.
resource_typeThe type of the resource.
resource_nameThe resource name for the ACL.
resource_pattern_typeThe pattern type for the ACL.
principalThe principal for the ACL.
hostThe host for the ACL.
operationThe operation type for the ACL (read, write, etc.).
permission_typeThe permission type for the ACL (allow, deny, etc.).
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
CreateAcls Request (Version: 3) => [creations] _tagged_fields 
  creations => resource_type resource_name resource_pattern_type principal host operation permission_type _tagged_fields 
    resource_type => INT8
    resource_name => COMPACT_STRING
    resource_pattern_type => INT8
    principal => COMPACT_STRING
    host => COMPACT_STRING
    operation => INT8
    permission_type => INT8

Request header version: 2

Field Description
creationsThe ACLs that we want to create.
resource_typeThe type of the resource.
resource_nameThe resource name for the ACL.
resource_pattern_typeThe pattern type for the ACL.
principalThe principal for the ACL.
hostThe host for the ACL.
operationThe operation type for the ACL (read, write, etc.).
permission_typeThe permission type for the ACL (allow, deny, etc.).
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
CreateAcls Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message 
    error_code => INT16
    error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe results for each ACL creation.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
CreateAcls Response (Version: 2) => throttle_time_ms [results] _tagged_fields 
  throttle_time_ms => INT32
  results => error_code error_message _tagged_fields 
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe results for each ACL creation.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DeleteAcls API (Key: 31):
Requests:
DeleteAcls Request (Version: 1) => [filters] 
  filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type 
    resource_type_filter => INT8
    resource_name_filter => NULLABLE_STRING
    pattern_type_filter => INT8
    principal_filter => NULLABLE_STRING
    host_filter => NULLABLE_STRING
    operation => INT8
    permission_type => INT8

Request header version: 1

Field Description
filtersThe filters to use when deleting ACLs.
resource_type_filterThe resource type.
resource_name_filterThe resource name.
pattern_type_filterThe pattern type.
principal_filterThe principal filter, or null to accept all principals.
host_filterThe host filter, or null to accept all hosts.
operationThe ACL operation.
permission_typeThe permission type.
DeleteAcls Request (Version: 2) => [filters] _tagged_fields 
  filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type _tagged_fields 
    resource_type_filter => INT8
    resource_name_filter => COMPACT_NULLABLE_STRING
    pattern_type_filter => INT8
    principal_filter => COMPACT_NULLABLE_STRING
    host_filter => COMPACT_NULLABLE_STRING
    operation => INT8
    permission_type => INT8

Request header version: 2

Field Description
filtersThe filters to use when deleting ACLs.
resource_type_filterThe resource type.
resource_name_filterThe resource name.
pattern_type_filterThe pattern type.
principal_filterThe principal filter, or null to accept all principals.
host_filterThe host filter, or null to accept all hosts.
operationThe ACL operation.
permission_typeThe permission type.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DeleteAcls Request (Version: 3) => [filters] _tagged_fields 
  filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type _tagged_fields 
    resource_type_filter => INT8
    resource_name_filter => COMPACT_NULLABLE_STRING
    pattern_type_filter => INT8
    principal_filter => COMPACT_NULLABLE_STRING
    host_filter => COMPACT_NULLABLE_STRING
    operation => INT8
    permission_type => INT8

Request header version: 2

Field Description
filtersThe filters to use when deleting ACLs.
resource_type_filterThe resource type.
resource_name_filterThe resource name.
pattern_type_filterThe pattern type.
principal_filterThe principal filter, or null to accept all principals.
host_filterThe host filter, or null to accept all hosts.
operationThe ACL operation.
permission_typeThe permission type.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
DeleteAcls Response (Version: 1) => throttle_time_ms [filter_results] 
  throttle_time_ms => INT32
  filter_results => error_code error_message [matching_acls] 
    error_code => INT16
    error_message => NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name pattern_type principal host operation permission_type 
      error_code => INT16
      error_message => NULLABLE_STRING
      resource_type => INT8
      resource_name => STRING
      pattern_type => INT8
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
filter_resultsThe results for each filter.
error_codeThe error code, or 0 if the filter succeeded.
error_messageThe error message, or null if the filter succeeded.
matching_aclsThe ACLs which matched this filter.
error_codeThe deletion error code, or 0 if the deletion succeeded.
error_messageThe deletion error message, or null if the deletion succeeded.
resource_typeThe ACL resource type.
resource_nameThe ACL resource name.
pattern_typeThe ACL resource pattern type.
principalThe ACL principal.
hostThe ACL host.
operationThe ACL operation.
permission_typeThe ACL permission type.
DeleteAcls Response (Version: 2) => throttle_time_ms [filter_results] _tagged_fields 
  throttle_time_ms => INT32
  filter_results => error_code error_message [matching_acls] _tagged_fields 
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name pattern_type principal host operation permission_type _tagged_fields 
      error_code => INT16
      error_message => COMPACT_NULLABLE_STRING
      resource_type => INT8
      resource_name => COMPACT_STRING
      pattern_type => INT8
      principal => COMPACT_STRING
      host => COMPACT_STRING
      operation => INT8
      permission_type => INT8

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
filter_resultsThe results for each filter.
error_codeThe error code, or 0 if the filter succeeded.
error_messageThe error message, or null if the filter succeeded.
matching_aclsThe ACLs which matched this filter.
error_codeThe deletion error code, or 0 if the deletion succeeded.
error_messageThe deletion error message, or null if the deletion succeeded.
resource_typeThe ACL resource type.
resource_nameThe ACL resource name.
pattern_typeThe ACL resource pattern type.
principalThe ACL principal.
hostThe ACL host.
operationThe ACL operation.
permission_typeThe ACL permission type.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeConfigs API (Key: 32):
Requests:
DescribeConfigs Request (Version: 1) => [resources] include_synonyms 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
  include_synonyms => BOOLEAN

Request header version: 1

Field Description
resourcesThe resources whose configurations we want to describe.
resource_typeThe resource type.
resource_nameThe resource name.
configuration_keysThe configuration keys to list, or null to list all configuration keys.
include_synonymsTrue if we should include all synonyms.
DescribeConfigs Request (Version: 2) => [resources] include_synonyms 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
  include_synonyms => BOOLEAN

Request header version: 1

Field Description
resourcesThe resources whose configurations we want to describe.
resource_typeThe resource type.
resource_nameThe resource name.
configuration_keysThe configuration keys to list, or null to list all configuration keys.
include_synonymsTrue if we should include all synonyms.
DescribeConfigs Request (Version: 3) => [resources] include_synonyms include_documentation 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
  include_synonyms => BOOLEAN
  include_documentation => BOOLEAN

Request header version: 1

Field Description
resourcesThe resources whose configurations we want to describe.
resource_typeThe resource type.
resource_nameThe resource name.
configuration_keysThe configuration keys to list, or null to list all configuration keys.
include_synonymsTrue if we should include all synonyms.
include_documentationTrue if we should include configuration documentation.
DescribeConfigs Request (Version: 4) => [resources] include_synonyms include_documentation _tagged_fields 
  resources => resource_type resource_name [configuration_keys] _tagged_fields 
    resource_type => INT8
    resource_name => COMPACT_STRING
    configuration_keys => COMPACT_STRING
  include_synonyms => BOOLEAN
  include_documentation => BOOLEAN

Request header version: 2

Field Description
resourcesThe resources whose configurations we want to describe.
resource_typeThe resource type.
resource_nameThe resource name.
configuration_keysThe configuration keys to list, or null to list all configuration keys.
_tagged_fieldsThe tagged fields
include_synonymsTrue if we should include all synonyms.
include_documentationTrue if we should include configuration documentation.
_tagged_fieldsThe tagged fields
Responses:
DescribeConfigs Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only config_source is_sensitive [synonyms] 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      synonyms => name value source 
        name => STRING
        value => NULLABLE_STRING
        source => INT8

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeThe resource type.
resource_nameThe resource name.
configsEach listed configuration.
nameThe configuration name.
valueThe configuration value.
read_onlyTrue if the configuration is read-only.
config_sourceThe configuration source.
is_sensitiveTrue if this configuration is sensitive.
synonymsThe synonyms for this configuration key.
nameThe synonym name.
valueThe synonym value.
sourceThe synonym source.
DescribeConfigs Response (Version: 2) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only config_source is_sensitive [synonyms] 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      synonyms => name value source 
        name => STRING
        value => NULLABLE_STRING
        source => INT8

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeThe resource type.
resource_nameThe resource name.
configsEach listed configuration.
nameThe configuration name.
valueThe configuration value.
read_onlyTrue if the configuration is read-only.
config_sourceThe configuration source.
is_sensitiveTrue if this configuration is sensitive.
synonymsThe synonyms for this configuration key.
nameThe synonym name.
valueThe synonym value.
sourceThe synonym source.
DescribeConfigs Response (Version: 3) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only config_source is_sensitive [synonyms] config_type documentation 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      synonyms => name value source 
        name => STRING
        value => NULLABLE_STRING
        source => INT8
      config_type => INT8
      documentation => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeThe resource type.
resource_nameThe resource name.
configsEach listed configuration.
nameThe configuration name.
valueThe configuration value.
read_onlyTrue if the configuration is read-only.
config_sourceThe configuration source.
is_sensitiveTrue if this configuration is sensitive.
synonymsThe synonyms for this configuration key.
nameThe synonym name.
valueThe synonym value.
sourceThe synonym source.
config_typeThe configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD.
documentationThe configuration documentation.
AlterConfigs API (Key: 33):
Requests:
AlterConfigs Request (Version: 0) => [resources] validate_only 
  resources => resource_type resource_name [configs] 
    resource_type => INT8
    resource_name => STRING
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  validate_only => BOOLEAN

Request header version: 1

Field Description
resourcesThe updates for each resource.
resource_typeThe resource type.
resource_nameThe resource name.
configsThe configurations.
nameThe configuration key name.
valueThe value to set for the configuration key.
validate_onlyTrue if we should validate the request, but not change the configurations.
AlterConfigs Request (Version: 1) => [resources] validate_only 
  resources => resource_type resource_name [configs] 
    resource_type => INT8
    resource_name => STRING
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  validate_only => BOOLEAN

Request header version: 1

Field Description
resourcesThe updates for each resource.
resource_typeThe resource type.
resource_nameThe resource name.
configsThe configurations.
nameThe configuration key name.
valueThe value to set for the configuration key.
validate_onlyTrue if we should validate the request, but not change the configurations.
AlterConfigs Request (Version: 2) => [resources] validate_only _tagged_fields 
  resources => resource_type resource_name [configs] _tagged_fields 
    resource_type => INT8
    resource_name => COMPACT_STRING
    configs => name value _tagged_fields 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
  validate_only => BOOLEAN

Request header version: 2

Field Description
resourcesThe updates for each resource.
resource_typeThe resource type.
resource_nameThe resource name.
configsThe configurations.
nameThe configuration key name.
valueThe value to set for the configuration key.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
validate_onlyTrue if we should validate the request, but not change the configurations.
_tagged_fieldsThe tagged fields
Responses:
AlterConfigs Response (Version: 0) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe responses for each resource.
error_codeThe resource error code.
error_messageThe resource error message, or null if there was no error.
resource_typeThe resource type.
resource_nameThe resource name.
AlterConfigs Response (Version: 1) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe responses for each resource.
error_codeThe resource error code.
error_messageThe resource error message, or null if there was no error.
resource_typeThe resource type.
resource_nameThe resource name.
AlterReplicaLogDirs API (Key: 34):
Requests:
AlterReplicaLogDirs Request (Version: 1) => [dirs] 
  dirs => path [topics] 
    path => STRING
    topics => name [partitions] 
      name => STRING
      partitions => INT32

Request header version: 1

Field Description
dirsThe alterations to make for each directory.
pathThe absolute directory path.
topicsThe topics to add to the directory.
nameThe topic name.
partitionsThe partition indexes.
AlterReplicaLogDirs Request (Version: 2) => [dirs] _tagged_fields 
  dirs => path [topics] _tagged_fields 
    path => COMPACT_STRING
    topics => name [partitions] _tagged_fields 
      name => COMPACT_STRING
      partitions => INT32

Request header version: 2

Field Description
dirsThe alterations to make for each directory.
pathThe absolute directory path.
topicsThe topics to add to the directory.
nameThe topic name.
partitionsThe partition indexes.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
AlterReplicaLogDirs Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => topic_name [partitions] 
    topic_name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe results for each topic.
topic_nameThe name of the topic.
partitionsThe results for each partition.
partition_indexThe partition index.
error_codeThe error code, or 0 if there was no error.
DescribeLogDirs API (Key: 35):
Requests:
DescribeLogDirs Request (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => INT32

Request header version: 1

Field Description
topicsEach topic that we want to describe log directories for, or null for all topics.
topicThe topic name.
partitionsThe partition indexes.
DescribeLogDirs Request (Version: 2) => [topics] _tagged_fields 
  topics => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => INT32

Request header version: 2

Field Description
topicsEach topic that we want to describe log directories for, or null for all topics.
topicThe topic name.
partitionsThe partition indexes.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeLogDirs Request (Version: 3) => [topics] _tagged_fields 
  topics => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => INT32

Request header version: 2

Field Description
topicsEach topic that we want to describe log directories for, or null for all topics.
topicThe topic name.
partitionsThe partition indexes.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeLogDirs Request (Version: 4) => [topics] _tagged_fields 
  topics => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => INT32

Request header version: 2

Field Description
topicsEach topic that we want to describe log directories for, or null for all topics.
topicThe topic name.
partitionsThe partition indexes.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
DescribeLogDirs Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code log_dir [topics] 
    error_code => INT16
    log_dir => STRING
    topics => name [partitions] 
      name => STRING
      partitions => partition_index partition_size offset_lag is_future_key 
        partition_index => INT32
        partition_size => INT64
        offset_lag => INT64
        is_future_key => BOOLEAN

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe log directories.
error_codeThe error code, or 0 if there was no error.
log_dirThe absolute log directory path.
topicsThe topics.
nameThe topic name.
partitionsThe partitions.
partition_indexThe partition index.
partition_sizeThe size of the log segments in this 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_future_keyTrue if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.
DescribeLogDirs Response (Version: 2) => throttle_time_ms [results] _tagged_fields 
  throttle_time_ms => INT32
  results => error_code log_dir [topics] _tagged_fields 
    error_code => INT16
    log_dir => COMPACT_STRING
    topics => name [partitions] _tagged_fields 
      name => COMPACT_STRING
      partitions => partition_index partition_size offset_lag is_future_key _tagged_fields 
        partition_index => INT32
        partition_size => INT64
        offset_lag => INT64
        is_future_key => BOOLEAN

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe log directories.
error_codeThe error code, or 0 if there was no error.
log_dirThe absolute log directory path.
topicsThe topics.
nameThe topic name.
partitionsThe partitions.
partition_indexThe partition index.
partition_sizeThe size of the log segments in this 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_future_keyTrue if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeLogDirs Response (Version: 3) => throttle_time_ms error_code [results] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  results => error_code log_dir [topics] _tagged_fields 
    error_code => INT16
    log_dir => COMPACT_STRING
    topics => name [partitions] _tagged_fields 
      name => COMPACT_STRING
      partitions => partition_index partition_size offset_lag is_future_key _tagged_fields 
        partition_index => INT32
        partition_size => INT64
        offset_lag => INT64
        is_future_key => BOOLEAN

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
resultsThe log directories.
error_codeThe error code, or 0 if there was no error.
log_dirThe absolute log directory path.
topicsThe topics.
nameThe topic name.
partitionsThe partitions.
partition_indexThe partition index.
partition_sizeThe size of the log segments in this 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_future_keyTrue if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
SaslAuthenticate API (Key: 36):
Requests:
SaslAuthenticate Request (Version: 0) => auth_bytes 
  auth_bytes => BYTES

Request header version: 1

Field Description
auth_bytesThe SASL authentication bytes from the client, as defined by the SASL mechanism.
SaslAuthenticate Request (Version: 1) => auth_bytes 
  auth_bytes => BYTES

Request header version: 1

Field Description
auth_bytesThe SASL authentication bytes from the client, as defined by the SASL mechanism.
SaslAuthenticate Request (Version: 2) => auth_bytes _tagged_fields 
  auth_bytes => COMPACT_BYTES

Request header version: 2

Field Description
auth_bytesThe SASL authentication bytes from the client, as defined by the SASL mechanism.
_tagged_fieldsThe tagged fields
Responses:
SaslAuthenticate Response (Version: 0) => error_code error_message auth_bytes 
  error_code => INT16
  error_message => NULLABLE_STRING
  auth_bytes => BYTES

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
auth_bytesThe SASL authentication bytes from the server, as defined by the SASL mechanism.
SaslAuthenticate Response (Version: 1) => error_code error_message auth_bytes session_lifetime_ms 
  error_code => INT16
  error_message => NULLABLE_STRING
  auth_bytes => BYTES
  session_lifetime_ms => INT64

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
error_messageThe error message, or null if there was no error.
auth_bytesThe SASL authentication bytes from the server, as defined by the SASL mechanism.
session_lifetime_msNumber of milliseconds after which only re-authentication over the existing connection to create a new session can occur.
CreatePartitions API (Key: 37):
Requests:
CreatePartitions Request (Version: 0) => [topics] timeout_ms validate_only 
  topics => name count [assignments] 
    name => STRING
    count => INT32
    assignments => [broker_ids] 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 1

Field Description
topicsEach topic that we want to create new partitions inside.
nameThe topic name.
countThe new partition count.
assignmentsThe new partition assignments.
broker_idsThe assigned broker IDs.
timeout_msThe 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) => [topics] timeout_ms validate_only 
  topics => name count [assignments] 
    name => STRING
    count => INT32
    assignments => [broker_ids] 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 1

Field Description
topicsEach topic that we want to create new partitions inside.
nameThe topic name.
countThe new partition count.
assignmentsThe new partition assignments.
broker_idsThe assigned broker IDs.
timeout_msThe 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: 2) => [topics] timeout_ms validate_only _tagged_fields 
  topics => name count [assignments] _tagged_fields 
    name => COMPACT_STRING
    count => INT32
    assignments => [broker_ids] _tagged_fields 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 2

Field Description
topicsEach topic that we want to create new partitions inside.
nameThe topic name.
countThe new partition count.
assignmentsThe new partition assignments.
broker_idsThe assigned broker IDs.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msThe 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.
_tagged_fieldsThe tagged fields
CreatePartitions Request (Version: 3) => [topics] timeout_ms validate_only _tagged_fields 
  topics => name count [assignments] _tagged_fields 
    name => COMPACT_STRING
    count => INT32
    assignments => [broker_ids] _tagged_fields 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN

Request header version: 2

Field Description
topicsEach topic that we want to create new partitions inside.
nameThe topic name.
countThe new partition count.
assignmentsThe new partition assignments.
broker_idsThe assigned broker IDs.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
timeout_msThe 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.
_tagged_fieldsThe tagged fields
Responses:
CreatePartitions Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe partition creation results for each topic.
nameThe topic name.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
CreatePartitions Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe partition creation results for each topic.
nameThe topic name.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
CreatePartitions Response (Version: 2) => throttle_time_ms [results] _tagged_fields 
  throttle_time_ms => INT32
  results => name error_code error_message _tagged_fields 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe partition creation results for each topic.
nameThe topic name.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
CreateDelegationToken API (Key: 38):
Requests:
CreateDelegationToken Request (Version: 1) => [renewers] max_lifetime_ms 
  renewers => principal_type principal_name 
    principal_type => STRING
    principal_name => STRING
  max_lifetime_ms => INT64

Request header version: 1

Field Description
renewersA list of those who are allowed to renew this token before it expires.
principal_typeThe type of the Kafka principal.
principal_nameThe name of the Kafka principal.
max_lifetime_msThe maximum lifetime of the token in milliseconds, or -1 to use the server side default.
CreateDelegationToken Request (Version: 2) => [renewers] max_lifetime_ms _tagged_fields 
  renewers => principal_type principal_name _tagged_fields 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING
  max_lifetime_ms => INT64

Request header version: 2

Field Description
renewersA list of those who are allowed to renew this token before it expires.
principal_typeThe type of the Kafka principal.
principal_nameThe name of the Kafka principal.
_tagged_fieldsThe tagged fields
max_lifetime_msThe maximum lifetime of the token in milliseconds, or -1 to use the server side default.
_tagged_fieldsThe tagged fields
CreateDelegationToken Request (Version: 3) => owner_principal_type owner_principal_name [renewers] max_lifetime_ms _tagged_fields 
  owner_principal_type => COMPACT_NULLABLE_STRING
  owner_principal_name => COMPACT_NULLABLE_STRING
  renewers => principal_type principal_name _tagged_fields 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING
  max_lifetime_ms => INT64

Request header version: 2

Field Description
owner_principal_typeThe principal type of the owner of the token. If it's null it defaults to the token request principal.
owner_principal_nameThe principal name of the owner of the token. If it's null it defaults to the token request principal.
renewersA list of those who are allowed to renew this token before it expires.
principal_typeThe type of the Kafka principal.
principal_nameThe name of the Kafka principal.
_tagged_fieldsThe tagged fields
max_lifetime_msThe maximum lifetime of the token in milliseconds, or -1 to use the server side default.
_tagged_fieldsThe tagged fields
Responses:
CreateDelegationToken Response (Version: 1) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms 
  error_code => INT16
  principal_type => STRING
  principal_name => STRING
  issue_timestamp_ms => INT64
  expiry_timestamp_ms => INT64
  max_timestamp_ms => INT64
  token_id => STRING
  hmac => BYTES
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe top-level error, or zero if there was no error.
principal_typeThe principal type of the token owner.
principal_nameThe name of the token owner.
issue_timestamp_msWhen this token was generated.
expiry_timestamp_msWhen this token expires.
max_timestamp_msThe maximum lifetime of this token.
token_idThe token UUID.
hmacHMAC of the delegation token.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
CreateDelegationToken Response (Version: 2) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms _tagged_fields 
  error_code => INT16
  principal_type => COMPACT_STRING
  principal_name => COMPACT_STRING
  issue_timestamp_ms => INT64
  expiry_timestamp_ms => INT64
  max_timestamp_ms => INT64
  token_id => COMPACT_STRING
  hmac => COMPACT_BYTES
  throttle_time_ms => INT32

Response header version: 1

Field Description
error_codeThe top-level error, or zero if there was no error.
principal_typeThe principal type of the token owner.
principal_nameThe name of the token owner.
issue_timestamp_msWhen this token was generated.
expiry_timestamp_msWhen this token expires.
max_timestamp_msThe maximum lifetime of this token.
token_idThe token UUID.
hmacHMAC of the delegation token.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
_tagged_fieldsThe tagged fields
RenewDelegationToken API (Key: 39):
Requests:
RenewDelegationToken Request (Version: 1) => hmac renew_period_ms 
  hmac => BYTES
  renew_period_ms => INT64

Request header version: 1

Field Description
hmacThe HMAC of the delegation token to be renewed.
renew_period_msThe renewal time period in milliseconds.
RenewDelegationToken Request (Version: 2) => hmac renew_period_ms _tagged_fields 
  hmac => COMPACT_BYTES
  renew_period_ms => INT64

Request header version: 2

Field Description
hmacThe HMAC of the delegation token to be renewed.
renew_period_msThe renewal time period in milliseconds.
_tagged_fieldsThe tagged fields
Responses:
RenewDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
expiry_timestamp_msThe timestamp in milliseconds at which this token expires.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
ExpireDelegationToken API (Key: 40):
Requests:
ExpireDelegationToken Request (Version: 1) => hmac expiry_time_period_ms 
  hmac => BYTES
  expiry_time_period_ms => INT64

Request header version: 1

Field Description
hmacThe HMAC of the delegation token to be expired.
expiry_time_period_msThe expiry time period in milliseconds.
ExpireDelegationToken Request (Version: 2) => hmac expiry_time_period_ms _tagged_fields 
  hmac => COMPACT_BYTES
  expiry_time_period_ms => INT64

Request header version: 2

Field Description
hmacThe HMAC of the delegation token to be expired.
expiry_time_period_msThe expiry time period in milliseconds.
_tagged_fieldsThe tagged fields
Responses:
ExpireDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
expiry_timestamp_msThe timestamp in milliseconds at which this token expires.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
DescribeDelegationToken API (Key: 41):
Requests:
DescribeDelegationToken Request (Version: 1) => [owners] 
  owners => principal_type principal_name 
    principal_type => STRING
    principal_name => STRING

Request header version: 1

Field Description
ownersEach owner that we want to describe delegation tokens for, or null to describe all tokens.
principal_typeThe owner principal type.
principal_nameThe owner principal name.
DescribeDelegationToken Request (Version: 2) => [owners] _tagged_fields 
  owners => principal_type principal_name _tagged_fields 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING

Request header version: 2

Field Description
ownersEach owner that we want to describe delegation tokens for, or null to describe all tokens.
principal_typeThe owner principal type.
principal_nameThe owner principal name.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeDelegationToken Request (Version: 3) => [owners] _tagged_fields 
  owners => principal_type principal_name _tagged_fields 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING

Request header version: 2

Field Description
ownersEach owner that we want to describe delegation tokens for, or null to describe all tokens.
principal_typeThe owner principal type.
principal_nameThe owner principal name.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
DescribeDelegationToken Response (Version: 1) => error_code [tokens] throttle_time_ms 
  error_code => INT16
  tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] 
    principal_type => STRING
    principal_name => STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => STRING
    hmac => BYTES
    renewers => principal_type principal_name 
      principal_type => STRING
      principal_name => STRING
  throttle_time_ms => INT32

Response header version: 0

Field Description
error_codeThe error code, or 0 if there was no error.
tokensThe tokens.
principal_typeThe token principal type.
principal_nameThe token principal name.
issue_timestampThe token issue timestamp in milliseconds.
expiry_timestampThe token expiry timestamp in milliseconds.
max_timestampThe token maximum timestamp length in milliseconds.
token_idThe token ID.
hmacThe token HMAC.
renewersThose who are able to renew this token before it expires.
principal_typeThe renewer principal type.
principal_nameThe renewer principal name.
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
DescribeDelegationToken Response (Version: 2) => error_code [tokens] throttle_time_ms _tagged_fields 
  error_code => INT16
  tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] _tagged_fields 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => COMPACT_STRING
    hmac => COMPACT_BYTES
    renewers => principal_type principal_name _tagged_fields 
      principal_type => COMPACT_STRING
      principal_name => COMPACT_STRING
  throttle_time_ms => INT32

Response header version: 1

Field Description
error_codeThe error code, or 0 if there was no error.
tokensThe tokens.
principal_typeThe token principal type.
principal_nameThe token principal name.
issue_timestampThe token issue timestamp in milliseconds.
expiry_timestampThe token expiry timestamp in milliseconds.
max_timestampThe token maximum timestamp length in milliseconds.
token_idThe token ID.
hmacThe token HMAC.
renewersThose who are able to renew this token before it expires.
principal_typeThe renewer principal type.
principal_nameThe renewer principal name.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
_tagged_fieldsThe tagged fields
DeleteGroups API (Key: 42):
Requests:
DeleteGroups Request (Version: 0) => [groups_names] 
  groups_names => STRING

Request header version: 1

Field Description
groups_namesThe group names to delete.
DeleteGroups Request (Version: 1) => [groups_names] 
  groups_names => STRING

Request header version: 1

Field Description
groups_namesThe group names to delete.
DeleteGroups Request (Version: 2) => [groups_names] _tagged_fields 
  groups_names => COMPACT_STRING

Request header version: 2

Field Description
groups_namesThe group names to delete.
_tagged_fieldsThe tagged fields
Responses:
DeleteGroups Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => group_id error_code 
    group_id => STRING
    error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe deletion results.
group_idThe group id.
error_codeThe deletion error, or 0 if the deletion succeeded.
DeleteGroups Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => group_id error_code 
    group_id => STRING
    error_code => INT16

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
resultsThe deletion results.
group_idThe group id.
error_codeThe deletion error, or 0 if the deletion succeeded.
ElectLeaders API (Key: 43):
Requests:
ElectLeaders Request (Version: 0) => [topic_partitions] timeout_ms 
  topic_partitions => topic [partitions] 
    topic => STRING
    partitions => INT32
  timeout_ms => INT32

Request header version: 1

Field Description
topic_partitionsThe topic partitions to elect leaders.
topicThe name of a topic.
partitionsThe partitions of this topic whose leader should be elected.
timeout_msThe time in ms to wait for the election to complete.
ElectLeaders Request (Version: 1) => election_type [topic_partitions] timeout_ms 
  election_type => INT8
  topic_partitions => topic [partitions] 
    topic => STRING
    partitions => INT32
  timeout_ms => INT32

Request header version: 1

Field Description
election_typeType of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica.
topic_partitionsThe topic partitions to elect leaders.
topicThe name of a topic.
partitionsThe partitions of this topic whose leader should be elected.
timeout_msThe time in ms to wait for the election to complete.
ElectLeaders Request (Version: 2) => election_type [topic_partitions] timeout_ms _tagged_fields 
  election_type => INT8
  topic_partitions => topic [partitions] _tagged_fields 
    topic => COMPACT_STRING
    partitions => INT32
  timeout_ms => INT32

Request header version: 2

Field Description
election_typeType of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica.
topic_partitionsThe topic partitions to elect leaders.
topicThe name of a topic.
partitionsThe partitions of this topic whose leader should be elected.
_tagged_fieldsThe tagged fields
timeout_msThe time in ms to wait for the election to complete.
_tagged_fieldsThe tagged fields
Responses:
ElectLeaders Response (Version: 0) => throttle_time_ms [replica_election_results] 
  throttle_time_ms => INT32
  replica_election_results => topic [partition_result] 
    topic => STRING
    partition_result => partition_id error_code error_message 
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
replica_election_resultsThe election results, or an empty array if the requester did not have permission and the request asks for all partitions.
topicThe topic name.
partition_resultThe results for each partition.
partition_idThe partition id.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
ElectLeaders Response (Version: 1) => throttle_time_ms error_code [replica_election_results] 
  throttle_time_ms => INT32
  error_code => INT16
  replica_election_results => topic [partition_result] 
    topic => STRING
    partition_result => partition_id error_code error_message 
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top level response error code.
replica_election_resultsThe election results, or an empty array if the requester did not have permission and the request asks for all partitions.
topicThe topic name.
partition_resultThe results for each partition.
partition_idThe partition id.
error_codeThe result error, or zero if there was no error.
error_messageThe result message, or null if there was no error.
IncrementalAlterConfigs API (Key: 44):
Requests:
IncrementalAlterConfigs Request (Version: 0) => [resources] validate_only 
  resources => resource_type resource_name [configs] 
    resource_type => INT8
    resource_name => STRING
    configs => name config_operation value 
      name => STRING
      config_operation => INT8
      value => NULLABLE_STRING
  validate_only => BOOLEAN

Request header version: 1

Field Description
resourcesThe incremental updates for each resource.
resource_typeThe resource type.
resource_nameThe resource name.
configsThe configurations.
nameThe configuration key name.
config_operationThe type (Set, Delete, Append, Subtract) of operation.
valueThe value to set for the configuration key.
validate_onlyTrue if we should validate the request, but not change the configurations.
IncrementalAlterConfigs Request (Version: 1) => [resources] validate_only _tagged_fields 
  resources => resource_type resource_name [configs] _tagged_fields 
    resource_type => INT8
    resource_name => COMPACT_STRING
    configs => name config_operation value _tagged_fields 
      name => COMPACT_STRING
      config_operation => INT8
      value => COMPACT_NULLABLE_STRING
  validate_only => BOOLEAN

Request header version: 2

Field Description
resourcesThe incremental updates for each resource.
resource_typeThe resource type.
resource_nameThe resource name.
configsThe configurations.
nameThe configuration key name.
config_operationThe type (Set, Delete, Append, Subtract) of operation.
valueThe value to set for the configuration key.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
validate_onlyTrue if we should validate the request, but not change the configurations.
_tagged_fieldsThe tagged fields
Responses:
IncrementalAlterConfigs Response (Version: 0) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING

Response header version: 0

Field Description
throttle_time_msDuration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
responsesThe responses for each resource.
error_codeThe resource error code.
error_messageThe resource error message, or null if there was no error.
resource_typeThe resource type.
resource_nameThe resource name.
AlterPartitionReassignments API (Key: 45):
Requests:
AlterPartitionReassignments Request (Version: 0) => timeout_ms [topics] _tagged_fields 
  timeout_ms => INT32
  topics => name [partitions] _tagged_fields 
    name => COMPACT_STRING
    partitions => partition_index [replicas] _tagged_fields 
      partition_index => INT32
      replicas => INT32

Request header version: 2

Field Description
timeout_msThe time in ms to wait for the request to complete.
topicsThe topics to reassign.
nameThe topic name.
partitionsThe partitions to reassign.
partition_indexThe partition index.
replicasThe replicas to place the partitions on, or null to cancel a pending reassignment for this partition.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
ListPartitionReassignments API (Key: 46):
Requests:
ListPartitionReassignments Request (Version: 0) => timeout_ms [topics] _tagged_fields 
  timeout_ms => INT32
  topics => name [partition_indexes] _tagged_fields 
    name => COMPACT_STRING
    partition_indexes => INT32

Request header version: 2

Field Description
timeout_msThe time in ms to wait for the request to complete.
topicsThe topics to list partition reassignments for, or null to list everything.
nameThe topic name.
partition_indexesThe partitions to list partition reassignments for.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
OffsetDelete API (Key: 47):
Requests:
OffsetDelete Request (Version: 0) => group_id [topics] 
  group_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index 
      partition_index => INT32

Request header version: 1

Field Description
group_idThe unique group identifier.
topicsThe topics to delete offsets for.
nameThe topic name.
partitionsEach partition to delete offsets for.
partition_indexThe partition index.
Responses:
DescribeClientQuotas API (Key: 48):
Requests:
DescribeClientQuotas Request (Version: 0) => [components] strict 
  components => entity_type match_type match 
    entity_type => STRING
    match_type => INT8
    match => NULLABLE_STRING
  strict => BOOLEAN

Request header version: 1

Field Description
componentsFilter components to apply to quota entities.
entity_typeThe entity type that the filter component applies to.
match_typeHow to match the entity {0 = exact name, 1 = default name, 2 = any specified name}.
matchThe string to match against, or null if unused for the match type.
strictWhether the match is strict, i.e. should exclude entities with unspecified entity types.
DescribeClientQuotas Request (Version: 1) => [components] strict _tagged_fields 
  components => entity_type match_type match _tagged_fields 
    entity_type => COMPACT_STRING
    match_type => INT8
    match => COMPACT_NULLABLE_STRING
  strict => BOOLEAN

Request header version: 2

Field Description
componentsFilter components to apply to quota entities.
entity_typeThe entity type that the filter component applies to.
match_typeHow to match the entity {0 = exact name, 1 = default name, 2 = any specified name}.
matchThe string to match against, or null if unused for the match type.
_tagged_fieldsThe tagged fields
strictWhether the match is strict, i.e. should exclude entities with unspecified entity types.
_tagged_fieldsThe tagged fields
Responses:
DescribeClientQuotas Response (Version: 0) => throttle_time_ms error_code error_message [entries] 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  entries => [entity] [values] 
    entity => entity_type entity_name 
      entity_type => STRING
      entity_name => NULLABLE_STRING
    values => key value 
      key => STRING
      value => FLOAT64

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or `0` if the quota description succeeded.
error_messageThe error message, or `null` if the quota description succeeded.
entriesA result entry.
entityThe quota entity description.
entity_typeThe entity type.
entity_nameThe entity name, or null if the default.
valuesThe quota values for the entity.
keyThe quota configuration key.
valueThe quota configuration value.
AlterClientQuotas API (Key: 49):
Requests:
AlterClientQuotas Request (Version: 0) => [entries] validate_only 
  entries => [entity] [ops] 
    entity => entity_type entity_name 
      entity_type => STRING
      entity_name => NULLABLE_STRING
    ops => key value remove 
      key => STRING
      value => FLOAT64
      remove => BOOLEAN
  validate_only => BOOLEAN

Request header version: 1

Field Description
entriesThe quota configuration entries to alter.
entityThe quota entity to alter.
entity_typeThe entity type.
entity_nameThe name of the entity, or null if the default.
opsAn individual quota configuration entry to alter.
keyThe quota configuration key.
valueThe value to set, otherwise ignored if the value is to be removed.
removeWhether the quota configuration value should be removed, otherwise set.
validate_onlyWhether the alteration should be validated, but not performed.
AlterClientQuotas Request (Version: 1) => [entries] validate_only _tagged_fields 
  entries => [entity] [ops] _tagged_fields 
    entity => entity_type entity_name _tagged_fields 
      entity_type => COMPACT_STRING
      entity_name => COMPACT_NULLABLE_STRING
    ops => key value remove _tagged_fields 
      key => COMPACT_STRING
      value => FLOAT64
      remove => BOOLEAN
  validate_only => BOOLEAN

Request header version: 2

Field Description
entriesThe quota configuration entries to alter.
entityThe quota entity to alter.
entity_typeThe entity type.
entity_nameThe name of the entity, or null if the default.
_tagged_fieldsThe tagged fields
opsAn individual quota configuration entry to alter.
keyThe quota configuration key.
valueThe value to set, otherwise ignored if the value is to be removed.
removeWhether the quota configuration value should be removed, otherwise set.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
validate_onlyWhether the alteration should be validated, but not performed.
_tagged_fieldsThe tagged fields
Responses:
AlterClientQuotas Response (Version: 0) => throttle_time_ms [entries] 
  throttle_time_ms => INT32
  entries => error_code error_message [entity] 
    error_code => INT16
    error_message => NULLABLE_STRING
    entity => entity_type entity_name 
      entity_type => STRING
      entity_name => NULLABLE_STRING

Response header version: 0

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
entriesThe quota configuration entries to alter.
error_codeThe error code, or `0` if the quota alteration succeeded.
error_messageThe error message, or `null` if the quota alteration succeeded.
entityThe quota entity to alter.
entity_typeThe entity type.
entity_nameThe name of the entity, or null if the default.
DescribeUserScramCredentials API (Key: 50):
Requests:
DescribeUserScramCredentials Request (Version: 0) => [users] _tagged_fields 
  users => name _tagged_fields 
    name => COMPACT_STRING

Request header version: 2

Field Description
usersThe users to describe, or null/empty to describe all users.
nameThe user name.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
AlterUserScramCredentials API (Key: 51):
Requests:
AlterUserScramCredentials Request (Version: 0) => [deletions] [upsertions] _tagged_fields 
  deletions => name mechanism _tagged_fields 
    name => COMPACT_STRING
    mechanism => INT8
  upsertions => name mechanism iterations salt salted_password _tagged_fields 
    name => COMPACT_STRING
    mechanism => INT8
    iterations => INT32
    salt => COMPACT_BYTES
    salted_password => COMPACT_BYTES

Request header version: 2

Field Description
deletionsThe SCRAM credentials to remove.
nameThe user name.
mechanismThe SCRAM mechanism.
_tagged_fieldsThe tagged fields
upsertionsThe SCRAM credentials to update/insert.
nameThe user name.
mechanismThe SCRAM mechanism.
iterationsThe number of iterations.
saltA random salt generated by the client.
salted_passwordThe salted password.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
DescribeQuorum API (Key: 55):
Requests:
DescribeQuorum Request (Version: 0) => [topics] _tagged_fields 
  topics => topic_name [partitions] _tagged_fields 
    topic_name => COMPACT_STRING
    partitions => partition_index _tagged_fields 
      partition_index => INT32

Request header version: 2

Field Description
topicsThe topics to describe.
topic_nameThe topic name.
partitionsThe partitions to describe.
partition_indexThe partition index.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeQuorum Request (Version: 1) => [topics] _tagged_fields 
  topics => topic_name [partitions] _tagged_fields 
    topic_name => COMPACT_STRING
    partitions => partition_index _tagged_fields 
      partition_index => INT32

Request header version: 2

Field Description
topicsThe topics to describe.
topic_nameThe topic name.
partitionsThe partitions to describe.
partition_indexThe partition index.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeQuorum Request (Version: 2) => [topics] _tagged_fields 
  topics => topic_name [partitions] _tagged_fields 
    topic_name => COMPACT_STRING
    partitions => partition_index _tagged_fields 
      partition_index => INT32

Request header version: 2

Field Description
topicsThe topics to describe.
topic_nameThe topic name.
partitionsThe partitions to describe.
partition_indexThe partition index.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
DescribeQuorum Response (Version: 0) => error_code [topics] _tagged_fields 
  error_code => INT16
  topics => topic_name [partitions] _tagged_fields 
    topic_name => COMPACT_STRING
    partitions => partition_index error_code leader_id leader_epoch high_watermark [current_voters] [observers] _tagged_fields 
      partition_index => INT32
      error_code => INT16
      leader_id => INT32
      leader_epoch => INT32
      high_watermark => INT64
      current_voters => replica_id log_end_offset _tagged_fields 
        replica_id => INT32
        log_end_offset => INT64
      observers => replica_id log_end_offset _tagged_fields 
        replica_id => INT32
        log_end_offset => INT64

Response header version: 1

Field Description
error_codeThe top level error code.
topicsThe response from the describe quorum API.
topic_nameThe topic name.
partitionsThe partition data.
partition_indexThe partition index.
error_codeThe partition error code.
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
high_watermarkThe high water mark.
current_votersThe current voters of the partition.
replica_idThe ID of the replica.
log_end_offsetThe last known log end offset of the follower or -1 if it is unknown.
_tagged_fieldsThe tagged fields
observersThe observers of the partition.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeQuorum Response (Version: 1) => error_code [topics] _tagged_fields 
  error_code => INT16
  topics => topic_name [partitions] _tagged_fields 
    topic_name => COMPACT_STRING
    partitions => partition_index error_code leader_id leader_epoch high_watermark [current_voters] [observers] _tagged_fields 
      partition_index => INT32
      error_code => INT16
      leader_id => INT32
      leader_epoch => INT32
      high_watermark => INT64
      current_voters => replica_id log_end_offset last_fetch_timestamp last_caught_up_timestamp _tagged_fields 
        replica_id => INT32
        log_end_offset => INT64
        last_fetch_timestamp => INT64
        last_caught_up_timestamp => INT64
      observers => replica_id log_end_offset last_fetch_timestamp last_caught_up_timestamp _tagged_fields 
        replica_id => INT32
        log_end_offset => INT64
        last_fetch_timestamp => INT64
        last_caught_up_timestamp => INT64

Response header version: 1

Field Description
error_codeThe top level error code.
topicsThe response from the describe quorum API.
topic_nameThe topic name.
partitionsThe partition data.
partition_indexThe partition index.
error_codeThe partition error code.
leader_idThe ID of the current leader or -1 if the leader is unknown.
leader_epochThe latest known leader epoch.
high_watermarkThe high water mark.
current_votersThe current voters of the partition.
replica_idThe ID of the replica.
log_end_offsetThe last known log end offset of the follower or -1 if it is unknown.
last_fetch_timestampThe last known leader wall clock time time when a follower fetched from the leader. This is reported as -1 both for the current leader or if it is unknown for a voter.
last_caught_up_timestampThe leader wall clock append time of the offset for which the follower made the most recent fetch request. This is reported as the current time for the leader and -1 if unknown for a voter.
_tagged_fieldsThe tagged fields
observersThe observers of the partition.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
UpdateFeatures API (Key: 57):
Requests:
UpdateFeatures Request (Version: 0) => timeout_ms [feature_updates] _tagged_fields 
  timeout_ms => INT32
  feature_updates => feature max_version_level allow_downgrade _tagged_fields 
    feature => COMPACT_STRING
    max_version_level => INT16
    allow_downgrade => BOOLEAN

Request header version: 2

Field Description
timeout_msHow long to wait in milliseconds before timing out the request.
feature_updatesThe list of updates to finalized features.
featureThe name of the finalized feature to be updated.
max_version_levelThe new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature.
allow_downgradeDEPRECATED in version 1 (see DowngradeType). When set to true, the finalized feature version level is allowed to be downgraded/deleted. The downgrade request will fail if the new maximum version level is a value that's not lower than the existing maximum finalized version level.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
UpdateFeatures Request (Version: 1) => timeout_ms [feature_updates] validate_only _tagged_fields 
  timeout_ms => INT32
  feature_updates => feature max_version_level upgrade_type _tagged_fields 
    feature => COMPACT_STRING
    max_version_level => INT16
    upgrade_type => INT8
  validate_only => BOOLEAN

Request header version: 2

Field Description
timeout_msHow long to wait in milliseconds before timing out the request.
feature_updatesThe list of updates to finalized features.
featureThe name of the finalized feature to be updated.
max_version_levelThe new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature.
upgrade_typeDetermine which type of upgrade will be performed: 1 will perform an upgrade only (default), 2 is safe downgrades only (lossless), 3 is unsafe downgrades (lossy).
_tagged_fieldsThe tagged fields
validate_onlyTrue if we should validate the request, but not perform the upgrade or downgrade.
_tagged_fieldsThe tagged fields
UpdateFeatures Request (Version: 2) => timeout_ms [feature_updates] validate_only _tagged_fields 
  timeout_ms => INT32
  feature_updates => feature max_version_level upgrade_type _tagged_fields 
    feature => COMPACT_STRING
    max_version_level => INT16
    upgrade_type => INT8
  validate_only => BOOLEAN

Request header version: 2

Field Description
timeout_msHow long to wait in milliseconds before timing out the request.
feature_updatesThe list of updates to finalized features.
featureThe name of the finalized feature to be updated.
max_version_levelThe new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature.
upgrade_typeDetermine which type of upgrade will be performed: 1 will perform an upgrade only (default), 2 is safe downgrades only (lossless), 3 is unsafe downgrades (lossy).
_tagged_fieldsThe tagged fields
validate_onlyTrue if we should validate the request, but not perform the upgrade or downgrade.
_tagged_fieldsThe tagged fields
Responses:
UpdateFeatures Response (Version: 0) => throttle_time_ms error_code error_message [results] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  results => feature error_code error_message _tagged_fields 
    feature => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top-level error code, or `0` if there was no top-level error.
error_messageThe top-level error message, or `null` if there was no top-level error.
resultsResults for each feature update.
featureThe name of the finalized feature.
error_codeThe feature update error code or `0` if the feature update succeeded.
error_messageThe feature update error, or `null` if the feature update succeeded.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
UpdateFeatures Response (Version: 1) => throttle_time_ms error_code error_message [results] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  results => feature error_code error_message _tagged_fields 
    feature => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top-level error code, or `0` if there was no top-level error.
error_messageThe top-level error message, or `null` if there was no top-level error.
resultsResults for each feature update.
featureThe name of the finalized feature.
error_codeThe feature update error code or `0` if the feature update succeeded.
error_messageThe feature update error, or `null` if the feature update succeeded.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
DescribeCluster API (Key: 60):
Requests:
DescribeCluster Request (Version: 0) => include_cluster_authorized_operations _tagged_fields 
  include_cluster_authorized_operations => BOOLEAN

Request header version: 2

Field Description
include_cluster_authorized_operationsWhether to include cluster authorized operations.
_tagged_fieldsThe tagged fields
DescribeCluster Request (Version: 1) => include_cluster_authorized_operations endpoint_type _tagged_fields 
  include_cluster_authorized_operations => BOOLEAN
  endpoint_type => INT8

Request header version: 2

Field Description
include_cluster_authorized_operationsWhether to include cluster authorized operations.
endpoint_typeThe endpoint type to describe. 1=brokers, 2=controllers.
_tagged_fieldsThe tagged fields
DescribeCluster Request (Version: 2) => include_cluster_authorized_operations endpoint_type include_fenced_brokers _tagged_fields 
  include_cluster_authorized_operations => BOOLEAN
  endpoint_type => INT8
  include_fenced_brokers => BOOLEAN

Request header version: 2

Field Description
include_cluster_authorized_operationsWhether to include cluster authorized operations.
endpoint_typeThe endpoint type to describe. 1=brokers, 2=controllers.
include_fenced_brokersWhether to include fenced brokers when listing brokers.
_tagged_fieldsThe tagged fields
Responses:
DescribeCluster Response (Version: 0) => throttle_time_ms error_code error_message cluster_id controller_id [brokers] cluster_authorized_operations _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  cluster_id => COMPACT_STRING
  controller_id => INT32
  brokers => broker_id host port rack _tagged_fields 
    broker_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top-level error code, or 0 if there was no error.
error_messageThe top-level error message, or null if there was no error.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
brokersEach broker in the response.
broker_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
cluster_authorized_operations32-bit bitfield to represent authorized operations for this cluster.
_tagged_fieldsThe tagged fields
DescribeCluster Response (Version: 1) => throttle_time_ms error_code error_message endpoint_type cluster_id controller_id [brokers] cluster_authorized_operations _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  endpoint_type => INT8
  cluster_id => COMPACT_STRING
  controller_id => INT32
  brokers => broker_id host port rack _tagged_fields 
    broker_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top-level error code, or 0 if there was no error.
error_messageThe top-level error message, or null if there was no error.
endpoint_typeThe endpoint type that was described. 1=brokers, 2=controllers.
cluster_idThe cluster ID that responding broker belongs to.
controller_idThe ID of the controller broker.
brokersEach broker in the response.
broker_idThe broker ID.
hostThe broker hostname.
portThe broker port.
rackThe rack of the broker, or null if it has not been assigned to a rack.
_tagged_fieldsThe tagged fields
cluster_authorized_operations32-bit bitfield to represent authorized operations for this cluster.
_tagged_fieldsThe tagged fields
DescribeProducers API (Key: 61):
Requests:
DescribeProducers Request (Version: 0) => [topics] _tagged_fields 
  topics => name [partition_indexes] _tagged_fields 
    name => COMPACT_STRING
    partition_indexes => INT32

Request header version: 2

Field Description
topicsThe topics to list producers for.
nameThe topic name.
partition_indexesThe indexes of the partitions to list producers for.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
UnregisterBroker API (Key: 64):
Requests:
UnregisterBroker Request (Version: 0) => broker_id _tagged_fields 
  broker_id => INT32

Request header version: 2

Field Description
broker_idThe broker ID to unregister.
_tagged_fieldsThe tagged fields
Responses:
DescribeTransactions API (Key: 65):
Requests:
DescribeTransactions Request (Version: 0) => [transactional_ids] _tagged_fields 
  transactional_ids => COMPACT_STRING

Request header version: 2

Field Description
transactional_idsArray of transactionalIds to include in describe results. If empty, then no results will be returned.
_tagged_fieldsThe tagged fields
Responses:
ListTransactions API (Key: 66):
Requests:
ListTransactions Request (Version: 0) => [state_filters] [producer_id_filters] _tagged_fields 
  state_filters => COMPACT_STRING
  producer_id_filters => INT64

Request header version: 2

Field Description
state_filtersThe transaction states to filter by: if empty, all transactions are returned; if non-empty, then only transactions matching one of the filtered states will be returned.
producer_id_filtersThe producerIds to filter by: if empty, all transactions will be returned; if non-empty, only transactions which match one of the filtered producerIds will be returned.
_tagged_fieldsThe tagged fields
ListTransactions Request (Version: 1) => [state_filters] [producer_id_filters] duration_filter _tagged_fields 
  state_filters => COMPACT_STRING
  producer_id_filters => INT64
  duration_filter => INT64

Request header version: 2

Field Description
state_filtersThe transaction states to filter by: if empty, all transactions are returned; if non-empty, then only transactions matching one of the filtered states will be returned.
producer_id_filtersThe producerIds to filter by: if empty, all transactions will be returned; if non-empty, only transactions which match one of the filtered producerIds will be returned.
duration_filterDuration (in millis) to filter by: if < 0, all transactions will be returned; otherwise, only transactions running longer than this duration will be returned.
_tagged_fieldsThe tagged fields
Responses:
ListTransactions Response (Version: 0) => throttle_time_ms error_code [unknown_state_filters] [transaction_states] _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  unknown_state_filters => COMPACT_STRING
  transaction_states => transactional_id producer_id transaction_state _tagged_fields 
    transactional_id => COMPACT_STRING
    producer_id => INT64
    transaction_state => COMPACT_STRING

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe error code, or 0 if there was no error.
unknown_state_filtersSet of state filters provided in the request which were unknown to the transaction coordinator.
transaction_statesThe current state of the transaction for the transactional id.
transactional_idThe transactional id.
producer_idThe producer id.
transaction_stateThe current transaction state of the producer.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ConsumerGroupHeartbeat API (Key: 68):
Requests:
ConsumerGroupHeartbeat Request (Version: 0) => group_id member_id member_epoch instance_id rack_id rebalance_timeout_ms [subscribed_topic_names] server_assignor [topic_partitions] _tagged_fields 
  group_id => COMPACT_STRING
  member_id => COMPACT_STRING
  member_epoch => INT32
  instance_id => COMPACT_NULLABLE_STRING
  rack_id => COMPACT_NULLABLE_STRING
  rebalance_timeout_ms => INT32
  subscribed_topic_names => COMPACT_STRING
  server_assignor => COMPACT_NULLABLE_STRING
  topic_partitions => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32

Request header version: 2

Field Description
group_idThe group identifier.
member_idThe member id generated by the consumer. The member id must be kept during the entire lifetime of the consumer process.
member_epochThe current member epoch; 0 to join the group; -1 to leave the group; -2 to indicate that the static member will rejoin.
instance_idnull if not provided or if it didn't change since the last heartbeat; the instance Id otherwise.
rack_idnull if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise.
rebalance_timeout_ms-1 if it didn't change since the last heartbeat; the maximum time in milliseconds that the coordinator will wait on the member to revoke its partitions otherwise.
subscribed_topic_namesnull if it didn't change since the last heartbeat; the subscribed topic names otherwise.
server_assignornull if not used or if it didn't change since the last heartbeat; the server side assignor to use otherwise.
topic_partitionsnull if it didn't change since the last heartbeat; the partitions owned by the member.
topic_idThe topic ID.
partitionsThe partitions.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ConsumerGroupHeartbeat Request (Version: 1) => group_id member_id member_epoch instance_id rack_id rebalance_timeout_ms [subscribed_topic_names] subscribed_topic_regex server_assignor [topic_partitions] _tagged_fields 
  group_id => COMPACT_STRING
  member_id => COMPACT_STRING
  member_epoch => INT32
  instance_id => COMPACT_NULLABLE_STRING
  rack_id => COMPACT_NULLABLE_STRING
  rebalance_timeout_ms => INT32
  subscribed_topic_names => COMPACT_STRING
  subscribed_topic_regex => COMPACT_NULLABLE_STRING
  server_assignor => COMPACT_NULLABLE_STRING
  topic_partitions => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32

Request header version: 2

Field Description
group_idThe group identifier.
member_idThe member id generated by the consumer. The member id must be kept during the entire lifetime of the consumer process.
member_epochThe current member epoch; 0 to join the group; -1 to leave the group; -2 to indicate that the static member will rejoin.
instance_idnull if not provided or if it didn't change since the last heartbeat; the instance Id otherwise.
rack_idnull if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise.
rebalance_timeout_ms-1 if it didn't change since the last heartbeat; the maximum time in milliseconds that the coordinator will wait on the member to revoke its partitions otherwise.
subscribed_topic_namesnull if it didn't change since the last heartbeat; the subscribed topic names otherwise.
subscribed_topic_regexnull if it didn't change since the last heartbeat; the subscribed topic regex otherwise.
server_assignornull if not used or if it didn't change since the last heartbeat; the server side assignor to use otherwise.
topic_partitionsnull if it didn't change since the last heartbeat; the partitions owned by the member.
topic_idThe topic ID.
partitionsThe partitions.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
ConsumerGroupHeartbeat Response (Version: 0) => throttle_time_ms error_code error_message member_id member_epoch heartbeat_interval_ms assignment _tagged_fields 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  member_id => COMPACT_NULLABLE_STRING
  member_epoch => INT32
  heartbeat_interval_ms => INT32
  assignment => [topic_partitions] _tagged_fields 
    topic_partitions => topic_id [partitions] _tagged_fields 
      topic_id => UUID
      partitions => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
error_codeThe top-level error code, or 0 if there was no error.
error_messageThe top-level error message, or null if there was no error.
member_idThe member id is generated by the consumer starting from version 1, while in version 0, it can be provided by users or generated by the group coordinator.
member_epochThe member epoch.
heartbeat_interval_msThe heartbeat interval in milliseconds.
assignmentnull if not provided; the assignment otherwise.
topic_partitionsThe partitions assigned to the member that can be used immediately.
topic_idThe topic ID.
partitionsThe partitions.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
ConsumerGroupDescribe API (Key: 69):
Requests:
ConsumerGroupDescribe Request (Version: 0) => [group_ids] include_authorized_operations _tagged_fields 
  group_ids => COMPACT_STRING
  include_authorized_operations => BOOLEAN

Request header version: 2

Field Description
group_idsThe ids of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
_tagged_fieldsThe tagged fields
ConsumerGroupDescribe Request (Version: 1) => [group_ids] include_authorized_operations _tagged_fields 
  group_ids => COMPACT_STRING
  include_authorized_operations => BOOLEAN

Request header version: 2

Field Description
group_idsThe ids of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
_tagged_fieldsThe tagged fields
Responses:
ConsumerGroupDescribe Response (Version: 0) => throttle_time_ms [groups] _tagged_fields 
  throttle_time_ms => INT32
  groups => error_code error_message group_id group_state group_epoch assignment_epoch assignor_name [members] authorized_operations _tagged_fields 
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    group_id => COMPACT_STRING
    group_state => COMPACT_STRING
    group_epoch => INT32
    assignment_epoch => INT32
    assignor_name => COMPACT_STRING
    members => member_id instance_id rack_id member_epoch client_id client_host [subscribed_topic_names] subscribed_topic_regex assignment target_assignment _tagged_fields 
      member_id => COMPACT_STRING
      instance_id => COMPACT_NULLABLE_STRING
      rack_id => COMPACT_NULLABLE_STRING
      member_epoch => INT32
      client_id => COMPACT_STRING
      client_host => COMPACT_STRING
      subscribed_topic_names => COMPACT_STRING
      subscribed_topic_regex => COMPACT_NULLABLE_STRING
      assignment => [topic_partitions] _tagged_fields 
        topic_partitions => topic_id topic_name [partitions] _tagged_fields 
          topic_id => UUID
          topic_name => COMPACT_STRING
          partitions => INT32
      target_assignment => [topic_partitions] _tagged_fields 
        topic_partitions => topic_id topic_name [partitions] _tagged_fields 
          topic_id => UUID
          topic_name => COMPACT_STRING
          partitions => INT32
    authorized_operations => INT32

Response header version: 1

Field Description
throttle_time_msThe duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
groupsEach described group.
error_codeThe describe error, or 0 if there was no error.
error_messageThe top-level error message, or null if there was no error.
group_idThe group ID string.
group_stateThe group state string, or the empty string.
group_epochThe group epoch.
assignment_epochThe assignment epoch.
assignor_nameThe selected assignor.
membersThe members.
member_idThe member ID.
instance_idThe member instance ID.
rack_idThe member rack ID.
member_epochThe current member epoch.
client_idThe client ID.
client_hostThe client host.
subscribed_topic_namesThe subscribed topic names.
subscribed_topic_regexthe subscribed topic regex otherwise or null of not provided.
assignmentThe current assignment.
topic_partitionsThe assigned topic-partitions to the member.
topic_idThe topic ID.
topic_nameThe topic name.
partitionsThe partitions.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
target_assignmentThe target assignment.
_tagged_fieldsThe tagged fields
authorized_operations32-bit bitfield to represent authorized operations for this group.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
GetTelemetrySubscriptions API (Key: 71):
Requests:
GetTelemetrySubscriptions Request (Version: 0) => client_instance_id _tagged_fields 
  client_instance_id => UUID

Request header version: 2

Field Description
client_instance_idUnique id for this client instance, must be set to 0 on the first request.
_tagged_fieldsThe tagged fields
Responses:
PushTelemetry API (Key: 72):
Requests:
PushTelemetry Request (Version: 0) => client_instance_id subscription_id terminating compression_type metrics _tagged_fields 
  client_instance_id => UUID
  subscription_id => INT32
  terminating => BOOLEAN
  compression_type => INT8
  metrics => COMPACT_BYTES

Request header version: 2

Field Description
client_instance_idUnique id for this client instance.
subscription_idUnique identifier for the current subscription.
terminatingClient is terminating the connection.
compression_typeCompression codec used to compress the metrics.
metricsMetrics encoded in OpenTelemetry MetricsData v1 protobuf format.
_tagged_fieldsThe tagged fields
Responses:
ListClientMetricsResources API (Key: 74):
Requests:
ListClientMetricsResources Request (Version: 0) => _tagged_fields 

Request header version: 2

Field Description
_tagged_fieldsThe tagged fields
Responses:
DescribeTopicPartitions API (Key: 75):
Requests:
DescribeTopicPartitions Request (Version: 0) => [topics] response_partition_limit cursor _tagged_fields 
  topics => name _tagged_fields 
    name => COMPACT_STRING
  response_partition_limit => INT32
  cursor => topic_name partition_index _tagged_fields 
    topic_name => COMPACT_STRING
    partition_index => INT32

Request header version: 2

Field Description
topicsThe topics to fetch details for.
nameThe topic name.
_tagged_fieldsThe tagged fields
response_partition_limitThe maximum number of partitions included in the response.
cursorThe first topic and partition index to fetch details for.
topic_nameThe name for the first topic to process.
partition_indexThe partition index to start with.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
ShareGroupHeartbeat API (Key: 76):
Requests:
ShareGroupHeartbeat Request (Version: 0) => group_id member_id member_epoch rack_id [subscribed_topic_names] _tagged_fields 
  group_id => COMPACT_STRING
  member_id => COMPACT_STRING
  member_epoch => INT32
  rack_id => COMPACT_NULLABLE_STRING
  subscribed_topic_names => COMPACT_STRING

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
member_idThe member id.
member_epochThe current member epoch; 0 to join the group; -1 to leave the group.
rack_idnull if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise.
subscribed_topic_namesnull if it didn't change since the last heartbeat; the subscribed topic names otherwise.
_tagged_fieldsThe tagged fields
Responses:
ShareGroupDescribe API (Key: 77):
Requests:
ShareGroupDescribe Request (Version: 0) => [group_ids] include_authorized_operations _tagged_fields 
  group_ids => COMPACT_STRING
  include_authorized_operations => BOOLEAN

This version of the request is unstable.

Request header version: 2

Field Description
group_idsThe ids of the groups to describe.
include_authorized_operationsWhether to include authorized operations.
_tagged_fieldsThe tagged fields
Responses:
ShareFetch API (Key: 78):
Requests:
ShareFetch Request (Version: 0) => group_id member_id share_session_epoch max_wait_ms min_bytes max_bytes [topics] [forgotten_topics_data] _tagged_fields 
  group_id => COMPACT_NULLABLE_STRING
  member_id => COMPACT_NULLABLE_STRING
  share_session_epoch => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition_index partition_max_bytes [acknowledgement_batches] _tagged_fields 
      partition_index => INT32
      partition_max_bytes => INT32
      acknowledgement_batches => first_offset last_offset [acknowledge_types] _tagged_fields 
        first_offset => INT64
        last_offset => INT64
        acknowledge_types => INT8
  forgotten_topics_data => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => INT32

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
member_idThe member ID.
share_session_epochThe current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
topicsThe topics to fetch.
topic_idThe unique topic ID.
partitionsThe partitions to fetch.
partition_indexThe partition index.
partition_max_bytesThe maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored.
acknowledgement_batchesRecord batches to acknowledge.
first_offsetFirst offset of batch of records to acknowledge.
last_offsetLast offset (inclusive) of batch of records to acknowledge.
acknowledge_typesArray of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
forgotten_topics_dataThe partitions to remove from this share session.
topic_idThe unique topic ID.
partitionsThe partitions indexes to forget.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
ShareAcknowledge API (Key: 79):
Requests:
ShareAcknowledge Request (Version: 0) => group_id member_id share_session_epoch [topics] _tagged_fields 
  group_id => COMPACT_NULLABLE_STRING
  member_id => COMPACT_NULLABLE_STRING
  share_session_epoch => INT32
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition_index [acknowledgement_batches] _tagged_fields 
      partition_index => INT32
      acknowledgement_batches => first_offset last_offset [acknowledge_types] _tagged_fields 
        first_offset => INT64
        last_offset => INT64
        acknowledge_types => INT8

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
member_idThe member ID.
share_session_epochThe current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests.
topicsThe topics containing records to acknowledge.
topic_idThe unique topic ID.
partitionsThe partitions containing records to acknowledge.
partition_indexThe partition index.
acknowledgement_batchesRecord batches to acknowledge.
first_offsetFirst offset of batch of records to acknowledge.
last_offsetLast offset (inclusive) of batch of records to acknowledge.
acknowledge_typesArray of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
AddRaftVoter API (Key: 80):
Requests:
AddRaftVoter Request (Version: 0) => cluster_id timeout_ms voter_id voter_directory_id [listeners] _tagged_fields 
  cluster_id => COMPACT_NULLABLE_STRING
  timeout_ms => INT32
  voter_id => INT32
  voter_directory_id => UUID
  listeners => name host port _tagged_fields 
    name => COMPACT_STRING
    host => COMPACT_STRING
    port => UINT16

Request header version: 2

Field Description
cluster_idThe cluster id.
timeout_msThe maximum time to wait for the request to complete before returning.
voter_idThe replica id of the voter getting added to the topic partition.
voter_directory_idThe directory id of the voter getting added to the topic partition.
listenersThe endpoints that can be used to communicate with the voter.
nameThe name of the endpoint.
hostThe hostname.
portThe port.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
RemoveRaftVoter API (Key: 81):
Requests:
RemoveRaftVoter Request (Version: 0) => cluster_id voter_id voter_directory_id _tagged_fields 
  cluster_id => COMPACT_NULLABLE_STRING
  voter_id => INT32
  voter_directory_id => UUID

Request header version: 2

Field Description
cluster_idThe cluster id of the request.
voter_idThe replica id of the voter getting removed from the topic partition.
voter_directory_idThe directory id of the voter getting removed from the topic partition.
_tagged_fieldsThe tagged fields
Responses:
InitializeShareGroupState API (Key: 83):
Requests:
InitializeShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition state_epoch start_offset _tagged_fields 
      partition => INT32
      state_epoch => INT32
      start_offset => INT64

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
topicsThe data for the topics.
topic_idThe topic identifier.
partitionsThe data for the partitions.
partitionThe partition index.
state_epochThe state epoch for this share-partition.
start_offsetThe share-partition start offset, or -1 if the start offset is not being initialized.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
ReadShareGroupState API (Key: 84):
Requests:
ReadShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition leader_epoch _tagged_fields 
      partition => INT32
      leader_epoch => INT32

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
topicsThe data for the topics.
topic_idThe topic identifier.
partitionsThe data for the partitions.
partitionThe partition index.
leader_epochThe leader epoch of the share-partition.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
WriteShareGroupState API (Key: 85):
Requests:
WriteShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition state_epoch leader_epoch start_offset [state_batches] _tagged_fields 
      partition => INT32
      state_epoch => INT32
      leader_epoch => INT32
      start_offset => INT64
      state_batches => first_offset last_offset delivery_state delivery_count _tagged_fields 
        first_offset => INT64
        last_offset => INT64
        delivery_state => INT8
        delivery_count => INT16

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
topicsThe data for the topics.
topic_idThe topic identifier.
partitionsThe data for the partitions.
partitionThe partition index.
state_epochThe state epoch for this share-partition.
leader_epochThe leader epoch of the share-partition.
start_offsetThe share-partition start offset, or -1 if the start offset is not being written.
state_batchesThe state batches for the share-partition.
first_offsetThe base offset of this state batch.
last_offsetThe last offset of this state batch.
delivery_stateThe state - 0:Available,2:Acked,4:Archived.
delivery_countThe delivery count.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
DeleteShareGroupState API (Key: 86):
Requests:
DeleteShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition _tagged_fields 
      partition => INT32

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
topicsThe data for the topics.
topic_idThe topic identifier.
partitionsThe data for the partitions.
partitionThe partition index.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:
ReadShareGroupStateSummary API (Key: 87):
Requests:
ReadShareGroupStateSummary Request (Version: 0) => group_id [topics] _tagged_fields 
  group_id => COMPACT_STRING
  topics => topic_id [partitions] _tagged_fields 
    topic_id => UUID
    partitions => partition leader_epoch _tagged_fields 
      partition => INT32
      leader_epoch => INT32

This version of the request is unstable.

Request header version: 2

Field Description
group_idThe group identifier.
topicsThe data for the topics.
topic_idThe topic identifier.
partitionsThe data for the partitions.
partitionThe partition index.
leader_epochThe leader epoch of the share-partition.
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
_tagged_fieldsThe tagged fields
Responses:

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.